You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@datasketches.apache.org by le...@apache.org on 2020/06/26 21:11:59 UTC

[incubator-datasketches-java] branch Refactor_Theta_Tuple created (now bc77a08)

This is an automated email from the ASF dual-hosted git repository.

leerho pushed a change to branch Refactor_Theta_Tuple
in repository https://gitbox.apache.org/repos/asf/incubator-datasketches-java.git.


      at bc77a08  Interim changes, all tests work.

This branch includes the following new commits:

     new 3a79cec  Current evolution of setOp and compact changes
     new dbd0350  Interim 2
     new bc77a08  Interim changes, all tests work.

The 3 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



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


[incubator-datasketches-java] 02/03: Interim 2

Posted by le...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

leerho pushed a commit to branch Refactor_Theta_Tuple
in repository https://gitbox.apache.org/repos/asf/incubator-datasketches-java.git

commit dbd0350647d7a1497955c4fc09cb304b2cd4dd06
Author: Lee Rhodes <le...@users.noreply.github.com>
AuthorDate: Mon Jun 22 14:00:53 2020 -0700

    Interim 2
---
 .../org/apache/datasketches/theta/AnotBimpl.java   |  10 +-
 .../datasketches/theta/CompactOperations.java      | 237 +++++++++++++++++++++
 .../apache/datasketches/theta/CompactSketch.java   | 163 +++++---------
 .../theta/DirectCompactOrderedSketch.java          |  17 +-
 .../datasketches/theta/DirectCompactSketch.java    |  56 ++++-
 .../theta/DirectCompactUnorderedSketch.java        |  16 +-
 .../theta/DirectQuickSelectSketchR.java            |   2 +-
 .../datasketches/theta/EmptyCompactSketch.java     |  19 ++
 .../apache/datasketches/theta/HeapAlphaSketch.java |   2 +-
 .../theta/HeapCompactOrderedSketch.java            |  40 ++--
 .../datasketches/theta/HeapCompactSketch.java      |   4 +-
 .../theta/HeapCompactUnorderedSketch.java          |  13 ++
 .../datasketches/theta/HeapQuickSelectSketch.java  |   2 +-
 .../datasketches/theta/IntersectionImplR.java      |  34 ++-
 .../datasketches/theta/SingleItemSketch.java       |  51 +++--
 .../java/org/apache/datasketches/theta/Sketch.java |  28 +--
 .../org/apache/datasketches/theta/UnionImpl.java   |   3 +-
 .../apache/datasketches/theta/UpdateSketch.java    |  20 +-
 src/main/javadoc/resources/dictionary.html         |   2 +-
 .../datasketches/theta/CompactSketchTest.java      |   2 +-
 .../datasketches/theta/SetOpsCornerCasesTest.java  |  16 +-
 .../org/apache/datasketches/theta/SketchTest.java  |   6 +-
 22 files changed, 522 insertions(+), 221 deletions(-)

diff --git a/src/main/java/org/apache/datasketches/theta/AnotBimpl.java b/src/main/java/org/apache/datasketches/theta/AnotBimpl.java
index 797c1de..702ae4f 100644
--- a/src/main/java/org/apache/datasketches/theta/AnotBimpl.java
+++ b/src/main/java/org/apache/datasketches/theta/AnotBimpl.java
@@ -26,8 +26,6 @@ import static org.apache.datasketches.HashOperations.hashSearch;
 import static org.apache.datasketches.Util.REBUILD_THRESHOLD;
 import static org.apache.datasketches.Util.checkSeedHashes;
 import static org.apache.datasketches.Util.simpleIntLog2;
-import static org.apache.datasketches.theta.CompactSketch.compactCache;
-import static org.apache.datasketches.theta.CompactSketch.loadCompactMemory;
 import static org.apache.datasketches.theta.PreambleUtil.COMPACT_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.ORDERED_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.READ_ONLY_FLAG_MASK;
@@ -184,15 +182,15 @@ final class AnotBimpl extends AnotB {
         final SingleItemSketch sis = new SingleItemSketch(hashArr[0], seedHash);
         dstMem.putByteArray(0, sis.toByteArray(), 0, 16);
       }
-      final int preLongs = Sketch.computeCompactPreLongs(thetaLong, false, curCount);
+      final int preLongs = CompactOperations.computeCompactPreLongs(thetaLong, false, curCount);
       if (dstOrdered) {
         final byte flags = (byte)(READ_ONLY_FLAG_MASK | COMPACT_FLAG_MASK | ORDERED_FLAG_MASK);
         Arrays.sort(hashArr);
-        loadCompactMemory(hashArr, seedHash, curCount, thetaLong, dstMem, flags, preLongs);
+        CompactOperations.loadCompactMemory(hashArr, seedHash, curCount, thetaLong, dstMem, flags, preLongs);
         result = new DirectCompactOrderedSketch(dstMem);
       } else {
         final byte flags = (byte)(READ_ONLY_FLAG_MASK | COMPACT_FLAG_MASK);
-        loadCompactMemory(hashArr, seedHash, curCount, thetaLong, dstMem, flags, preLongs);
+        CompactOperations.loadCompactMemory(hashArr, seedHash, curCount, thetaLong, dstMem, flags, preLongs);
         result = new DirectCompactUnorderedSketch(dstMem);
       }
     }
@@ -411,7 +409,7 @@ final class AnotBimpl extends AnotB {
         thetaLongR_ = skA_.getThetaLong();
         emptyR_ = false;
         curCountR_ = skA_.getRetainedEntries(true);
-        hashArrR_ = compactCache(skA_.getCache(), curCountR_, thetaLongR_, false);
+        hashArrR_ = CompactOperations.compactCache(skA_.getCache(), curCountR_, thetaLongR_, false);
         break;
 
       case 18:   //A Compact, B Compact; CheckAB, B -> H; => C,H; scanAllAsearchB()
diff --git a/src/main/java/org/apache/datasketches/theta/CompactOperations.java b/src/main/java/org/apache/datasketches/theta/CompactOperations.java
new file mode 100644
index 0000000..47030fd
--- /dev/null
+++ b/src/main/java/org/apache/datasketches/theta/CompactOperations.java
@@ -0,0 +1,237 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches.theta;
+
+import static org.apache.datasketches.theta.PreambleUtil.COMPACT_FLAG_MASK;
+import static org.apache.datasketches.theta.PreambleUtil.EMPTY_FLAG_MASK;
+import static org.apache.datasketches.theta.PreambleUtil.ORDERED_FLAG_MASK;
+import static org.apache.datasketches.theta.PreambleUtil.READ_ONLY_FLAG_MASK;
+import static org.apache.datasketches.theta.PreambleUtil.SER_VER;
+import static org.apache.datasketches.theta.PreambleUtil.SINGLEITEM_FLAG_MASK;
+import static org.apache.datasketches.theta.PreambleUtil.extractCurCount;
+import static org.apache.datasketches.theta.PreambleUtil.extractFamilyID;
+import static org.apache.datasketches.theta.PreambleUtil.extractFlags;
+import static org.apache.datasketches.theta.PreambleUtil.extractLgArrLongs;
+import static org.apache.datasketches.theta.PreambleUtil.extractPreLongs;
+import static org.apache.datasketches.theta.PreambleUtil.extractSeedHash;
+import static org.apache.datasketches.theta.PreambleUtil.extractSerVer;
+import static org.apache.datasketches.theta.PreambleUtil.extractThetaLong;
+import static org.apache.datasketches.theta.PreambleUtil.insertCurCount;
+import static org.apache.datasketches.theta.PreambleUtil.insertFamilyID;
+import static org.apache.datasketches.theta.PreambleUtil.insertFlags;
+import static org.apache.datasketches.theta.PreambleUtil.insertP;
+import static org.apache.datasketches.theta.PreambleUtil.insertPreLongs;
+import static org.apache.datasketches.theta.PreambleUtil.insertSeedHash;
+import static org.apache.datasketches.theta.PreambleUtil.insertSerVer;
+import static org.apache.datasketches.theta.PreambleUtil.insertThetaLong;
+
+import java.util.Arrays;
+
+import org.apache.datasketches.Family;
+import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+
+/**
+ * @author Lee Rhodes
+ */
+final class CompactOperations {
+
+  private CompactOperations() {}
+
+  static CompactSketch componentsToCompact( //No error checking
+      final long thetaLong,
+      final int curCount,
+      final short seedHash,
+      final boolean srcCompact,
+      boolean srcOrdered,
+      final boolean dstOrdered,
+      final WritableMemory dstMem,
+      final long[] hashArr) //may not be compacted, ordered or unordered
+
+  {
+    final boolean direct = dstMem != null;
+    final boolean empty = (curCount == 0) && (thetaLong == Long.MAX_VALUE);
+    final boolean single = (curCount == 1) && (thetaLong == Long.MAX_VALUE);
+    final long[] hashArrOut;
+    if (!srcCompact) {
+      hashArrOut = CompactOperations.compactCache(hashArr, curCount, thetaLong, dstOrdered);
+      srcOrdered = true;
+    } else {
+      hashArrOut = hashArr;
+    }
+    if (!srcOrdered && dstOrdered && !empty && !single) {
+      Arrays.sort(hashArrOut);
+    }
+    if (direct) {
+      final int preLongs = computeCompactPreLongs(thetaLong, empty, curCount);
+      int flags = READ_ONLY_FLAG_MASK | COMPACT_FLAG_MASK; //always LE
+      flags |=  empty ? EMPTY_FLAG_MASK : 0;
+      flags |= dstOrdered ? ORDERED_FLAG_MASK : 0;
+      flags |= single ? SINGLEITEM_FLAG_MASK : 0;
+      final Memory mem =
+          loadCompactMemory(hashArr, seedHash, curCount, thetaLong, dstMem, (byte)flags, preLongs);
+      if (dstOrdered) {
+        return new DirectCompactOrderedSketch(mem);
+      } else {
+        return new DirectCompactUnorderedSketch(mem);
+      }
+    } else { //Heap
+      if (empty) {
+        return EmptyCompactSketch.getInstance();
+      }
+      if (single) {
+        return new SingleItemSketch(hashArrOut[0], seedHash);
+      }
+      if (dstOrdered) {
+        return new HeapCompactOrderedSketch(hashArrOut, empty, seedHash, curCount, thetaLong);
+      } else {
+        return new HeapCompactUnorderedSketch(hashArrOut, empty, seedHash, curCount, thetaLong);
+      }
+    }
+  }
+
+  @SuppressWarnings("unused")
+  static CompactSketch memoryToCompact(
+      final Memory srcMem,
+      final boolean dstOrdered,
+      final WritableMemory dstMem)
+  {
+    //extract Pre0 fields
+    final int preLongs = extractPreLongs(srcMem);
+    final int serVer = extractSerVer(srcMem);
+    final int famId = extractFamilyID(srcMem);
+    final int lgArrLongs = extractLgArrLongs(srcMem);
+    final int flags = extractFlags(srcMem);
+    final short seedHash = (short) extractSeedHash(srcMem);
+
+
+
+    final int curCount = extractCurCount(srcMem);
+    final long thetaLong = extractThetaLong(srcMem);
+
+    final boolean empty = (flags & EMPTY_FLAG_MASK) > 0;
+    final boolean srcCompact = (flags & COMPACT_FLAG_MASK) > 0;
+    final boolean srcOrdered = (flags & ORDERED_FLAG_MASK) > 0;
+    final boolean single = (flags & SINGLEITEM_FLAG_MASK) > 0;
+    if (!srcOrdered) {
+
+    }
+    final long[] hashArr ;
+
+    //do checks ...
+    final boolean direct = dstMem != null;
+    if (empty)  { assert (curCount == 0) && (thetaLong == Long.MAX_VALUE); }
+    if (single) { assert (curCount == 1) && (thetaLong == Long.MAX_VALUE); }
+    if (direct) {
+
+    } else { //heap
+      //dispatch empty and single
+      //dispatch other
+    }
+    return null;
+  }
+
+  //All arguments must be valid and correct including flags.
+  // Used as helper to create byte arrays as well as loading Memory for direct compact sketches
+  static final Memory loadCompactMemory(
+      final long[] compactHashArr,
+      final short seedHash,
+      final int curCount,
+      final long thetaLong,
+      final WritableMemory dstMem,
+      final byte flags,
+      final int preLongs)
+  {
+    assert (dstMem != null) && (compactHashArr != null);
+    final int outLongs = preLongs + curCount;
+    final int outBytes = outLongs << 3;
+    final int dstBytes = (int) dstMem.getCapacity();
+    if (outBytes > dstBytes) {
+      throw new SketchesArgumentException("Insufficient Memory: " + dstBytes
+        + ", Need: " + outBytes);
+    }
+    final byte famID = (byte) Family.COMPACT.getID();
+
+    insertPreLongs(dstMem, preLongs); //RF not used = 0
+    insertSerVer(dstMem, SER_VER);
+    insertFamilyID(dstMem, famID);
+    //ignore lgNomLongs, lgArrLongs bytes for compact sketches
+    insertFlags(dstMem, flags);
+    insertSeedHash(dstMem, seedHash);
+
+    if ((preLongs == 1) && (curCount == 1)) { //singleItem, theta = 1.0
+      dstMem.putLong(8, compactHashArr[0]);
+      return dstMem;
+    }
+    if (preLongs > 1) {
+      insertCurCount(dstMem, curCount);
+      insertP(dstMem, (float) 1.0);
+    }
+    if (preLongs > 2) {
+      insertThetaLong(dstMem, thetaLong);
+    }
+    if (curCount > 0) { //theta could be < 1.0.
+      dstMem.putLongArray(preLongs << 3, compactHashArr, 0, curCount);
+    }
+    return dstMem; //curCount == 0, theta could be < 1.0
+  }
+
+  static final int computeCompactPreLongs(final long thetaLong, final boolean empty,
+      final int curCount) {
+    return (thetaLong < Long.MAX_VALUE) ? 3 : empty ? 1 : (curCount > 1) ? 2 : 1;
+  }
+
+  /**
+   * Compact the given array. The source cache can be a hash table with interstitial zeros or
+   * "dirty" values, which are hash values greater than theta. These can be generated by the
+   * Alpha sketch.
+   * @param srcCache anything
+   * @param curCount must be correct
+   * @param thetaLong The correct
+   * <a href="{@docRoot}/resources/dictionary.html#thetaLong">thetaLong</a>.
+   * @param dstOrdered true if output array must be sorted
+   * @return the compacted array
+   */
+  static final long[] compactCache(final long[] srcCache, final int curCount,
+      final long thetaLong, final boolean dstOrdered) {
+    if (curCount == 0) {
+      return new long[0];
+    }
+    final long[] cacheOut = new long[curCount];
+    final int len = srcCache.length;
+    int j = 0;
+    for (int i = 0; i < len; i++) { //scan the full srcCache
+      final long v = srcCache[i];
+      if ((v <= 0L) || (v >= thetaLong) ) { continue; } //ignoring zeros or dirty values
+      cacheOut[j++] = v;
+    }
+    assert curCount == j;
+    if (dstOrdered && (curCount > 1)) {
+      Arrays.sort(cacheOut);
+    }
+    return cacheOut;
+  }
+
+
+
+
+}
+
diff --git a/src/main/java/org/apache/datasketches/theta/CompactSketch.java b/src/main/java/org/apache/datasketches/theta/CompactSketch.java
index 92510a2..dd667d6 100644
--- a/src/main/java/org/apache/datasketches/theta/CompactSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/CompactSketch.java
@@ -19,20 +19,18 @@
 
 package org.apache.datasketches.theta;
 
-import static org.apache.datasketches.theta.PreambleUtil.SER_VER;
-import static org.apache.datasketches.theta.PreambleUtil.insertCurCount;
-import static org.apache.datasketches.theta.PreambleUtil.insertFamilyID;
-import static org.apache.datasketches.theta.PreambleUtil.insertFlags;
-import static org.apache.datasketches.theta.PreambleUtil.insertP;
-import static org.apache.datasketches.theta.PreambleUtil.insertPreLongs;
-import static org.apache.datasketches.theta.PreambleUtil.insertSeedHash;
-import static org.apache.datasketches.theta.PreambleUtil.insertSerVer;
-import static org.apache.datasketches.theta.PreambleUtil.insertThetaLong;
+import static org.apache.datasketches.theta.PreambleUtil.COMPACT_FLAG_MASK;
+import static org.apache.datasketches.theta.PreambleUtil.EMPTY_FLAG_MASK;
+import static org.apache.datasketches.theta.PreambleUtil.SINGLEITEM_FLAG_MASK;
+import static org.apache.datasketches.theta.PreambleUtil.extractCurCount;
+import static org.apache.datasketches.theta.PreambleUtil.extractFlags;
+import static org.apache.datasketches.theta.PreambleUtil.extractLgArrLongs;
+import static org.apache.datasketches.theta.PreambleUtil.extractPreLongs;
+import static org.apache.datasketches.theta.PreambleUtil.extractThetaLong;
 
 import java.util.Arrays;
 
 import org.apache.datasketches.Family;
-import org.apache.datasketches.SketchesArgumentException;
 import org.apache.datasketches.memory.Memory;
 import org.apache.datasketches.memory.WritableMemory;
 
@@ -54,12 +52,10 @@ public abstract class CompactSketch extends Sketch {
   //Sketch
 
   @Override
-  public CompactSketch compact() { return this; }
+  public abstract CompactSketch compact();
 
   @Override
-  public CompactSketch compact(final boolean dstOrdered, final WritableMemory dstMem) {
-    return this;
-  }
+  public abstract CompactSketch compact(final boolean dstOrdered, final WritableMemory dstMem);
 
   @Override
   public Family getFamily() {
@@ -74,105 +70,52 @@ public abstract class CompactSketch extends Sketch {
   //restricted methods
 
   /**
-   * Compact the given array. The source cache can be a hash table with interstitial zeros or
-   * "dirty" values, which are hash values greater than theta. These can be generated by the
-   * Alpha sketch.
-   * @param srcCache anything
-   * @param curCount must be correct
-   * @param thetaLong The correct
-   * <a href="{@docRoot}/resources/dictionary.html#thetaLong">thetaLong</a>.
-   * @param dstOrdered true if output array must be sorted
-   * @return the compacted array
-   */
-  static final long[] compactCache(final long[] srcCache, final int curCount,
-      final long thetaLong, final boolean dstOrdered) {
-    if (curCount == 0) {
-      return new long[0];
-    }
-    final long[] cacheOut = new long[curCount];
-    final int len = srcCache.length;
-    int j = 0;
-    for (int i = 0; i < len; i++) { //scan the full srcCache
-      final long v = srcCache[i];
-      if ((v <= 0L) || (v >= thetaLong) ) { continue; } //ignoring zeros or dirty values
-      cacheOut[j++] = v;
-    }
-    assert curCount == j;
-    if (dstOrdered && (curCount > 1)) {
-      Arrays.sort(cacheOut);
-    }
-    return cacheOut;
-  }
-
-  /**
-   * Compact first 2^lgArrLongs of given array
-   * @param srcCache anything
-   * @param lgArrLongs The correct
-   * <a href="{@docRoot}/resources/dictionary.html#lgArrLongs">lgArrLongs</a>.
-   * @param curCount must be correct
-   * @param thetaLong The correct
-   * <a href="{@docRoot}/resources/dictionary.html#thetaLong">thetaLong</a>.
-   * @param dstOrdered true if output array must be sorted
-   * @return the compacted array
+   * Heapifies the given source Memory with seedHash. We assume that the destination sketch type has
+   * been determined to be Compact and that the memory image is valid and the seedHash is correct.
+   * @param srcMem <a href="{@docRoot}/resources/dictionary.html#mem">See Memory</a>
+   * @param seedHash <a href="{@docRoot}/resources/dictionary.html#seedHash">See Seed Hash</a>.
+   * @return a CompactSketch
    */
-  static final long[] compactCachePart(final long[] srcCache, final int lgArrLongs,
-      final int curCount, final long thetaLong, final boolean dstOrdered) {
-    if (curCount == 0) {
-      return new long[0];
-    }
-    final long[] cacheOut = new long[curCount];
-    final int len = 1 << lgArrLongs;
-    int j = 0;
-    for (int i = 0; i < len; i++) {
-      final long v = srcCache[i];
-      if ((v <= 0L) || (v >= thetaLong) ) { continue; }
-      cacheOut[j++] = v;
-    }
-    assert curCount == j;
-    if (dstOrdered) {
-      Arrays.sort(cacheOut);
-    }
-    return cacheOut;
-  }
-
-  //All arguments must be valid and correct.
-  // Used as helper to create byte arrays as well as loading Memory for direct compact sketches
-  static final Memory loadCompactMemory(final long[] compactCache, final short seedHash,
-      final int curCount, final long thetaLong, final WritableMemory dstMem,
-      final byte flags, final int preLongs) {
-
-    assert (dstMem != null) && (compactCache != null);
-    final int outLongs = preLongs + curCount;
-    final int outBytes = outLongs << 3;
-    final int dstBytes = (int) dstMem.getCapacity();
-    if (outBytes > dstBytes) {
-      throw new SketchesArgumentException("Insufficient Memory: " + dstBytes
-        + ", Need: " + outBytes);
-    }
-    final byte famID = (byte) Family.COMPACT.getID();
-
-    insertPreLongs(dstMem, preLongs); //RF not used = 0
-    insertSerVer(dstMem, SER_VER);
-    insertFamilyID(dstMem, famID);
-    //ignore lgNomLongs, lgArrLongs bytes for compact sketches
-    insertFlags(dstMem, flags);
-    insertSeedHash(dstMem, seedHash);
-
-    if ((preLongs == 1) && (curCount == 1)) { //singleItem
-      dstMem.putLong(8, compactCache[0]);
-      return dstMem;
-    }
-    if (preLongs > 1) {
-      insertCurCount(dstMem, curCount);
-      insertP(dstMem, (float) 1.0);
-    }
-    if (preLongs > 2) {
-      insertThetaLong(dstMem, thetaLong);
+  static CompactSketch You(final Memory srcMem, final short seedHash,
+      final boolean dstOrdered) {
+    final int flags = extractFlags(srcMem);
+    final boolean empty = (flags & EMPTY_FLAG_MASK) > 0;
+    if (empty) { return EmptyCompactSketch.getInstance(); }
+    //EMPTY FLAG is FALSE
+    final int preLongs = extractPreLongs(srcMem);
+    final boolean single = (flags & SINGLEITEM_FLAG_MASK) > 0;
+
+    if (preLongs == 1) {
+      if (single) {
+        return new SingleItemSketch(srcMem.getLong(8), seedHash);
+      } else {
+        return EmptyCompactSketch.getInstance();
+      }
     }
-    if (curCount > 0) {
-      dstMem.putLongArray(preLongs << 3, compactCache, 0, curCount);
+    //preLongs > 1
+    final int curCount = extractCurCount(srcMem);
+    final long thetaLong = (preLongs > 2) ? extractThetaLong(srcMem) : Long.MAX_VALUE;
+    final boolean srcCompact = (flags & COMPACT_FLAG_MASK) > 0;
+    final long[] hashArrOut;
+    if (srcCompact) {
+      if ((curCount == 0) && (thetaLong == Long.MAX_VALUE)) {
+        return EmptyCompactSketch.getInstance();
+      }
+      if ((curCount == 1) && (thetaLong == Long.MAX_VALUE)) {
+        //TODO
+      }
+      hashArrOut = new long[curCount];
+      srcMem.getLongArray(8 * preLongs, hashArrOut, 0, curCount);
+      if (dstOrdered) { Arrays.sort(hashArrOut); }
+    } else { //src is hashTable
+      final int lgArrLongs = extractLgArrLongs(srcMem);
+      final long[] hashArr = new long[1 << lgArrLongs];
+      srcMem.getLongArray(8 * preLongs, hashArr, 0, 1 << lgArrLongs);
+      hashArrOut = CompactOperations.compactCache(hashArr, curCount, thetaLong, dstOrdered);
     }
-    return dstMem;
+    return dstOrdered
+        ? new HeapCompactOrderedSketch(hashArrOut, empty, seedHash, curCount, thetaLong)
+        : new HeapCompactUnorderedSketch(hashArrOut, empty, seedHash, curCount, thetaLong);
   }
 
 }
diff --git a/src/main/java/org/apache/datasketches/theta/DirectCompactOrderedSketch.java b/src/main/java/org/apache/datasketches/theta/DirectCompactOrderedSketch.java
index 27f6451..2fc662b 100644
--- a/src/main/java/org/apache/datasketches/theta/DirectCompactOrderedSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/DirectCompactOrderedSketch.java
@@ -77,14 +77,27 @@ final class DirectCompactOrderedSketch extends DirectCompactSketch {
       final int curCount,
       final long thetaLong,
       final WritableMemory dstMem) {
-    final int preLongs = computeCompactPreLongs(thetaLong, empty, curCount);
+    final int preLongs = CompactOperations.computeCompactPreLongs(thetaLong, empty, curCount);
     int flags = READ_ONLY_FLAG_MASK | COMPACT_FLAG_MASK | ORDERED_FLAG_MASK;
     flags |= empty ? EMPTY_FLAG_MASK : 0;
     flags |= (curCount == 1) ? SINGLEITEM_FLAG_MASK : 0;
-    loadCompactMemory(cache, seedHash, curCount, thetaLong, dstMem, (byte)flags, preLongs);
+    CompactOperations.loadCompactMemory(cache, seedHash, curCount, thetaLong, dstMem, (byte)flags, preLongs);
     return new DirectCompactOrderedSketch(dstMem);
   }
 
+  @Override //ordered, on-heap
+  public CompactSketch compact() {
+    //TODO
+    return null;
+  }
+
+  @Override
+  public CompactSketch compact(final boolean dstOrdered, final WritableMemory wmem) {
+    //TODO
+    return null;
+  }
+
+
   @Override
   public boolean isOrdered() {
     return true;
diff --git a/src/main/java/org/apache/datasketches/theta/DirectCompactSketch.java b/src/main/java/org/apache/datasketches/theta/DirectCompactSketch.java
index 4bfb514..bc62a09 100644
--- a/src/main/java/org/apache/datasketches/theta/DirectCompactSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/DirectCompactSketch.java
@@ -40,6 +40,60 @@ abstract class DirectCompactSketch extends CompactSketch {
 
   //Sketch
 
+//  @Override //ordered, on-heap
+//  public CompactSketch compact() {
+//    //TODO
+//    return null;
+//  }
+//
+//  @Override
+//  public CompactSketch compact(final boolean dstOrdered, final WritableMemory wmem) {
+//    final int srcFlags = extractFlags(mem_);
+//    final boolean srcOrdered = (srcFlags & ORDERED_FLAG_MASK) > 0;
+//    final int srcPreLongs = extractPreLongs(mem_);
+//    final int srcCurCount = (srcPreLongs == 1) ? 0 : extractCurCount(mem_);
+//    final long srcThetaLong = (srcPreLongs <= 2) ? Long.MAX_VALUE : extractThetaLong(mem_);
+//    final int bytes = (srcPreLongs + srcCurCount) << 3;
+//    if (srcCurCount == 0) {
+//      if (srcThetaLong == Long.MAX_VALUE) {
+//        //this sets the ordered to true independent of the dstOrdered request
+//        return EmptyCompactSketch.getInstance().compact(true, wmem);
+//      } else {
+//        assert srcPreLongs == 3 : "Theta < 1.0, thus PreLong must be 3: " + srcPreLongs;
+//        mem_.copyTo(0, wmem, 0, srcPreLongs << 3);
+//        if (dstOrdered) {
+//          return new DirectCompactOrderedSketch(wmem);
+//        } else {
+//          return new DirectCompactUnorderedSketch(wmem);
+//        }
+//      }
+//    }
+//    if (srcCurCount == 1) {
+//      if (srcThetaLong == Long.MAX_VALUE) {
+//        //TODO
+//      }
+//    }
+//    if (!srcOrdered && dstOrdered) { //sort this src mem and place in wmem
+//      if (srcCurCount == 0) {
+//        final long thetaLong = extractThetaLong(mem_);
+//        if (thetaLong == Long.MAX_VALUE) {
+//          //TODO
+//        }
+//      } else {
+//        final byte[] srcBytes = new byte[bytes];
+//        mem_.getByteArray(0, srcBytes, 0, bytes);
+//        wmem.putByteArray(0, srcBytes, 0, bytes);
+//        final byte dstFlags = (byte) (srcFlags & ORDERED_FLAG_MASK);
+//        wmem.putByte(FLAGS_BYTE, dstFlags);
+//      }
+//
+//    } else {
+//      mem_.copyTo(0, wmem, 0, bytes);
+//    }
+//
+//    return null;  //TODO
+//  }
+
   //overidden by EmptyCompactSketch and SingleItemSketch
   @Override
   public int getCurrentBytes(final boolean compact) { //compact is ignored here
@@ -95,7 +149,7 @@ abstract class DirectCompactSketch extends CompactSketch {
     return new MemoryHashIterator(mem_, getRetainedEntries(), getThetaLong());
   }
 
-  @Override
+  @Override //order is already determined.
   public byte[] toByteArray() {
     final int curCount = getRetainedEntries(true);
     Sketch.checkIllegalCurCountAndEmpty(isEmpty(), curCount);
diff --git a/src/main/java/org/apache/datasketches/theta/DirectCompactUnorderedSketch.java b/src/main/java/org/apache/datasketches/theta/DirectCompactUnorderedSketch.java
index b1bc10c..f8743ef 100644
--- a/src/main/java/org/apache/datasketches/theta/DirectCompactUnorderedSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/DirectCompactUnorderedSketch.java
@@ -76,14 +76,26 @@ final class DirectCompactUnorderedSketch extends DirectCompactSketch {
       final int curCount,
       final long thetaLong,
       final WritableMemory dstMem) {
-    final int preLongs = computeCompactPreLongs(thetaLong, empty, curCount);
+    final int preLongs = CompactOperations.computeCompactPreLongs(thetaLong, empty, curCount);
     int flags = READ_ONLY_FLAG_MASK | COMPACT_FLAG_MASK;
     flags |= empty ? EMPTY_FLAG_MASK : 0;
     flags |= (curCount == 1) ? SINGLEITEM_FLAG_MASK : 0;
-    loadCompactMemory(cache, seedHash, curCount, thetaLong, dstMem, (byte)flags, preLongs);
+    CompactOperations.loadCompactMemory(cache, seedHash, curCount, thetaLong, dstMem, (byte)flags, preLongs);
     return new DirectCompactUnorderedSketch(dstMem);
   }
 
+  @Override //ordered, on-heap
+  public CompactSketch compact() {
+    //TODO
+    return null;
+  }
+
+  @Override
+  public CompactSketch compact(final boolean dstOrdered, final WritableMemory wmem) {
+    //TODO
+    return null;
+  }
+
   @Override
   public boolean isOrdered() {
     return false;
diff --git a/src/main/java/org/apache/datasketches/theta/DirectQuickSelectSketchR.java b/src/main/java/org/apache/datasketches/theta/DirectQuickSelectSketchR.java
index 52cc536..7900451 100644
--- a/src/main/java/org/apache/datasketches/theta/DirectQuickSelectSketchR.java
+++ b/src/main/java/org/apache/datasketches/theta/DirectQuickSelectSketchR.java
@@ -220,7 +220,7 @@ class DirectQuickSelectSketchR extends UpdateSketch {
   @Override
   int getCurrentPreambleLongs(final boolean compact) {
     if (!compact) { return PreambleUtil.extractPreLongs(wmem_); }
-    return computeCompactPreLongs(getThetaLong(), isEmpty(), getRetainedEntries(true));
+    return CompactOperations.computeCompactPreLongs(getThetaLong(), isEmpty(), getRetainedEntries(true));
   }
 
   @Override
diff --git a/src/main/java/org/apache/datasketches/theta/EmptyCompactSketch.java b/src/main/java/org/apache/datasketches/theta/EmptyCompactSketch.java
index 81d80f1..666a9cd 100644
--- a/src/main/java/org/apache/datasketches/theta/EmptyCompactSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/EmptyCompactSketch.java
@@ -21,6 +21,7 @@ package org.apache.datasketches.theta;
 
 import org.apache.datasketches.SketchesArgumentException;
 import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
 
 /**
  * Singleton empty CompactSketch.
@@ -44,6 +45,7 @@ final class EmptyCompactSketch extends CompactSketch {
     return EMPTY_COMPACT_SKETCH;
   }
 
+  //This should be a heapify
   static EmptyCompactSketch getInstance(final Memory srcMem) {
     final long pre0 = srcMem.getLong(0);
     if (testCandidatePre0(pre0)) {
@@ -54,6 +56,23 @@ final class EmptyCompactSketch extends CompactSketch {
         + "Memory Pre0: " + maskedPre0 + ", required Pre0: " +  EMPTY_SKETCH_MASK);
   }
 
+  @Override
+  // This returns with ordered flag = true independent of dstOrdered.
+  // This is required for fast detection.
+  // The hashSeed is ignored and set == 0.
+  public CompactSketch compact() {
+    return EmptyCompactSketch.getInstance();
+  }
+
+  @Override
+  // This returns with ordered flag = true independent of dstOrdered.
+  // This is required for fast detection.
+  // The hashSeed is ignored and set == 0.
+  public CompactSketch compact(final boolean dstOrdered, final WritableMemory wmem) {
+    wmem.putByteArray(0, EMPTY_COMPACT_SKETCH_ARR, 0, 8);
+    return new DirectCompactOrderedSketch(wmem);
+  }
+
   //static
 
   static boolean testCandidatePre0(final long candidate) {
diff --git a/src/main/java/org/apache/datasketches/theta/HeapAlphaSketch.java b/src/main/java/org/apache/datasketches/theta/HeapAlphaSketch.java
index 78ba811..32a73c4 100644
--- a/src/main/java/org/apache/datasketches/theta/HeapAlphaSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/HeapAlphaSketch.java
@@ -269,7 +269,7 @@ final class HeapAlphaSketch extends HeapUpdateSketch {
   @Override
   int getCurrentPreambleLongs(final boolean compact) {
     if (!compact) { return Family.ALPHA.getMinPreLongs(); }
-    return computeCompactPreLongs(thetaLong_, empty_, curCount_);
+    return CompactOperations.computeCompactPreLongs(thetaLong_, empty_, curCount_);
   }
 
   @Override
diff --git a/src/main/java/org/apache/datasketches/theta/HeapCompactOrderedSketch.java b/src/main/java/org/apache/datasketches/theta/HeapCompactOrderedSketch.java
index f359381..48c38fb 100644
--- a/src/main/java/org/apache/datasketches/theta/HeapCompactOrderedSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/HeapCompactOrderedSketch.java
@@ -19,12 +19,7 @@
 
 package org.apache.datasketches.theta;
 
-import static org.apache.datasketches.theta.PreambleUtil.checkMemorySeedHash;
-import static org.apache.datasketches.theta.PreambleUtil.extractCurCount;
-import static org.apache.datasketches.theta.PreambleUtil.extractPreLongs;
-import static org.apache.datasketches.theta.PreambleUtil.extractThetaLong;
-
-import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
 
 /**
  * An on-heap, compact, ordered, read-only sketch.
@@ -48,30 +43,19 @@ final class HeapCompactOrderedSketch extends HeapCompactSketch {
     super(cache, empty, seedHash, curCount, thetaLong);
   }
 
-  /**
-   * Heapifies the given source Memory with seed
-   * @param srcMem <a href="{@docRoot}/resources/dictionary.html#mem">See Memory</a>
-   * @param seed <a href="{@docRoot}/resources/dictionary.html#seed">See Update Hash Seed</a>.
-   * @return a CompactSketch
-   */
-  //Note Empty and SingleItemSketches should be filtered out before we get here.
-  static CompactSketch heapifyInstance(final Memory srcMem, final long seed) {
-    final short memSeedHash = checkMemorySeedHash(srcMem, seed);
-    final int preLongs = extractPreLongs(srcMem);
-    final boolean empty = PreambleUtil.isEmptySketch(srcMem);
-    long thetaLong = Long.MAX_VALUE;
-    final int curCount = extractCurCount(srcMem);
-    final long[] cache = new long[curCount];
-    if (preLongs == 2) {
-      srcMem.getLongArray(16, cache, 0, curCount);
-    } else { //preLongs == 3
-      srcMem.getLongArray(24, cache, 0, curCount);
-      thetaLong = extractThetaLong(srcMem);
-    }
-    return new HeapCompactOrderedSketch(cache, empty, memSeedHash, curCount, thetaLong);
+  //Sketch interface
+
+  @Override //ordered, on-heap
+  public CompactSketch compact() {
+    //TODO
+    return null;
   }
 
-  //Sketch interface
+  @Override
+  public CompactSketch compact(final boolean dstOrdered, final WritableMemory wmem) {
+    //TODO
+    return null;
+  }
 
   @Override
   public byte[] toByteArray() {
diff --git a/src/main/java/org/apache/datasketches/theta/HeapCompactSketch.java b/src/main/java/org/apache/datasketches/theta/HeapCompactSketch.java
index fc5d4d5..b78002d 100644
--- a/src/main/java/org/apache/datasketches/theta/HeapCompactSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/HeapCompactSketch.java
@@ -57,7 +57,7 @@ abstract class HeapCompactSketch extends CompactSketch {
     curCount_ = empty ? 0 : curCount;
     thetaLong_ = empty ? Long.MAX_VALUE : thetaLong;
     cache_ = cache;
-    preLongs_ = computeCompactPreLongs(thetaLong, empty, curCount);
+    preLongs_ = CompactOperations.computeCompactPreLongs(thetaLong, empty, curCount);
   }
 
   //Sketch
@@ -133,7 +133,7 @@ abstract class HeapCompactSketch extends CompactSketch {
     final int orderedBit = ordered ? (byte) ORDERED_FLAG_MASK : 0;
     final byte flags = (byte) (emptyBit |  READ_ONLY_FLAG_MASK | COMPACT_FLAG_MASK | orderedBit);
     final int preLongs = getCurrentPreambleLongs(true);
-    loadCompactMemory(getCache(), getSeedHash(), getRetainedEntries(true), getThetaLong(),
+    CompactOperations.loadCompactMemory(getCache(), getSeedHash(), getRetainedEntries(true), getThetaLong(),
         dstMem, flags, preLongs);
     return byteArray;
   }
diff --git a/src/main/java/org/apache/datasketches/theta/HeapCompactUnorderedSketch.java b/src/main/java/org/apache/datasketches/theta/HeapCompactUnorderedSketch.java
index d1cd085..4fc35c3 100644
--- a/src/main/java/org/apache/datasketches/theta/HeapCompactUnorderedSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/HeapCompactUnorderedSketch.java
@@ -25,6 +25,7 @@ import static org.apache.datasketches.theta.PreambleUtil.extractPreLongs;
 import static org.apache.datasketches.theta.PreambleUtil.extractThetaLong;
 
 import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
 
 /**
  * An on-heap, compact, unordered, read-only sketch.
@@ -73,6 +74,18 @@ final class HeapCompactUnorderedSketch extends HeapCompactSketch {
 
   //Sketch interface
 
+  @Override //ordered, on-heap
+  public CompactSketch compact() {
+    //TODO
+    return null;
+  }
+
+  @Override
+  public CompactSketch compact(final boolean dstOrdered, final WritableMemory wmem) {
+    //TODO
+    return null;
+  }
+
   @Override
   public byte[] toByteArray() {
     return toByteArray(false); //unordered
diff --git a/src/main/java/org/apache/datasketches/theta/HeapQuickSelectSketch.java b/src/main/java/org/apache/datasketches/theta/HeapQuickSelectSketch.java
index c549018..7fe3e9e 100644
--- a/src/main/java/org/apache/datasketches/theta/HeapQuickSelectSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/HeapQuickSelectSketch.java
@@ -215,7 +215,7 @@ class HeapQuickSelectSketch extends HeapUpdateSketch {
   @Override
   int getCurrentPreambleLongs(final boolean compact) {
     if (!compact) { return preambleLongs_; }
-    return computeCompactPreLongs(thetaLong_, empty_, curCount_);
+    return CompactOperations.computeCompactPreLongs(thetaLong_, empty_, curCount_);
   }
 
   //only used by ConcurrentHeapThetaBuffer & Test
diff --git a/src/main/java/org/apache/datasketches/theta/IntersectionImplR.java b/src/main/java/org/apache/datasketches/theta/IntersectionImplR.java
index c05a05f..0cc7517 100644
--- a/src/main/java/org/apache/datasketches/theta/IntersectionImplR.java
+++ b/src/main/java/org/apache/datasketches/theta/IntersectionImplR.java
@@ -21,7 +21,6 @@ package org.apache.datasketches.theta;
 
 import static org.apache.datasketches.Util.MIN_LG_ARR_LONGS;
 import static org.apache.datasketches.Util.floorPowerOf2;
-import static org.apache.datasketches.theta.CompactSketch.compactCachePart;
 import static org.apache.datasketches.theta.PreambleUtil.EMPTY_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.FAMILY_BYTE;
 import static org.apache.datasketches.theta.PreambleUtil.FLAGS_BYTE;
@@ -35,6 +34,8 @@ import static org.apache.datasketches.theta.PreambleUtil.SER_VER;
 import static org.apache.datasketches.theta.PreambleUtil.SER_VER_BYTE;
 import static org.apache.datasketches.theta.PreambleUtil.THETA_LONG;
 
+import java.util.Arrays;
+
 import org.apache.datasketches.Family;
 import org.apache.datasketches.SketchesArgumentException;
 import org.apache.datasketches.SketchesReadOnlyException;
@@ -300,4 +301,35 @@ class IntersectionImplR extends Intersection {
     return maxLgArrLongs;
   }
 
+  /**
+   * Compact first 2^lgArrLongs of given array
+   * @param srcCache anything
+   * @param lgArrLongs The correct
+   * <a href="{@docRoot}/resources/dictionary.html#lgArrLongs">lgArrLongs</a>.
+   * @param curCount must be correct
+   * @param thetaLong The correct
+   * <a href="{@docRoot}/resources/dictionary.html#thetaLong">thetaLong</a>.
+   * @param dstOrdered true if output array must be sorted
+   * @return the compacted array
+   */ //Only used in IntersectionImplR
+  static final long[] compactCachePart(final long[] srcCache, final int lgArrLongs,
+      final int curCount, final long thetaLong, final boolean dstOrdered) {
+    if (curCount == 0) {
+      return new long[0];
+    }
+    final long[] cacheOut = new long[curCount];
+    final int len = 1 << lgArrLongs;
+    int j = 0;
+    for (int i = 0; i < len; i++) {
+      final long v = srcCache[i];
+      if ((v <= 0L) || (v >= thetaLong) ) { continue; }
+      cacheOut[j++] = v;
+    }
+    assert curCount == j;
+    if (dstOrdered) {
+      Arrays.sort(cacheOut);
+    }
+    return cacheOut;
+  }
+
 }
diff --git a/src/main/java/org/apache/datasketches/theta/SingleItemSketch.java b/src/main/java/org/apache/datasketches/theta/SingleItemSketch.java
index 28702ea..38bf862 100644
--- a/src/main/java/org/apache/datasketches/theta/SingleItemSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/SingleItemSketch.java
@@ -29,13 +29,14 @@ import static org.apache.datasketches.theta.PreambleUtil.isSingleItemSketch;
 
 import org.apache.datasketches.SketchesArgumentException;
 import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
 
 /**
  * A CompactSketch that holds only one item hash.
  *
  * @author Lee Rhodes
  */
-public final class SingleItemSketch extends CompactSketch {
+final class SingleItemSketch extends CompactSketch {
   private static final long DEFAULT_SEED_HASH = computeSeedHash(DEFAULT_UPDATE_SEED) & 0xFFFFL;
 
   // For backward compatibility, a candidate pre0_ long must have:
@@ -46,11 +47,9 @@ public final class SingleItemSketch extends CompactSketch {
   // and the hash seed matches, it is virtually guaranteed that we have a SingleItem Sketch.
 
   private static final long PRE0_LO6_SI   = 0X00_00_3A_00_00_03_03_01L; //with SI flag
-
   private long pre0_ = 0;
   private long hash_ = 0;
 
-
   //Internal Constructor. All checking & hashing has been done, assumes default seed
   private SingleItemSketch(final long hash) {
     pre0_ = (DEFAULT_SEED_HASH << 48) | PRE0_LO6_SI;
@@ -76,7 +75,7 @@ public final class SingleItemSketch extends CompactSketch {
    * DEFAULT_UPDATE_SEED.
    * @param srcMem the Memory to be heapified.  It must be a least 16 bytes.
    * @return a SingleItemSketch
-   */
+   */ //does not override Sketch
   public static SingleItemSketch heapify(final Memory srcMem) {
     return heapify(srcMem, DEFAULT_UPDATE_SEED);
   }
@@ -87,7 +86,7 @@ public final class SingleItemSketch extends CompactSketch {
    * @param srcMem the Memory to be heapified.
    * @param seed a given hash seed
    * @return a SingleItemSketch
-   */
+   */ //does not override Sketch
   public static SingleItemSketch heapify(final Memory srcMem, final long seed) {
     final short seedHashMem = checkMemorySeedHash(srcMem, seed);
     if (isSingleItemSketch(srcMem)) {
@@ -96,6 +95,20 @@ public final class SingleItemSketch extends CompactSketch {
     throw new SketchesArgumentException("Input Memory Preamble is not a SingleItemSketch.");
   }
 
+  @Override
+  public CompactSketch compact() {
+    final long[] hashArr = getCache();
+    final short seedHash = getSeedHash();
+    return new HeapCompactOrderedSketch(hashArr, false, seedHash, 1, Long.MAX_VALUE);
+  }
+
+  @Override
+  public CompactSketch compact(final boolean dstOrdered, final WritableMemory dstMem) {
+    dstMem.putLong(0, pre0_);
+    dstMem.putLong(8, hash_);
+    return new DirectCompactOrderedSketch(dstMem);
+  }
+
   //Create methods using the default seed
 
   /**
@@ -104,7 +117,7 @@ public final class SingleItemSketch extends CompactSketch {
    * @param datum The given long datum.
    * @return a SingleItemSketch
    */
-  public static SingleItemSketch create(final long datum) {
+  static SingleItemSketch create(final long datum) {
     final long[] data = { datum };
     return new SingleItemSketch(hash(data, DEFAULT_UPDATE_SEED)[0] >>> 1);
   }
@@ -119,7 +132,7 @@ public final class SingleItemSketch extends CompactSketch {
    * @param datum The given double datum.
    * @return a SingleItemSketch
    */
-  public static SingleItemSketch create(final double datum) {
+  static SingleItemSketch create(final double datum) {
     final double d = (datum == 0.0) ? 0.0 : datum; // canonicalize -0.0, 0.0
     final long[] data = { Double.doubleToLongBits(d) };// canonicalize all NaN forms
     return new SingleItemSketch(hash(data, DEFAULT_UPDATE_SEED)[0] >>> 1);
@@ -137,7 +150,7 @@ public final class SingleItemSketch extends CompactSketch {
    * @param datum The given String.
    * @return a SingleItemSketch or null
    */
-  public static SingleItemSketch create(final String datum) {
+  static SingleItemSketch create(final String datum) {
     if ((datum == null) || datum.isEmpty()) { return null; }
     final byte[] data = datum.getBytes(UTF_8);
     return new SingleItemSketch(hash(data, DEFAULT_UPDATE_SEED)[0] >>> 1);
@@ -150,7 +163,7 @@ public final class SingleItemSketch extends CompactSketch {
    * @param data The given byte array.
    * @return a SingleItemSketch or null
    */
-  public static SingleItemSketch create(final byte[] data) {
+  static SingleItemSketch create(final byte[] data) {
     if ((data == null) || (data.length == 0)) { return null; }
     return new SingleItemSketch(hash(data, DEFAULT_UPDATE_SEED)[0] >>> 1);
   }
@@ -165,7 +178,7 @@ public final class SingleItemSketch extends CompactSketch {
    * @param data The given char array.
    * @return a SingleItemSketch or null
    */
-  public static SingleItemSketch create(final char[] data) {
+  static SingleItemSketch create(final char[] data) {
     if ((data == null) || (data.length == 0)) { return null; }
     return new SingleItemSketch(hash(data, DEFAULT_UPDATE_SEED)[0] >>> 1);
   }
@@ -177,7 +190,7 @@ public final class SingleItemSketch extends CompactSketch {
    * @param data The given int array.
    * @return a SingleItemSketch or null
    */
-  public static SingleItemSketch create(final int[] data) {
+  static SingleItemSketch create(final int[] data) {
     if ((data == null) || (data.length == 0)) { return null; }
     return new SingleItemSketch(hash(data, DEFAULT_UPDATE_SEED)[0] >>> 1);
   }
@@ -189,7 +202,7 @@ public final class SingleItemSketch extends CompactSketch {
    * @param data The given long array.
    * @return a SingleItemSketch or null
    */
-  public static SingleItemSketch create(final long[] data) {
+  static SingleItemSketch create(final long[] data) {
     if ((data == null) || (data.length == 0)) { return null; }
     return new SingleItemSketch(hash(data, DEFAULT_UPDATE_SEED)[0] >>> 1);
   }
@@ -203,7 +216,7 @@ public final class SingleItemSketch extends CompactSketch {
    * @param seed used to hash the given value.
    * @return a SingleItemSketch
    */
-  public static SingleItemSketch create(final long datum, final long seed) {
+  static SingleItemSketch create(final long datum, final long seed) {
     final long[] data = { datum };
     return new SingleItemSketch(hash(data, seed)[0] >>> 1);
   }
@@ -219,7 +232,7 @@ public final class SingleItemSketch extends CompactSketch {
    * @param seed used to hash the given value.
    * @return a SingleItemSketch
    */
-  public static SingleItemSketch create(final double datum, final long seed) {
+  static SingleItemSketch create(final double datum, final long seed) {
     final double d = (datum == 0.0) ? 0.0 : datum; // canonicalize -0.0, 0.0
     final long[] data = { Double.doubleToLongBits(d) };// canonicalize all NaN forms
     return new SingleItemSketch(hash(data, seed)[0] >>> 1, seed);
@@ -238,7 +251,7 @@ public final class SingleItemSketch extends CompactSketch {
    * @param seed used to hash the given value.
    * @return a SingleItemSketch or null
    */
-  public static SingleItemSketch create(final String datum, final long seed) {
+  static SingleItemSketch create(final String datum, final long seed) {
     if ((datum == null) || datum.isEmpty()) { return null; }
     final byte[] data = datum.getBytes(UTF_8);
     return new SingleItemSketch(hash(data, seed)[0] >>> 1, seed);
@@ -252,7 +265,7 @@ public final class SingleItemSketch extends CompactSketch {
    * @param seed used to hash the given value.
    * @return a SingleItemSketch or null
    */
-  public static SingleItemSketch create(final byte[] data, final long seed) {
+  static SingleItemSketch create(final byte[] data, final long seed) {
     if ((data == null) || (data.length == 0)) { return null; }
     return new SingleItemSketch(hash(data, seed)[0] >>> 1, seed);
   }
@@ -268,7 +281,7 @@ public final class SingleItemSketch extends CompactSketch {
    * @param seed used to hash the given value.
    * @return a SingleItemSketch or null
    */
-  public static SingleItemSketch create(final char[] data, final long seed) {
+  static SingleItemSketch create(final char[] data, final long seed) {
     if ((data == null) || (data.length == 0)) { return null; }
     return new SingleItemSketch(hash(data, seed)[0] >>> 1, seed);
   }
@@ -281,7 +294,7 @@ public final class SingleItemSketch extends CompactSketch {
    * @param seed used to hash the given value.
    * @return a SingleItemSketch or null
    */
-  public static SingleItemSketch create(final int[] data, final long seed) {
+  static SingleItemSketch create(final int[] data, final long seed) {
     if ((data == null) || (data.length == 0)) { return null; }
     return new SingleItemSketch(hash(data, seed)[0] >>> 1, seed);
   }
@@ -294,7 +307,7 @@ public final class SingleItemSketch extends CompactSketch {
    * @param seed used to hash the given value.
    * @return a SingleItemSketch or null
    */
-  public static SingleItemSketch create(final long[] data, final long seed) {
+  static SingleItemSketch create(final long[] data, final long seed) {
     if ((data == null) || (data.length == 0)) { return null; }
     return new SingleItemSketch(hash(data, seed)[0] >>> 1, seed);
   }
diff --git a/src/main/java/org/apache/datasketches/theta/Sketch.java b/src/main/java/org/apache/datasketches/theta/Sketch.java
index d61a312..c54d521 100644
--- a/src/main/java/org/apache/datasketches/theta/Sketch.java
+++ b/src/main/java/org/apache/datasketches/theta/Sketch.java
@@ -163,8 +163,9 @@ public abstract class Sketch {
             throw new SketchesArgumentException(
                 "Corrupted: COMPACT family sketch image must have Read-Only flag set");
           }
-          return orderedFlag ? DirectCompactOrderedSketch.wrapInstance(srcMem, seed)
-                : DirectCompactUnorderedSketch.wrapInstance(srcMem, seed);
+          return orderedFlag
+              ? DirectCompactOrderedSketch.wrapInstance(srcMem, seed)
+              : DirectCompactUnorderedSketch.wrapInstance(srcMem, seed);
         } //end of serVer 3
         else if (serVer == 1) {
           return ForwardCompatibility.heapify1to3(srcMem, seed);
@@ -563,11 +564,6 @@ public abstract class Sketch {
    */
   abstract int getCurrentPreambleLongs(boolean compact);
 
-  static final int computeCompactPreLongs(final long thetaLong, final boolean empty,
-      final int curCount) {
-    return (thetaLong < Long.MAX_VALUE) ? 3 : empty ? 1 : (curCount > 1) ? 2 : 1;
-  }
-
   /**
    * Returns the Memory object if it exists, otherwise null.
    * @return the Memory object if it exists, otherwise null.
@@ -696,9 +692,7 @@ public abstract class Sketch {
     }
     final byte familyID = srcMem.getByte(FAMILY_BYTE);
     final Family family = idToFamily(familyID);
-    final int preLongs = PreambleUtil.extractPreLongs(srcMem);
     final int flags = PreambleUtil.extractFlags(srcMem);
-    final boolean orderedFlag = (flags & ORDERED_FLAG_MASK) != 0;
     final boolean compactFlag = (flags & COMPACT_FLAG_MASK) != 0;
 
     switch (family) {
@@ -713,7 +707,7 @@ public abstract class Sketch {
         return HeapQuickSelectSketch.heapifyInstance(srcMem, seed);
       }
       case COMPACT: {
-
+        final boolean srcOrdered = (flags & ORDERED_FLAG_MASK) != 0;
         if (!compactFlag) {
           throw new SketchesArgumentException(
               "Corrupted: COMPACT family sketch image must have compact flag set");
@@ -723,18 +717,8 @@ public abstract class Sketch {
           throw new SketchesArgumentException(
               "Corrupted: COMPACT family sketch image must have Read-Only flag set");
         }
-        if (PreambleUtil.isEmptySketch(srcMem)) { //emptyFlag OR capacity < 16 bytes.
-          return EmptyCompactSketch.getInstance(srcMem);
-        }
-        if (preLongs == 1) {
-          if (isSingleItemSketch(srcMem)) { //SINGLE ITEM
-            return SingleItemSketch.heapify(srcMem, seed);
-          } else { //EMPTY. Note very old sketches ( before 2014) have no empty flag.
-            return EmptyCompactSketch.getInstance(srcMem);
-          }
-        }
-        return orderedFlag ? HeapCompactOrderedSketch.heapifyInstance(srcMem, seed)
-                       : HeapCompactUnorderedSketch.heapifyInstance(srcMem, seed);
+        final short memSeedHash = PreambleUtil.checkMemorySeedHash(srcMem, seed);
+        return CompactSketch.anyMemoryToCompactHeap(srcMem, memSeedHash, srcOrdered);
       } //end of Compact
       default: {
         throw new SketchesArgumentException(
diff --git a/src/main/java/org/apache/datasketches/theta/UnionImpl.java b/src/main/java/org/apache/datasketches/theta/UnionImpl.java
index e4d84d8..bf0dc7d 100644
--- a/src/main/java/org/apache/datasketches/theta/UnionImpl.java
+++ b/src/main/java/org/apache/datasketches/theta/UnionImpl.java
@@ -22,7 +22,6 @@ package org.apache.datasketches.theta;
 import static java.lang.Math.min;
 import static org.apache.datasketches.QuickSelect.selectExcludingZeros;
 import static org.apache.datasketches.Util.DEFAULT_UPDATE_SEED;
-import static org.apache.datasketches.theta.CompactSketch.compactCache;
 import static org.apache.datasketches.theta.PreambleUtil.COMPACT_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.ORDERED_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.PREAMBLE_LONGS_BYTE;
@@ -242,7 +241,7 @@ final class UnionImpl extends Union {
 
     //Compact the cache
     final long[] compactCacheOut =
-        compactCache(gadgetCacheCopy, curCountOut, minThetaLong, dstOrdered);
+        CompactOperations.compactCache(gadgetCacheCopy, curCountOut, minThetaLong, dstOrdered);
     final boolean empty = gadget_.isEmpty() && unionEmpty_;
     return createCompactSketch(
         compactCacheOut, empty, seedHash_, curCountOut, minThetaLong, dstOrdered, dstMem);
diff --git a/src/main/java/org/apache/datasketches/theta/UpdateSketch.java b/src/main/java/org/apache/datasketches/theta/UpdateSketch.java
index f67324b..352af73 100644
--- a/src/main/java/org/apache/datasketches/theta/UpdateSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/UpdateSketch.java
@@ -26,8 +26,6 @@ import static org.apache.datasketches.Util.MIN_LG_NOM_LONGS;
 import static org.apache.datasketches.Util.checkSeedHashes;
 import static org.apache.datasketches.Util.computeSeedHash;
 import static org.apache.datasketches.hash.MurmurHash3.hash;
-import static org.apache.datasketches.theta.CompactSketch.compactCache;
-import static org.apache.datasketches.theta.CompactSketch.loadCompactMemory;
 import static org.apache.datasketches.theta.PreambleUtil.BIG_ENDIAN_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.COMPACT_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.FAMILY_BYTE;
@@ -160,7 +158,7 @@ public abstract class UpdateSketch extends Sketch {
     final long[] cache = usk.getCache();
     if ((thetaLong == Long.MAX_VALUE) && (curCount == 1)) {
 
-      final long[] cacheOut = compactCache(cache, curCount, thetaLong, dstOrdered);
+      final long[] cacheOut = CompactOperations.compactCache(cache, curCount, thetaLong, dstOrdered);
       final long hash = cacheOut[0];
       final SingleItemSketch sis = new SingleItemSketch(hash, usk.getSeedHash());
       if (dstMem != null) {
@@ -169,7 +167,7 @@ public abstract class UpdateSketch extends Sketch {
       return sis;
     }
     if (dstMem == null) {
-      final long[] cacheOut = CompactSketch.compactCache(cache, curCount, thetaLong, dstOrdered);
+      final long[] cacheOut = CompactOperations.compactCache(cache, curCount, thetaLong, dstOrdered);
       if (dstOrdered) {
         return new HeapCompactOrderedSketch(cacheOut, false, seedHash, curCount, thetaLong);
       } else {
@@ -190,11 +188,11 @@ public abstract class UpdateSketch extends Sketch {
    * @param thetaLong the value of theta.
    * @return a CompactSketch
    */
-  private static CompactSketch compactHeap(final UpdateSketch sketch, final boolean ordered,
+  static CompactSketch compactHeap(final UpdateSketch sketch, final boolean ordered,
       final int curCount, final long thetaLong) {
     final short seedHash = sketch.getSeedHash();
     final long[] cache = sketch.getCache();
-    final long[] cacheOut = CompactSketch.compactCache(cache, curCount, thetaLong, ordered);
+    final long[] cacheOut = CompactOperations.compactCache(cache, curCount, thetaLong, ordered);
     if (ordered) {
       return new HeapCompactOrderedSketch(cacheOut, false, seedHash, curCount, thetaLong);
     } else {
@@ -212,19 +210,19 @@ public abstract class UpdateSketch extends Sketch {
    * @param thetaLong the value of theta.
    * @return a CompactSketch.
    */
-  private static CompactSketch compactDirect(final UpdateSketch sketch,
+  static CompactSketch compactDirect(final UpdateSketch sketch,
       final WritableMemory dstMem, final boolean ordered, final int curCount, final long thetaLong) {
-    final int preLongs = computeCompactPreLongs(thetaLong, false, curCount);
+    final int preLongs = CompactOperations.computeCompactPreLongs(thetaLong, false, curCount);
     final short seedHash = sketch.getSeedHash();
     final long[] cache = sketch.getCache();
-    final long[] cacheOut = CompactSketch.compactCache(cache, curCount, thetaLong, ordered);
+    final long[] cacheOut = CompactOperations.compactCache(cache, curCount, thetaLong, ordered);
     if (ordered) {
       final byte flags = (byte)(READ_ONLY_FLAG_MASK | COMPACT_FLAG_MASK | ORDERED_FLAG_MASK);
-      loadCompactMemory(cacheOut, seedHash, curCount, thetaLong, dstMem, flags, preLongs);
+      CompactOperations.loadCompactMemory(cacheOut, seedHash, curCount, thetaLong, dstMem, flags, preLongs);
       return new DirectCompactOrderedSketch(dstMem);
     } else {
       final byte flags = (byte)(READ_ONLY_FLAG_MASK | COMPACT_FLAG_MASK);
-      loadCompactMemory(cacheOut, seedHash, curCount, thetaLong, dstMem, flags, preLongs);
+      CompactOperations.loadCompactMemory(cacheOut, seedHash, curCount, thetaLong, dstMem, flags, preLongs);
       return new DirectCompactUnorderedSketch(dstMem);
     }
   }
diff --git a/src/main/javadoc/resources/dictionary.html b/src/main/javadoc/resources/dictionary.html
index 12c37bd..48b879d 100644
--- a/src/main/javadoc/resources/dictionary.html
+++ b/src/main/javadoc/resources/dictionary.html
@@ -217,7 +217,7 @@ some security and protection against "dictionary attacks".
 of sketches that were generated with different seeds a short, 16-bit, 
 <a href="#seedHash">Seed Hash</a> is stored with the sketch image.
 When heapifying or wrapping an UpdateSketch image, which can be either a byte array or a Memory object, 
-the user must provide the original seed either directly or indirectly by assuming the <i>DEFAULT_UPDATE_SEED</i>. 
+the user must provide the original seed either directly or indirectly by assuming the <a href="#defaultUpdateSeed">DEFAULT_UPDATE_SEED</a>. 
 The provided seed will be hashed and validated against the internal short Seed Hash and an error will be thrown if the seed hashes do not match.
 The Set Operations classes, Union, Intersection and AnotB also require the user to provide the seed either directly or indirectly.
 <p>An internal check will be made to make sure that the provided seed does not hash to a 16-bit value of zero. 
diff --git a/src/test/java/org/apache/datasketches/theta/CompactSketchTest.java b/src/test/java/org/apache/datasketches/theta/CompactSketchTest.java
index f294edb..6170e2f 100644
--- a/src/test/java/org/apache/datasketches/theta/CompactSketchTest.java
+++ b/src/test/java/org/apache/datasketches/theta/CompactSketchTest.java
@@ -222,7 +222,7 @@ public class CompactSketchTest {
   @Test
   public void checkCompactCachePart() {
     //phony values except for curCount = 0.
-    long[] result = CompactSketch.compactCachePart(null, 4, 0, 0L, false);
+    long[] result = IntersectionImplR.compactCachePart(null, 4, 0, 0L, false);
     assertEquals(result.length, 0);
   }
 
diff --git a/src/test/java/org/apache/datasketches/theta/SetOpsCornerCasesTest.java b/src/test/java/org/apache/datasketches/theta/SetOpsCornerCasesTest.java
index 0095a47..36abe4a 100644
--- a/src/test/java/org/apache/datasketches/theta/SetOpsCornerCasesTest.java
+++ b/src/test/java/org/apache/datasketches/theta/SetOpsCornerCasesTest.java
@@ -96,7 +96,7 @@ public class SetOpsCornerCasesTest {
   }
 
   @Test
-  public void checkExactNull() {
+  public void checkExactNullSpecificCase() {
     cornerCaseChecksMemory(State.EXACT, State.NULL, 64);
   }
 
@@ -109,31 +109,31 @@ public class SetOpsCornerCasesTest {
 
     CompactSketch rcskStdU = doStdUnion(tcskA, tcskB, k, null);
     CompactSketch rcskPwU = doPwUnion(tcskA, tcskB, k);
-    checkCornerCase(rcskPwU, rcskStdU);
+    checkCornerCase(rcskPwU, rcskStdU); //heap, heap
 
     rcskStdU = doStdUnion(tcskA, tcskB, k, wmem);
     CompactSketch rcskStdPairU = doStdPairUnion(tcskA, tcskB, k, wmem);
-    checkCornerCase(rcskStdPairU, rcskStdU);
+    checkCornerCase(rcskStdPairU, rcskStdU); //direct, direct
 
     wmem = WritableMemory.allocate(SetOperation.getMaxIntersectionBytes(k));
 
     CompactSketch rcskStdI = doStdIntersection(tcskA, tcskB, null);
     CompactSketch rcskPwI = doPwIntersection(tcskA, tcskB);
-    checkCornerCase(rcskPwI, rcskStdI);
+    checkCornerCase(rcskPwI, rcskStdI); //empty, empty
 
     rcskStdI = doStdIntersection(tcskA, tcskB, wmem);
     CompactSketch rcskStdPairI = doStdPairIntersection(tcskA, tcskB, wmem);
-    checkCornerCase(rcskStdPairI, rcskStdI);
+    checkCornerCase(rcskStdPairI, rcskStdI); //empty, empty //direct, direct???
 
     wmem = WritableMemory.allocate(SetOperation.getMaxAnotBResultBytes(k));
 
     CompactSketch rcskStdAnotB = doStdAnotB(tcskA, tcskB, null);
     CompactSketch rcskPwAnotB = doPwAnotB(tcskA, tcskB);
-    checkCornerCase(rcskPwAnotB, rcskStdAnotB);
+    checkCornerCase(rcskPwAnotB, rcskStdAnotB); //heap, heap
 
     rcskStdAnotB = doStdAnotB(tcskA, tcskB, wmem);
     CompactSketch rcskStdStatefulAnotB = doStdStatefulAnotB(tcskA, tcskB, wmem);
-    checkCornerCase(rcskStdStatefulAnotB, rcskStdAnotB);
+    checkCornerCase(rcskStdStatefulAnotB, rcskStdAnotB); //direct, heap
   }
 
   private static void cornerCaseChecks(State stateA, State stateB, int k) {
@@ -230,6 +230,8 @@ public class SetOpsCornerCasesTest {
     Assert.assertEquals(emptyB, emptyA);
     Assert.assertEquals(thetaLongB, thetaLongA);
     Assert.assertEquals(countB, countA);
+    String A = rskA.getClass().getSimpleName();
+    String B = rskB.getClass().getSimpleName();
     Assert.assertEquals(rskA.getClass().getSimpleName(), rskB.getClass().getSimpleName());
   }
 
diff --git a/src/test/java/org/apache/datasketches/theta/SketchTest.java b/src/test/java/org/apache/datasketches/theta/SketchTest.java
index 92a6edd..3cf9783 100644
--- a/src/test/java/org/apache/datasketches/theta/SketchTest.java
+++ b/src/test/java/org/apache/datasketches/theta/SketchTest.java
@@ -76,7 +76,7 @@ public class SketchTest {
     assertEquals(compSk.getCurrentBytes(true), 8);
     assertEquals(compSk.getCurrentBytes(false), 8);
 
-    int compPreLongs = Sketch.computeCompactPreLongs(sketch.getThetaLong(), sketch.isEmpty(),
+    int compPreLongs = CompactOperations.computeCompactPreLongs(sketch.getThetaLong(), sketch.isEmpty(),
         sketch.getRetainedEntries(true));
     assertEquals(compPreLongs, 1);
 
@@ -91,7 +91,7 @@ public class SketchTest {
     assertEquals(sketch.getCurrentBytes(false), (k*2*8) + (lowQSPreLongs << 3));
     assertEquals(sketch.getCurrentBytes(true), (k*8) + (2*8)); //compact form  //FAILS HERE
 
-    compPreLongs = Sketch.computeCompactPreLongs(sketch.getThetaLong(), sketch.isEmpty(),
+    compPreLongs = CompactOperations.computeCompactPreLongs(sketch.getThetaLong(), sketch.isEmpty(),
         sketch.getRetainedEntries(true));
     assertEquals(compPreLongs, 2);
 
@@ -107,7 +107,7 @@ public class SketchTest {
     assertEquals(sketch.getCurrentBytes(false), (k*2*8) + (lowQSPreLongs << 3));
     assertEquals(sketch.getCurrentBytes(true), (curCount*8) + (3*8)); //compact form
 
-    compPreLongs = Sketch.computeCompactPreLongs(sketch.getThetaLong(), sketch.isEmpty(),
+    compPreLongs = CompactOperations.computeCompactPreLongs(sketch.getThetaLong(), sketch.isEmpty(),
         sketch.getRetainedEntries(true));
     assertEquals(compPreLongs, 3);
 


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


[incubator-datasketches-java] 01/03: Current evolution of setOp and compact changes

Posted by le...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

leerho pushed a commit to branch Refactor_Theta_Tuple
in repository https://gitbox.apache.org/repos/asf/incubator-datasketches-java.git

commit 3a79cec4ef1338fb0b89ea2cc89ae0fcadac9d68
Author: Lee Rhodes <le...@users.noreply.github.com>
AuthorDate: Fri Jun 19 14:54:21 2020 -0700

    Current evolution of setOp and compact changes
---
 .../datasketches/BoundsOnBinomialProportions.java  |  27 +-
 .../datasketches/BoundsOnRatiosInSampledSets.java  |  43 +-
 .../BoundsOnRatiosInThetaSketchedSets.java         |  49 +-
 .../org/apache/datasketches/HashOperations.java    |  26 +
 src/main/java/org/apache/datasketches/Util.java    |   5 +
 .../java/org/apache/datasketches/theta/AnotB.java  | 112 ++++-
 .../org/apache/datasketches/theta/AnotBimpl.java   | 528 +++++++++++++++++++++
 .../apache/datasketches/theta/CompactSketch.java   |   3 +-
 .../theta/DirectCompactOrderedSketch.java          |  26 +-
 .../datasketches/theta/DirectCompactSketch.java    |   6 +-
 .../theta/DirectCompactUnorderedSketch.java        |  20 +-
 .../theta/DirectQuickSelectSketch.java             |  13 +-
 .../theta/DirectQuickSelectSketchR.java            |   2 +-
 .../datasketches/theta/ForwardCompatibility.java   |   9 +-
 .../apache/datasketches/theta/HeapAlphaSketch.java |  22 +-
 .../org/apache/datasketches/theta/HeapAnotB.java   | 345 --------------
 .../theta/HeapCompactOrderedSketch.java            |  21 +-
 .../theta/HeapCompactUnorderedSketch.java          |  32 +-
 .../datasketches/theta/HeapQuickSelectSketch.java  |  16 +-
 .../datasketches/theta/IntersectionImpl.java       |   2 +-
 .../datasketches/theta/PairwiseSetOperations.java  |  26 +-
 .../apache/datasketches/theta/PreambleUtil.java    |  77 ++-
 .../apache/datasketches/theta/SetOperation.java    |  21 +-
 .../datasketches/theta/SetOperationBuilder.java    |   4 +-
 .../datasketches/theta/SingleItemSketch.java       |  16 +-
 .../java/org/apache/datasketches/theta/Sketch.java |  64 ++-
 .../org/apache/datasketches/theta/Sketches.java    |  12 +
 .../java/org/apache/datasketches/theta/Union.java  |  30 ++
 .../org/apache/datasketches/theta/UnionImpl.java   |  28 +-
 .../apache/datasketches/theta/UpdateSketch.java    |  46 +-
 .../java/org/apache/datasketches/tuple/AnotB.java  | 167 +++----
 .../apache/datasketches/tuple/CompactSketch.java   |   5 +
 .../datasketches/tuple/QuickSelectSketch.java      |   1 +
 .../java/org/apache/datasketches/tuple/Sketch.java |  10 +
 .../{HeapAnotBTest.java => AnotBimplTest.java}     |   4 +-
 .../theta/ForwardCompatibilityTest.java            |   4 +-
 .../theta/PairwiseSetOperationsTest.java           |   2 +-
 .../datasketches/theta/PreambleUtilTest.java       |  85 ++--
 ...erCasesTest.java => SetOpsCornerCasesTest.java} | 347 ++++++++------
 .../datasketches/theta/SingleItemSketchTest.java   |   9 +-
 40 files changed, 1377 insertions(+), 888 deletions(-)

diff --git a/src/main/java/org/apache/datasketches/BoundsOnBinomialProportions.java b/src/main/java/org/apache/datasketches/BoundsOnBinomialProportions.java
index c467a83..5ccc499 100644
--- a/src/main/java/org/apache/datasketches/BoundsOnBinomialProportions.java
+++ b/src/main/java/org/apache/datasketches/BoundsOnBinomialProportions.java
@@ -30,8 +30,7 @@ package org.apache.datasketches;
  * that are called <i>n</i> and <i>k</i> in our sketching library. There is also a third
  * parameter, numStdDev, that specifies the desired confidence level.</p>
  * <ul>
- * <li><i>n</i> is the number of independent randomized trials. It is given and therefore known.
- * </li>
+ * <li><i>n</i> is the number of independent randomized trials. It is given and therefore known.</li>
  * <li><i>p</i> is the probability of a trial being a success. It is unknown.</li>
  * <li><i>k</i> is the number of trials (out of <i>n</i>) that turn out to be successes. It is
  * a random variable governed by a binomial distribution. After any given
@@ -193,21 +192,15 @@ public final class BoundsOnBinomialProportions { // confidence intervals for bin
   }
 
   //@formatter:off
-  // Abramowitz and Stegun formula 7.1.28, p. 88; Claims accuracy of about 7 decimal digits */
+  // Abramowitz and Stegun formula 7.1.28, p. 88; Claims accuracy of about 7 decimal digits
   private static double erf_of_nonneg(final double x) {
-    // The constants that appear below, formatted for easy checking against the book.
-    //    a1 = 0.07052 30784
-    //    a3 = 0.00927 05272
-    //    a5 = 0.00027 65672
-    //    a2 = 0.04228 20123
-    //    a4 = 0.00015 20143
-    //    a6 = 0.00004 30638
-    final double a1 = 0.0705230784;
-    final double a3 = 0.0092705272;
-    final double a5 = 0.0002765672;
-    final double a2 = 0.0422820123;
-    final double a4 = 0.0001520143;
-    final double a6 = 0.0000430638;
+    // The constants from the book
+    final double a1 = 0.07052_30784;
+    final double a3 = 0.00927_05272;
+    final double a5 = 0.00027_65672;
+    final double a2 = 0.04228_20123;
+    final double a4 = 0.00015_20143;
+    final double a6 = 0.00004_30638;
     final double x2 = x * x; // x squared, x cubed, etc.
     final double x3 = x2 * x;
     final double x4 = x2 * x2;
@@ -226,12 +219,12 @@ public final class BoundsOnBinomialProportions { // confidence intervals for bin
     final double sum16 = sum8 * sum8;
     return (1.0 - (1.0 / sum16));
   }
+  //@formatter:on
 
   private static double deltaOfNumStdevs(final double kappa) {
     return (normalCDF(-1.0 * kappa));
   }
 
-  //@formatter:on
   // Formula 26.5.22 on page 945 of Abramowitz & Stegun, which is an approximation
   // of the inverse of the incomplete beta function I_x(a,b) = delta
   // viewed as a scalar function of x.
diff --git a/src/main/java/org/apache/datasketches/BoundsOnRatiosInSampledSets.java b/src/main/java/org/apache/datasketches/BoundsOnRatiosInSampledSets.java
index 78bc432..45327e0 100644
--- a/src/main/java/org/apache/datasketches/BoundsOnRatiosInSampledSets.java
+++ b/src/main/java/org/apache/datasketches/BoundsOnRatiosInSampledSets.java
@@ -83,6 +83,28 @@ public final class BoundsOnRatiosInSampledSets {
   }
 
   /**
+   * Return the estimate of A. See class javadoc.
+   * @param a See class javadoc
+   * @param f the inclusion probability used to produce the set with size <i>a</i>.
+   * @return the approximate lower bound
+   */
+  public static double getEstimateOfA(final long a, final double f) {
+    checkInputs(a, 1, f);
+    return a / f;
+  }
+
+  /**
+   * Return the estimate of B. See class javadoc.
+   * @param b See class javadoc
+   * @param f the inclusion probability used to produce the set with size <i>a</i>.
+   * @return the approximate lower bound
+   */
+  public static double getEstimateOfB(final long b, final double f) {
+    checkInputs(b + 1, b, f);
+    return b / f;
+  }
+
+  /**
    * This hackyAdjuster is tightly coupled with the width of the confidence interval normally
    * specified with number of standard deviations. To simplify this interface the number of
    * standard deviations has been fixed to 2.0, which corresponds to a confidence interval of
@@ -105,25 +127,4 @@ public final class BoundsOnRatiosInSampledSets {
     }
   }
 
-  /**
-   * Return the estimate of A. See class javadoc.
-   * @param a See class javadoc
-   * @param f the inclusion probability used to produce the set with size <i>a</i>.
-   * @return the approximate lower bound
-   */
-  public static double getEstimateOfA(final long a, final double f) {
-    checkInputs(a, 1, f);
-    return a / f;
-  }
-
-  /**
-   * Return the estimate of B. See class javadoc.
-   * @param b See class javadoc
-   * @param f the inclusion probability used to produce the set with size <i>a</i>.
-   * @return the approximate lower bound
-   */
-  public static double getEstimateOfB(final long b, final double f) {
-    checkInputs(b + 1, b, f);
-    return b / f;
-  }
 }
diff --git a/src/main/java/org/apache/datasketches/BoundsOnRatiosInThetaSketchedSets.java b/src/main/java/org/apache/datasketches/BoundsOnRatiosInThetaSketchedSets.java
index b724fc6..d46b0be 100644
--- a/src/main/java/org/apache/datasketches/BoundsOnRatiosInThetaSketchedSets.java
+++ b/src/main/java/org/apache/datasketches/BoundsOnRatiosInThetaSketchedSets.java
@@ -19,6 +19,8 @@
 
 package org.apache.datasketches;
 
+import static org.apache.datasketches.Util.LONG_MAX_VALUE_AS_DOUBLE;
+
 import org.apache.datasketches.theta.Sketch;
 
 /**
@@ -53,17 +55,18 @@ public final class BoundsOnRatiosInThetaSketchedSets {
    * @return the approximate lower bound for B over A
    */
   public static double getLowerBoundForBoverA(final Sketch sketchA, final Sketch sketchB) {
-    final double thetaA = sketchA.getTheta();
-    final double thetaB = sketchB.getTheta();
-    checkThetas(thetaA, thetaB);
+    final long thetaLongA = sketchA.getThetaLong();
+    final long thetaLongB = sketchB.getThetaLong();
+    checkThetas(thetaLongA, thetaLongB);
 
     final int countB = sketchB.getRetainedEntries(true);
-    final int countA = (thetaB == thetaA) ? sketchA.getRetainedEntries(true)
-        : sketchA.getCountLessThanTheta(thetaB);
+    final int countA = (thetaLongB == thetaLongA)
+        ? sketchA.getRetainedEntries(true)
+        : sketchA.getCountLessThanThetaLong(thetaLongB);
 
     if (countA <= 0) { return 0; }
-
-    return BoundsOnRatiosInSampledSets.getLowerBoundForBoverA(countA, countB, thetaB);
+    final double f = thetaLongB / LONG_MAX_VALUE_AS_DOUBLE;
+    return BoundsOnRatiosInSampledSets.getLowerBoundForBoverA(countA, countB, f);
   }
 
   /**
@@ -73,17 +76,18 @@ public final class BoundsOnRatiosInThetaSketchedSets {
    * @return the approximate upper bound for B over A
    */
   public static double getUpperBoundForBoverA(final Sketch sketchA, final Sketch sketchB) {
-    final double thetaA = sketchA.getTheta();
-    final double thetaB = sketchB.getTheta();
-    checkThetas(thetaA, thetaB);
+    final long thetaLongA = sketchA.getThetaLong();
+    final long thetaLongB = sketchB.getThetaLong();
+    checkThetas(thetaLongA, thetaLongB);
 
     final int countB = sketchB.getRetainedEntries(true);
-    final int countA = (thetaB == thetaA) ? sketchA.getRetainedEntries(true)
-        : sketchA.getCountLessThanTheta(thetaB);
+    final int countA = (thetaLongB == thetaLongA)
+        ? sketchA.getRetainedEntries(true)
+        : sketchA.getCountLessThanThetaLong(thetaLongB);
 
     if (countA <= 0) { return 1.0; }
-
-    return BoundsOnRatiosInSampledSets.getUpperBoundForBoverA(countA, countB, thetaB);
+    final double f = thetaLongB / LONG_MAX_VALUE_AS_DOUBLE;
+    return BoundsOnRatiosInSampledSets.getUpperBoundForBoverA(countA, countB, f);
   }
 
   /**
@@ -93,22 +97,23 @@ public final class BoundsOnRatiosInThetaSketchedSets {
    * @return the estimate for B over A
    */
   public static double getEstimateOfBoverA(final Sketch sketchA, final Sketch sketchB) {
-    final double thetaA = sketchA.getTheta();
-    final double thetaB = sketchB.getTheta();
-    checkThetas(thetaA, thetaB);
+    final long thetaLongA = sketchA.getThetaLong();
+    final long thetaLongB = sketchB.getThetaLong();
+    checkThetas(thetaLongA, thetaLongB);
 
     final int countB = sketchB.getRetainedEntries(true);
-    final int countA = (thetaB == thetaA) ? sketchA.getRetainedEntries(true)
-        : sketchA.getCountLessThanTheta(thetaB);
+    final int countA = (thetaLongB == thetaLongA)
+        ? sketchA.getRetainedEntries(true)
+        : sketchA.getCountLessThanThetaLong(thetaLongB);
 
     if (countA <= 0) { return 0.5; }
 
     return (double) countB / (double) countA;
   }
 
-  static void checkThetas(final double thetaA, final double thetaB) {
-    if (thetaB > thetaA) {
-      throw new SketchesArgumentException("ThetaB cannot be > ThetaA.");
+  static void checkThetas(final long thetaLongA, final long thetaLongB) {
+    if (thetaLongB > thetaLongA) {
+      throw new SketchesArgumentException("ThetaLongB cannot be > ThetaLongA.");
     }
   }
 }
diff --git a/src/main/java/org/apache/datasketches/HashOperations.java b/src/main/java/org/apache/datasketches/HashOperations.java
index abb0b93..47bf16a 100644
--- a/src/main/java/org/apache/datasketches/HashOperations.java
+++ b/src/main/java/org/apache/datasketches/HashOperations.java
@@ -19,6 +19,9 @@
 
 package org.apache.datasketches;
 
+import static org.apache.datasketches.Util.MIN_LG_NOM_LONGS;
+import static org.apache.datasketches.Util.ceilingPowerOf2;
+
 import org.apache.datasketches.memory.Memory;
 import org.apache.datasketches.memory.WritableMemory;
 
@@ -352,4 +355,27 @@ public final class HashOperations {
     return (( (hash - 1L) | (thetaLong - hash - 1L)) < 0L );
   }
 
+  /**
+   * Converts the given array to a hash table.
+   * @param hashArr The given array of hashes. Gaps are OK.
+   * @param count The number of valid hashes in the array
+   * @param thetaLong Any hashes equal to or greater than thetaLong will be ignored
+   * @param rebuildThreshold The fill fraction for the hash table forcing a rebuild or resize.
+   * @return a HashTable
+   */
+  public static long[] convertToHashTable(
+      final long[] hashArr,
+      final int count,
+      final long thetaLong,
+      final double rebuildThreshold) {
+    final int size = Math.max(
+      ceilingPowerOf2((int) Math.ceil(count / rebuildThreshold)),
+      1 << MIN_LG_NOM_LONGS
+    );
+    final long[] hashTable = new long[size];
+    hashArrayInsert(
+        hashArr, hashTable, Integer.numberOfTrailingZeros(size), thetaLong);
+    return hashTable;
+  }
+
 }
diff --git a/src/main/java/org/apache/datasketches/Util.java b/src/main/java/org/apache/datasketches/Util.java
index 69f51fa..01fcb02 100644
--- a/src/main/java/org/apache/datasketches/Util.java
+++ b/src/main/java/org/apache/datasketches/Util.java
@@ -123,6 +123,11 @@ public final class Util {
    */
   public static final double iGolden = 0.6180339887498949025; // the inverse golden ratio
 
+  /**
+   * Long.MAX_VALUE as a double.
+   */
+  public static final double LONG_MAX_VALUE_AS_DOUBLE = Long.MAX_VALUE;
+
   private Util() {}
 
   //Byte Conversions
diff --git a/src/main/java/org/apache/datasketches/theta/AnotB.java b/src/main/java/org/apache/datasketches/theta/AnotB.java
index ada10c0..4a4527a 100644
--- a/src/main/java/org/apache/datasketches/theta/AnotB.java
+++ b/src/main/java/org/apache/datasketches/theta/AnotB.java
@@ -47,55 +47,125 @@ public abstract class AnotB extends SetOperation {
   }
 
   /**
-   * Gets the result of this operation as an ordered CompactSketch on the Java heap
-   * @return the result of this operation as an ordered CompactSketch on the Java heap
+   * This is a stateful input operation. This method sets the given Sketch as the first
+   * argument <i>A</i> of a stateful <i>AnotB</i> operation. This overwrites the internal state of
+   * this AnotB operator with the contents of the given sketch. This sets the stage for multiple
+   * stateful subsequent {@link #notB(Sketch)} operations. The ultimate result is obtained using
+   * the {@link #getResult(boolean)} or {@link #getResult(boolean, WritableMemory, boolean)}.
+   *
+   * <p>An input argument of null will throw an exception.</p>
+   *
+   * @param skA The incoming sketch for the first argument, <i>A</i>.
    */
-  public abstract CompactSketch getResult();
+  public abstract void setA(Sketch skA);
+
+  /**
+   * Performs a stateful <i>AND NOT</i> operation with the existing internal state of this AnotB
+   * operator. Use {@link #getResult(boolean)} or {@link #getResult(boolean, WritableMemory, boolean)}
+   * to obtain the result.
+   *
+   * <p>An input argument of null or empty is ignored.</p>
+   *
+   * @param skB The incoming sketch for the second (or following) argument <i>B</i>.
+   */
+  public abstract void notB(Sketch skB);
+
+  /**
+   * Gets the result of this operation as an ordered CompactSketch on the Java heap.
+   * @param reset If true, clears this operator to the empty state after result is returned.
+   * @return the result of this operation as a CompactSketch.
+   */
+  public abstract CompactSketch getResult(boolean reset);
 
   /**
-   * Gets the result of this set operation as a CompactSketch of the chosen form
+   * Gets the result of this stateful set operation as a CompactSketch of the chosen form. The
+   * stateful input operations are {@link #setA(Sketch)} and {@link #notB(Sketch)}.
+   *
    * @param dstOrdered
    * <a href="{@docRoot}/resources/dictionary.html#dstOrdered">See Destination Ordered</a>.
    *
    * @param dstMem
    * <a href="{@docRoot}/resources/dictionary.html#dstMem">See Destination Memory</a>.
    *
-   * @return the result of this set operation as a CompactSketch of the chosen form
-   */
-  public abstract CompactSketch getResult(boolean dstOrdered, WritableMemory dstMem);
-
-  /**
-   * Perform A-and-not-B set operation on the two given sketches.
-   * A null sketch is interpreted as an empty sketch.
+   * @param reset If true, clears this operator to the empty state after result is returned.
    *
-   * @param a The incoming sketch for the first argument
-   * @param b The incoming sketch for the second argument
+   * @return the result of this set operation as a CompactSketch of the chosen form
    */
-  public abstract void update(Sketch a, Sketch b);
+  public abstract CompactSketch getResult(boolean dstOrdered, WritableMemory dstMem, boolean reset);
 
   /**
    * Perform A-and-not-B set operation on the two given sketches and return the result as an
    * ordered CompactSketch on the heap.
-   * @param a The incoming sketch for the first argument
-   * @param b The incoming sketch for the second argument
+   *
+   * <p>This a stateless operation and has no impact on the internal state of this operator.
+   * Thus, this is not an accumulating update and does not interact with the {@link #setA(Sketch)},
+   * {@link #notB(Sketch)}, {@link #getResult(boolean)}, or
+   * {@link #getResult(boolean, WritableMemory, boolean)} methods.</p>
+   *
+   * @param skA The incoming sketch for the first argument
+   * @param skB The incoming sketch for the second argument
    * @return an ordered CompactSketch on the heap
    */
-  public CompactSketch aNotB(final Sketch a, final Sketch b) {
-    return aNotB(a, b, true, null);
+  public CompactSketch aNotB(final Sketch skA, final Sketch skB) {
+    return aNotB(skA, skB, true, null);
   }
 
   /**
    * Perform A-and-not-B set operation on the two given sketches and return the result as a
    * CompactSketch.
-   * @param a The incoming sketch for the first argument
-   * @param b The incoming sketch for the second argument
+   *
+   * <p>This a stateless operation and has no impact on the internal state of this operator.
+   * Thus, this is not an accumulating update and does not interact with the {@link #setA(Sketch)}
+   * or {@link #notB(Sketch)} methods.</p>
+   *
+   * @param skA The incoming sketch for the first argument
+   * @param skB The incoming sketch for the second argument
    * @param dstOrdered
    * <a href="{@docRoot}/resources/dictionary.html#dstOrdered">See Destination Ordered</a>.
    * @param dstMem
    * <a href="{@docRoot}/resources/dictionary.html#dstMem">See Destination Memory</a>.
    * @return the result as a CompactSketch.
    */
-  public abstract CompactSketch aNotB(Sketch a, Sketch b, boolean dstOrdered,
+  public abstract CompactSketch aNotB(Sketch skA, Sketch skB, boolean dstOrdered,
       WritableMemory dstMem);
 
+  //Deprecated methods
+
+  /**
+   * Perform A-and-not-B set operation on the two given sketches.
+   * A null sketch is interpreted as an empty sketch.
+   *
+   * @param skA The incoming sketch for the first argument
+   * @param skB The incoming sketch for the second argument
+   * @deprecated Instead use {@link #aNotB(Sketch, Sketch)}.
+   */
+  @Deprecated
+  public abstract void update(Sketch skA, Sketch skB);
+
+  /**
+   * Gets the result of this operation as an ordered CompactSketch on the Java heap.
+   * This clears the state of this operator after the result is returned.
+   * @return the result of this operation as an ordered CompactSketch on the Java heap.
+   * @deprecated Instead use {@link #getResult(boolean)} or
+   * {@link #getResult(boolean, WritableMemory, boolean)}.
+   */
+  @Deprecated
+  public abstract CompactSketch getResult();
+
+  /**
+   * Gets the result of this set operation as a CompactSketch of the chosen form.
+   * This clears the state of this operator after the result is returned.
+   * @param dstOrdered
+   * <a href="{@docRoot}/resources/dictionary.html#dstOrdered">See Destination Ordered</a>.
+   *
+   * @param dstMem
+   * <a href="{@docRoot}/resources/dictionary.html#dstMem">See Destination Memory</a>.
+   *
+   * @return the result of this set operation as a CompactSketch of the chosen form.
+   * @deprecated Instead use {@link #getResult(boolean)} or
+   * {@link #getResult(boolean, WritableMemory, boolean)}.
+   */
+  @Deprecated
+  public abstract CompactSketch getResult(boolean dstOrdered, WritableMemory dstMem);
+
 }
diff --git a/src/main/java/org/apache/datasketches/theta/AnotBimpl.java b/src/main/java/org/apache/datasketches/theta/AnotBimpl.java
new file mode 100644
index 0000000..797c1de
--- /dev/null
+++ b/src/main/java/org/apache/datasketches/theta/AnotBimpl.java
@@ -0,0 +1,528 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.datasketches.theta;
+
+import static java.lang.Math.min;
+import static org.apache.datasketches.HashOperations.convertToHashTable;
+import static org.apache.datasketches.HashOperations.hashArrayInsert;
+import static org.apache.datasketches.HashOperations.hashSearch;
+import static org.apache.datasketches.Util.REBUILD_THRESHOLD;
+import static org.apache.datasketches.Util.checkSeedHashes;
+import static org.apache.datasketches.Util.simpleIntLog2;
+import static org.apache.datasketches.theta.CompactSketch.compactCache;
+import static org.apache.datasketches.theta.CompactSketch.loadCompactMemory;
+import static org.apache.datasketches.theta.PreambleUtil.COMPACT_FLAG_MASK;
+import static org.apache.datasketches.theta.PreambleUtil.ORDERED_FLAG_MASK;
+import static org.apache.datasketches.theta.PreambleUtil.READ_ONLY_FLAG_MASK;
+
+import java.util.Arrays;
+
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+
+/**
+ * Implements the A-and-not-B operation.
+ * @author Lee Rhodes
+ * @author Kevin Lang
+ */
+final class AnotBimpl extends AnotB {
+  private final short seedHashR_;
+  private boolean emptyR_;
+  private long thetaLongR_;
+  private long[] hashArrR_; //compact array w curCountR_ entries
+  private int curCountR_;
+
+  //Remove all 4 of these with deprecated
+  private Sketch skA_;
+  private Sketch skB_;
+  private int lgArrLongsHT_; //for Hash Table only. may not need to be member after refactoring
+  private long[] bHashTable_; //may not need to be member after refactoring.
+
+  /**
+   * Construct a new AnotB SetOperation on the java heap.  Called by SetOperation.Builder.
+   *
+   * @param seed <a href="{@docRoot}/resources/dictionary.html#seed">See seed</a>
+   */
+  AnotBimpl(final long seed) {
+    this(computeSeedHash(seed));
+  }
+
+  /**
+   * Construct a new AnotB SetOperation on the java heap.  Called by PairwiseSetOperation.
+   *
+   * @param seedHash 16 bit hash of the chosen update seed.
+   */
+  AnotBimpl(final short seedHash) {
+    seedHashR_ = seedHash;
+    reset();
+  }
+
+  @Override
+  public void setA(final Sketch skA) {
+    if (skA == null) {
+      reset();
+      return;
+      //throw new SketchesArgumentException("The input argument may not be null");
+    }
+    if (skA.isEmpty()) {
+      reset();
+      return;
+    }
+    //skA is not empty
+    checkSeedHashes(seedHashR_, skA.getSeedHash());
+    emptyR_ = false;
+    thetaLongR_ = skA.getThetaLong();
+    final CompactSketch cskA = (skA instanceof CompactSketch)
+        ? (CompactSketch) skA
+        : ((UpdateSketch) skA).compact();
+    hashArrR_ = skA.isDirect() ? cskA.getCache() : cskA.getCache().clone();
+    curCountR_ = cskA.getRetainedEntries(true);
+  }
+
+  @Override
+  public void notB(final Sketch skB) {
+    if (emptyR_ || (skB == null) || skB.isEmpty()) { return; }
+    //skB is not empty
+    checkSeedHashes(seedHashR_, skB.getSeedHash());
+    final long thetaLongB = skB.getThetaLong();
+    thetaLongR_ = Math.min(thetaLongR_, thetaLongB);
+
+    //Build hashtable and removes hashes of skB >= theta
+    final int countB = skB.getRetainedEntries();
+    CompactSketch cskB = null;
+    UpdateSketch uskB = null;
+    final long[] hashTableB;
+    if (skB instanceof CompactSketch) {
+      cskB = (CompactSketch) skB;
+      hashTableB = convertToHashTable(cskB.getCache(), countB, thetaLongR_, REBUILD_THRESHOLD);
+    } else {
+      uskB = (UpdateSketch) skB;
+      hashTableB = (thetaLongR_ < thetaLongB)
+          ? convertToHashTable(uskB.getCache(), countB, thetaLongR_, REBUILD_THRESHOLD)
+          : uskB.getCache();
+      cskB = uskB.compact();
+    }
+
+    //build temporary arrays of skA
+    final long[] tmpHashArrA = new long[curCountR_];
+
+    //search for non matches and build temp arrays
+    final int lgHTBLen = simpleIntLog2(hashTableB.length);
+    int nonMatches = 0;
+    for (int i = 0; i < curCountR_; i++) {
+      final long hash = hashArrR_[i];
+      if ((hash != 0) && (hash < thetaLongR_)) { //skips hashes of A >= theta
+        final int index = hashSearch(hashTableB, lgHTBLen, hash);
+        if (index == -1) {
+          tmpHashArrA[nonMatches] = hash;
+          nonMatches++;
+        }
+      }
+    }
+    hashArrR_ = Arrays.copyOfRange(tmpHashArrA, 0, nonMatches);
+    curCountR_ = nonMatches;
+    emptyR_ = (nonMatches == 0) && (thetaLongR_ == Long.MAX_VALUE);
+  }
+
+  @Override
+  public CompactSketch getResult(final boolean reset) {
+    return getResult(true, null, reset);
+  }
+
+  @Override
+  public CompactSketch getResult(final boolean dstOrdered, final WritableMemory dstMem,
+      final boolean reset) {
+    final CompactSketch result =
+        getResult(hashArrR_, emptyR_, seedHashR_, curCountR_, thetaLongR_, dstOrdered, dstMem);
+    if (reset) { reset(); }
+    return result;
+  }
+
+  private static CompactSketch getResult(
+      final long[] hashArr,
+      final boolean empty,
+      final short seedHash,
+      final int curCount,
+      final long thetaLong,
+      final boolean dstOrdered,
+      final WritableMemory dstMem) {
+    final CompactSketch result;
+    if (dstMem == null) { //Heap
+      if (empty) { return EmptyCompactSketch.getInstance(); }
+      if (curCount == 1) { return new SingleItemSketch(hashArr[0], seedHash); }
+      //curCount > 1
+      if (dstOrdered) {
+        Arrays.sort(hashArr);
+        result = new HeapCompactOrderedSketch(hashArr, false, seedHash, curCount, thetaLong);
+      } else {
+        result = new HeapCompactUnorderedSketch(hashArr, false, seedHash, curCount, thetaLong);
+      }
+    }
+    else { //Direct
+      if (empty) {
+        dstMem.putByteArray(0, EmptyCompactSketch.EMPTY_COMPACT_SKETCH_ARR, 0, 8);
+        return EmptyCompactSketch.getInstance();
+      }
+      if (curCount == 1) {
+        final SingleItemSketch sis = new SingleItemSketch(hashArr[0], seedHash);
+        dstMem.putByteArray(0, sis.toByteArray(), 0, 16);
+      }
+      final int preLongs = Sketch.computeCompactPreLongs(thetaLong, false, curCount);
+      if (dstOrdered) {
+        final byte flags = (byte)(READ_ONLY_FLAG_MASK | COMPACT_FLAG_MASK | ORDERED_FLAG_MASK);
+        Arrays.sort(hashArr);
+        loadCompactMemory(hashArr, seedHash, curCount, thetaLong, dstMem, flags, preLongs);
+        result = new DirectCompactOrderedSketch(dstMem);
+      } else {
+        final byte flags = (byte)(READ_ONLY_FLAG_MASK | COMPACT_FLAG_MASK);
+        loadCompactMemory(hashArr, seedHash, curCount, thetaLong, dstMem, flags, preLongs);
+        result = new DirectCompactUnorderedSketch(dstMem);
+      }
+    }
+    return result;
+  }
+
+  @Override
+  public CompactSketch aNotB(final Sketch skA, final Sketch skB, final boolean dstOrdered,
+      final WritableMemory dstMem) {
+    if ((skA == null) || skA.isEmpty()) { return EmptyCompactSketch.getInstance(); }
+    if ((skB == null) || skB.isEmpty()) { return skA.compact(dstOrdered, dstMem); }
+    final short seedHashA = skA.getSeedHash();
+    final short seedHashB = skB.getSeedHash();
+    checkSeedHashes(seedHashA, seedHashR_);
+    checkSeedHashes(seedHashB, seedHashR_);
+
+    //Both skA & skB are not empty
+    //Load skA into local tmp registers
+    boolean empty = false;
+    final long thetaLongA = skA.getThetaLong();
+    final CompactSketch cskA = (skA instanceof CompactSketch)
+        ? (CompactSketch)skA
+        : ((UpdateSketch)skA).compact();
+    final long[] hashArrA = cskA.getCache().clone();
+    final int countA = cskA.getRetainedEntries();
+
+    //Compare with skB
+    final long thetaLongB = skB.getThetaLong();
+    final long thetaLong = Math.min(thetaLongA, thetaLongB);
+    final int countB = skB.getRetainedEntries();
+
+    //Rebuild hashtable and removes hashes of skB >= thetaLong
+    final long[] hashTableB = convertToHashTable(skB.getCache(), countB, thetaLong, REBUILD_THRESHOLD);
+
+    //build temporary hash array for values from skA
+    final long[] tmpHashArrA = new long[countA];
+
+    //search for non matches and build temp hash array
+    int nonMatches = 0;
+    for (int i = 0; i < countA; i++) {
+      final long hash = hashArrA[i];
+      if ((hash != 0) && (hash < thetaLong)) { //skips hashes of A >= theta
+        final int index =
+            hashSearch(hashTableB, simpleIntLog2(hashTableB.length), hash);
+        if (index == -1) {
+          tmpHashArrA[nonMatches] = hash;
+          nonMatches++;
+        }
+      }
+    }
+    //final compaction
+    empty = ((nonMatches == 0) && (thetaLong == Long.MAX_VALUE));
+    final long[] hashArrOut = Arrays.copyOfRange(tmpHashArrA, 0, nonMatches);
+    final CompactSketch result =
+        AnotBimpl.getResult(hashArrOut, empty, seedHashR_, nonMatches, thetaLong, dstOrdered, dstMem);
+    return result;
+  }
+
+  @Override
+  int getRetainedEntries(final boolean valid) {
+    return curCountR_;
+  }
+
+  @Override
+  boolean isEmpty() {
+    return emptyR_;
+  }
+
+  @Override
+  public boolean isSameResource(final Memory that) {
+    return false;
+  }
+
+  //Deprecated methods
+
+  @Override
+  public void update(final Sketch a, final Sketch b) {
+    skA_ = a;
+    skB_ = b;
+    thetaLongR_ = Long.MAX_VALUE;
+    emptyR_ = true;
+    hashArrR_ = null;
+    curCountR_ = 0;
+    lgArrLongsHT_ = 5;
+    bHashTable_ = null;
+    compute();
+  }
+
+  @Override
+  public CompactSketch getResult() {
+    return getResult(true, null);
+  }
+
+  @Override
+  public CompactSketch getResult(final boolean dstOrdered, final WritableMemory dstMem) {
+    final long[] compactCache = (curCountR_ <= 0)
+        ? new long[0]
+        : Arrays.copyOfRange(hashArrR_, 0, curCountR_);
+    if (dstOrdered && (curCountR_ > 1)) {
+      Arrays.sort(compactCache);
+    }
+    //Create the CompactSketch
+    final CompactSketch comp = createCompactSketch(
+        compactCache, emptyR_, seedHashR_, curCountR_, thetaLongR_, dstOrdered, dstMem);
+    reset(); //TODO
+    return comp;
+  }
+
+
+
+  //restricted
+
+  void compute() {
+    final int swA = ((skA_ == null) || (skA_ instanceof EmptyCompactSketch))
+        ? 0
+        : (skA_.isEmpty())
+          ? 1
+          : (skA_ instanceof UpdateSketch)
+            ? 4
+            : (skA_.isOrdered())
+              ? 3
+              : 2;
+    final int swB = ((skB_ == null) || (skB_ instanceof EmptyCompactSketch))
+        ? 0
+        : (skB_.isEmpty())
+          ? 1
+          : (skB_ instanceof UpdateSketch)
+            ? 4
+            : (skB_.isOrdered())
+              ? 3
+              : 2;
+    final int sw = (swA * 8) | swB;
+
+    //  NOTES:
+    //    In the table below, A and B refer to the two input sketches in the order A-not-B.
+    //    The Theta rule: min(ThetaA, ThetaB)
+    //    The Empty rule: Whatever the empty state of A is: E(A)
+    //    The Return triple is defined as: (Theta, Count, EmptyFlag).
+    //    bHashTable temporarily stores the values of B.
+    //    A sketch in stored form can be in one of 5 states.
+    //    Null is not actually a state, but is included for completeness.
+    //    Null is interpreted as {Theta = 1.0, count = 0, empty = true}.
+    //    In some cases the empty state may have Theta < 1.0 but it is ignored; count must be zero.
+    //    State:
+    //      0 N Null or instance of EmptyCompactSketch
+    //      1 E Empty bit set
+    //      2 C Compact, not ordered
+    //      3 O Compact Ordered
+    //      4 H Hash-Table
+    //
+    //A    B    swA  swB  Case  Actions
+    //N    N    0    0    0     Return (1.0, 0, T)
+    //N    E    0    1    1     CheckB,  Return (1.0, 0, T)
+    //N    C    0    2    2     CheckB,  Return (1.0, 0, T)
+    //N    O    0    3    3     CheckB,  Return (1.0, 0, T)
+    //N    H    0    4    4     CheckB,  Return (1.0, 0, T)
+    //E    N    1    0    8     CheckA,  Return (1.0, 0, T)
+    //E    E    1    1    9     CheckAB, Return (1.0, 0, T)
+    //E    C    1    2    10    CheckAB, Return (1.0, 0, T)
+    //E    O    1    3    11    CheckAB, Return (1.0, 0, T)
+    //E    H    1    4    12    CheckAB, Return (1.0, 0, T)
+    //C    N    2    0    16    CheckA,  Return (ThA, |A|, F), copyA
+    //C    E    2    1    17    CheckAB, Return (ThA, |A|, F)), copyA
+    //C    C    2    2    18    CheckAB, B -> H; => C,H; scanAllAsearchB()
+    //C    O    2    3    19    CheckAB, B -> H; => C,H; scanAllAsearchB()
+    //C    H    2    4    20    CheckAB, scanAllAsearchB()
+    //O    N    3    0    24    CheckA,  Return (ThA, |A|, F), copyA
+    //O    E    3    1    25    CheckAB, Return (ThA, |A|, F), copyA
+    //O    C    3    2    26    CheckAB, B -> H; => O,H; scanEarlyStopAsearchB()
+    //O    O    3    3    27    CheckAB, B -> H; => O,H; scanEarlyStopAsearchB()
+    //O    H    3    4    28    CheckAB, scanEarlyStopAsearchB()
+    //H    N    4    0    32    CheckA,  Return (ThA, |A|, F), copyA
+    //H    E    4    1    33    CheckAB, Return (ThA, |A|, F), copyA
+    //H    C    4    2    34    CheckAB, B -> H; => H,H; scanAllAsearchB()
+    //H    O    4    3    35    CheckAB, B -> H; => H,H; scanAllAsearchB()
+    //H    H    4    4    36    CheckAB, scanAllAsearchB()
+
+    switch (sw) {
+      case 0 :  //A Null, B Null;    Return (1.0, 0, T)
+        thetaLongR_ = Long.MAX_VALUE;
+        emptyR_ = true;
+        break;
+
+      case 10:   //A Empty, B Compact; CheckAB, Return (1.0, 0, T)
+      case 11:   //A Empty, B Ordered; CheckAB, Return (1.0, 0, T)
+      case 12:   //A Empty, B HashTbl; CheckAB, Return (1.0, 0, T)
+        checkSeedHashes(seedHashR_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        //$FALL-THROUGH$
+      case 1:    //A Null, B Empty;   CheckB,  Return (1.0, 0, T)
+      case 2:    //A Null, B Compact; CheckB,  Return (1.0, 0, T)
+      case 3:    //A Null, B Ordered; CheckB,  Return (1.0, 0, T)
+      case 4:    //A Null, B HashTbl; CheckB,  Return (1.0, 0, T)
+        checkSeedHashes(seedHashR_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        thetaLongR_ = Long.MAX_VALUE;
+        emptyR_ = true;
+        break;
+
+      case 9:   //A Empty, B Empty;   CheckAB, Return (1.0, 0, T)
+        checkSeedHashes(seedHashR_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        //$FALL-THROUGH$
+      case 8:   //A Empty, B Null;    CheckA,  Return (1.0, 0, T)
+        checkSeedHashes(seedHashR_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        thetaLongR_ = Long.MAX_VALUE;
+        emptyR_ = true;
+        break;
+
+      case 17:   //A Compact, B Empty; CheckAB, Return (ThA, |A|, F), copyA
+      case 25:   //A Ordered, B Empty; CheckAB, Return (ThA, |A|, F), copyA
+      case 33:  //A HashTbl, B Empty; CheckAB, Return (ThA, |A|, F), copyA
+        checkSeedHashes(seedHashR_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        //$FALL-THROUGH$
+      case 16:   //A Compact, B Null;  CheckA,  Return (ThA, |A|, F), copyA
+      case 24:   //A Ordered, B Null;  CheckA,  Return (ThA, |A|, F), copyA
+      case 32:  //A HashTbl, B Null;  CheckA,  Return (ThA, |A|, F), copyA
+        checkSeedHashes(seedHashR_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        thetaLongR_ = skA_.getThetaLong();
+        emptyR_ = false;
+        curCountR_ = skA_.getRetainedEntries(true);
+        hashArrR_ = compactCache(skA_.getCache(), curCountR_, thetaLongR_, false);
+        break;
+
+      case 18:   //A Compact, B Compact; CheckAB, B -> H; => C,H; scanAllAsearchB()
+      case 19:   //A Compact, B Ordered; CheckAB, B -> H; => C,H; scanAllAsearchB()
+      case 34:   //A HashTbl, B Compact; CheckAB, B -> H; => H,H; scanAllAsearchB()
+      case 35:  //A HashTbl, B Ordered; CheckAB, B -> H; => H,H; scanAllAsearchB()
+        checkSeedHashes(seedHashR_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        checkSeedHashes(seedHashR_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        thetaLongR_ = min(skA_.getThetaLong(), skB_.getThetaLong());
+        emptyR_ = false;
+        convertBtoHT();
+        scanAllAsearchB();
+        break;
+
+      case 26:   //A Ordered, B Compact; CheckAB, B -> H; => O,H; scanEarlyStopAsearchB()
+      case 27:  //A Ordered, B Ordered; CheckAB, B -> H; => O,H; scanEarlyStopAsearchB()
+        checkSeedHashes(seedHashR_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        checkSeedHashes(seedHashR_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        thetaLongR_ = min(skA_.getThetaLong(), skB_.getThetaLong());
+        emptyR_ = false;
+        convertBtoHT();
+        scanEarlyStopAsearchB();
+        break;
+
+      case 20:   //A Compact, B HashTbl; CheckAB, scanAllAsearchB()
+      case 36:  //A HashTbl, B HashTbl; CheckAB, scanAllAsearchB()
+        checkSeedHashes(seedHashR_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        checkSeedHashes(seedHashR_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        thetaLongR_ = min(skA_.getThetaLong(), skB_.getThetaLong());
+        emptyR_ = false;
+        lgArrLongsHT_ = ((UpdateSketch)skB_).getLgArrLongs();
+        bHashTable_ = skB_.getCache();
+        scanAllAsearchB();
+        break;
+
+      case 28:  //A Ordered, B HashTbl; CheckAB, scanEarlyStopAsearchB()
+        checkSeedHashes(seedHashR_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        checkSeedHashes(seedHashR_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        thetaLongR_ = min(skA_.getThetaLong(), skB_.getThetaLong());
+        emptyR_ = false;
+        lgArrLongsHT_ = ((UpdateSketch)skB_).getLgArrLongs();
+        bHashTable_ = skB_.getCache();
+        scanEarlyStopAsearchB();
+        break;
+
+      //default: //This cannot happen and cannot be tested
+    }
+  }
+
+  private void convertBtoHT() {
+    final int curCountB = skB_.getRetainedEntries(true);
+    lgArrLongsHT_ = computeMinLgArrLongsFromCount(curCountB);
+    bHashTable_ = new long[1 << lgArrLongsHT_];
+    hashArrayInsert(skB_.getCache(), bHashTable_, lgArrLongsHT_, thetaLongR_);
+  }
+
+  //Sketch A is either unordered compact or hash table
+  private void scanAllAsearchB() {
+    final long[] scanAArr = skA_.getCache();
+    final int arrLongsIn = scanAArr.length;
+    hashArrR_ = new long[arrLongsIn];
+    for (int i = 0; i < arrLongsIn; i++ ) {
+      final long hashIn = scanAArr[i];
+      if ((hashIn <= 0L) || (hashIn >= thetaLongR_)) { continue; }
+      final int foundIdx = hashSearch(bHashTable_, lgArrLongsHT_, hashIn);
+      if (foundIdx > -1) { continue; }
+      hashArrR_[curCountR_++] = hashIn;
+    }
+  }
+
+  //Sketch A is ordered compact, which enables early stop
+  private void scanEarlyStopAsearchB() {
+    final long[] scanAArr = skA_.getCache();
+    final int arrLongsIn = scanAArr.length;
+    hashArrR_ = new long[arrLongsIn]; //maybe 2x what is needed, but getRetainedEntries can be slow.
+    for (int i = 0; i < arrLongsIn; i++ ) {
+      final long hashIn = scanAArr[i];
+      if (hashIn <= 0L) { continue; }
+      if (hashIn >= thetaLongR_) {
+        break; //early stop assumes that hashes in input sketch are ordered!
+      }
+      final int foundIdx = hashSearch(bHashTable_, lgArrLongsHT_, hashIn);
+      if (foundIdx > -1) { continue; }
+      hashArrR_[curCountR_++] = hashIn;
+    }
+  }
+
+  private void reset() {
+    skA_ = null;
+    skB_ = null;
+    thetaLongR_ = Long.MAX_VALUE;
+    emptyR_ = true;
+    hashArrR_ = null;
+    curCountR_ = 0;
+    lgArrLongsHT_ = 5;
+    bHashTable_ = null;
+  }
+
+  @Override
+  long[] getCache() {
+    return null;
+  }
+
+  @Override
+  short getSeedHash() {
+    return seedHashR_;
+  }
+
+  @Override
+  long getThetaLong() {
+    return thetaLongR_;
+  }
+
+}
diff --git a/src/main/java/org/apache/datasketches/theta/CompactSketch.java b/src/main/java/org/apache/datasketches/theta/CompactSketch.java
index dab61af..92510a2 100644
--- a/src/main/java/org/apache/datasketches/theta/CompactSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/CompactSketch.java
@@ -135,7 +135,8 @@ public abstract class CompactSketch extends Sketch {
     return cacheOut;
   }
 
-  //compactCache and dstMem must be valid
+  //All arguments must be valid and correct.
+  // Used as helper to create byte arrays as well as loading Memory for direct compact sketches
   static final Memory loadCompactMemory(final long[] compactCache, final short seedHash,
       final int curCount, final long thetaLong, final WritableMemory dstMem,
       final byte flags, final int preLongs) {
diff --git a/src/main/java/org/apache/datasketches/theta/DirectCompactOrderedSketch.java b/src/main/java/org/apache/datasketches/theta/DirectCompactOrderedSketch.java
index 6a88213..27f6451 100644
--- a/src/main/java/org/apache/datasketches/theta/DirectCompactOrderedSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/DirectCompactOrderedSketch.java
@@ -23,14 +23,15 @@ import static org.apache.datasketches.theta.PreambleUtil.COMPACT_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.EMPTY_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.ORDERED_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.READ_ONLY_FLAG_MASK;
+import static org.apache.datasketches.theta.PreambleUtil.SINGLEITEM_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.checkMemorySeedHash;
 
 import org.apache.datasketches.memory.Memory;
 import org.apache.datasketches.memory.WritableMemory;
 
 /**
- * An off-heap (Direct), compact, ordered, read-only sketch. This sketch may be associated
- * with Serialization Version 3 format binary image.
+ * An off-heap (Direct), compact, ordered, read-only sketch.  This sketch can only be associated
+ * with a Serialization Version 3 format binary image.
  *
  * <p>This implementation uses data in a given Memory that is owned and managed by the caller.
  * This Memory can be off-heap, which if managed properly will greatly reduce the need for
@@ -48,7 +49,8 @@ final class DirectCompactOrderedSketch extends DirectCompactSketch {
    * Wraps the given Memory, which must be a SerVer 3, ordered, Compact Sketch image.
    * Must check the validity of the Memory before calling.
    * @param srcMem <a href="{@docRoot}/resources/dictionary.html#mem">See Memory</a>
-   * @param seed the update seed
+   * @param seed The update seed.
+   * <a href="{@docRoot}/resources/dictionary.html#seed">See Update Hash Seed</a>.
    * @return this sketch
    */
   static DirectCompactOrderedSketch wrapInstance(final Memory srcMem, final long seed) {
@@ -68,17 +70,21 @@ final class DirectCompactOrderedSketch extends DirectCompactSketch {
    * @param dstMem the given destination Memory. This clears it before use.
    * @return a DirectCompactOrderedSketch
    */
-  static DirectCompactOrderedSketch compact(final long[] cache, final boolean empty,
-      final short seedHash, final int curCount, final long thetaLong, final WritableMemory dstMem) {
+  static DirectCompactOrderedSketch compact(
+      final long[] cache,
+      final boolean empty,
+      final short seedHash,
+      final int curCount,
+      final long thetaLong,
+      final WritableMemory dstMem) {
     final int preLongs = computeCompactPreLongs(thetaLong, empty, curCount);
-    final int requiredFlags = READ_ONLY_FLAG_MASK | COMPACT_FLAG_MASK | ORDERED_FLAG_MASK;
-    final byte flags = (byte) (requiredFlags | (empty ? EMPTY_FLAG_MASK : 0));
-    loadCompactMemory(cache, seedHash, curCount, thetaLong, dstMem, flags, preLongs);
+    int flags = READ_ONLY_FLAG_MASK | COMPACT_FLAG_MASK | ORDERED_FLAG_MASK;
+    flags |= empty ? EMPTY_FLAG_MASK : 0;
+    flags |= (curCount == 1) ? SINGLEITEM_FLAG_MASK : 0;
+    loadCompactMemory(cache, seedHash, curCount, thetaLong, dstMem, (byte)flags, preLongs);
     return new DirectCompactOrderedSketch(dstMem);
   }
 
-  //restricted methods
-
   @Override
   public boolean isOrdered() {
     return true;
diff --git a/src/main/java/org/apache/datasketches/theta/DirectCompactSketch.java b/src/main/java/org/apache/datasketches/theta/DirectCompactSketch.java
index 88b8d11..4bfb514 100644
--- a/src/main/java/org/apache/datasketches/theta/DirectCompactSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/DirectCompactSketch.java
@@ -39,6 +39,7 @@ abstract class DirectCompactSketch extends CompactSketch {
   }
 
   //Sketch
+
   //overidden by EmptyCompactSketch and SingleItemSketch
   @Override
   public int getCurrentBytes(final boolean compact) { //compact is ignored here
@@ -81,7 +82,7 @@ abstract class DirectCompactSketch extends CompactSketch {
 
   @Override
   public boolean isEmpty() {
-    return PreambleUtil.isEmpty(mem_);
+    return PreambleUtil.isEmptySketch(mem_);
   }
 
   @Override
@@ -107,6 +108,9 @@ abstract class DirectCompactSketch extends CompactSketch {
 
   //restricted methods
 
+
+
+
   @Override
   long[] getCache() {
     final int curCount = getRetainedEntries(true);
diff --git a/src/main/java/org/apache/datasketches/theta/DirectCompactUnorderedSketch.java b/src/main/java/org/apache/datasketches/theta/DirectCompactUnorderedSketch.java
index eb9d425..b1bc10c 100644
--- a/src/main/java/org/apache/datasketches/theta/DirectCompactUnorderedSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/DirectCompactUnorderedSketch.java
@@ -22,6 +22,7 @@ package org.apache.datasketches.theta;
 import static org.apache.datasketches.theta.PreambleUtil.COMPACT_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.EMPTY_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.READ_ONLY_FLAG_MASK;
+import static org.apache.datasketches.theta.PreambleUtil.SINGLEITEM_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.checkMemorySeedHash;
 
 import org.apache.datasketches.memory.Memory;
@@ -47,7 +48,8 @@ final class DirectCompactUnorderedSketch extends DirectCompactSketch {
    * Wraps the given Memory, which must be a SerVer 3, unordered, Compact Sketch image.
    * Must check the validity of the Memory before calling.
    * @param srcMem <a href="{@docRoot}/resources/dictionary.html#mem">See Memory</a>
-   * @param seed <a href="{@docRoot}/resources/dictionary.html#seed">See Update Hash Seed</a>.
+   * @param seed The update seed.
+   * <a href="{@docRoot}/resources/dictionary.html#seed">See Update Hash Seed</a>.
    * @return this sketch
    */
   static DirectCompactUnorderedSketch wrapInstance(final Memory srcMem, final long seed) {
@@ -67,12 +69,18 @@ final class DirectCompactUnorderedSketch extends DirectCompactSketch {
    * @param dstMem the given destination Memory. This clears it before use.
    * @return a DirectCompactUnorderedSketch
    */
-  static DirectCompactUnorderedSketch compact(final long[] cache, final boolean empty,
-      final short seedHash, final int curCount, final long thetaLong, final WritableMemory dstMem) {
+  static DirectCompactUnorderedSketch compact(
+      final long[] cache,
+      final boolean empty,
+      final short seedHash,
+      final int curCount,
+      final long thetaLong,
+      final WritableMemory dstMem) {
     final int preLongs = computeCompactPreLongs(thetaLong, empty, curCount);
-    final int requiredFlags = READ_ONLY_FLAG_MASK | COMPACT_FLAG_MASK;
-    final byte flags = (byte) (requiredFlags | (empty ? EMPTY_FLAG_MASK : 0));
-    loadCompactMemory(cache, seedHash, curCount, thetaLong, dstMem, flags, preLongs);
+    int flags = READ_ONLY_FLAG_MASK | COMPACT_FLAG_MASK;
+    flags |= empty ? EMPTY_FLAG_MASK : 0;
+    flags |= (curCount == 1) ? SINGLEITEM_FLAG_MASK : 0;
+    loadCompactMemory(cache, seedHash, curCount, thetaLong, dstMem, (byte)flags, preLongs);
     return new DirectCompactUnorderedSketch(dstMem);
   }
 
diff --git a/src/main/java/org/apache/datasketches/theta/DirectQuickSelectSketch.java b/src/main/java/org/apache/datasketches/theta/DirectQuickSelectSketch.java
index f731d45..f80473a 100644
--- a/src/main/java/org/apache/datasketches/theta/DirectQuickSelectSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/DirectQuickSelectSketch.java
@@ -19,10 +19,12 @@
 
 package org.apache.datasketches.theta;
 
+import static org.apache.datasketches.Util.LONG_MAX_VALUE_AS_DOUBLE;
 import static org.apache.datasketches.Util.MIN_LG_ARR_LONGS;
+import static org.apache.datasketches.Util.computeSeedHash;
+import static org.apache.datasketches.Util.startingSubMultiple;
 import static org.apache.datasketches.theta.PreambleUtil.EMPTY_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.FLAGS_BYTE;
-import static org.apache.datasketches.theta.PreambleUtil.MAX_THETA_LONG_AS_DOUBLE;
 import static org.apache.datasketches.theta.PreambleUtil.PREAMBLE_LONGS_BYTE;
 import static org.apache.datasketches.theta.PreambleUtil.P_FLOAT;
 import static org.apache.datasketches.theta.PreambleUtil.RETAINED_ENTRIES_INT;
@@ -57,7 +59,6 @@ import org.apache.datasketches.Family;
 import org.apache.datasketches.HashOperations;
 import org.apache.datasketches.ResizeFactor;
 import org.apache.datasketches.SketchesArgumentException;
-import org.apache.datasketches.Util;
 import org.apache.datasketches.memory.MemoryRequestServer;
 import org.apache.datasketches.memory.WritableMemory;
 
@@ -142,10 +143,10 @@ class DirectQuickSelectSketch extends DirectQuickSelectSketchR {
     insertLgArrLongs(dstMem, lgArrLongs);                  //byte 4
     //flags: bigEndian = readOnly = compact = ordered = false; empty = true : 00100 = 4
     insertFlags(dstMem, EMPTY_FLAG_MASK);                  //byte 5
-    insertSeedHash(dstMem, Util.computeSeedHash(seed));    //bytes 6,7
+    insertSeedHash(dstMem, computeSeedHash(seed));    //bytes 6,7
     insertCurCount(dstMem, 0);                             //bytes 8-11
     insertP(dstMem, p);                                    //bytes 12-15
-    final long thetaLong = (long)(p * MAX_THETA_LONG_AS_DOUBLE);
+    final long thetaLong = (long)(p * LONG_MAX_VALUE_AS_DOUBLE);
     insertThetaLong(dstMem, thetaLong);                    //bytes 16-23
     if (unionGadget) {
       insertUnionThetaLong(dstMem, thetaLong);
@@ -178,7 +179,7 @@ class DirectQuickSelectSketch extends DirectQuickSelectSketchR {
     final int lgRF = extractLgResizeFactor(srcMem);               //byte 0
     final ResizeFactor myRF = ResizeFactor.getRF(lgRF);
     if ((myRF == ResizeFactor.X1)
-            && (lgArrLongs != Util.startingSubMultiple(lgNomLongs + 1, myRF, MIN_LG_ARR_LONGS))) {
+            && (lgArrLongs != startingSubMultiple(lgNomLongs + 1, myRF, MIN_LG_ARR_LONGS))) {
       insertLgResizeFactor(srcMem, ResizeFactor.X2.lg());
     }
 
@@ -234,7 +235,7 @@ class DirectQuickSelectSketch extends DirectQuickSelectSketchR {
     wmem_.putByte(FLAGS_BYTE, (byte) EMPTY_FLAG_MASK);
     wmem_.putInt(RETAINED_ENTRIES_INT, 0);
     final float p = wmem_.getFloat(P_FLOAT);
-    final long thetaLong = (long) (p * MAX_THETA_LONG_AS_DOUBLE);
+    final long thetaLong = (long) (p * LONG_MAX_VALUE_AS_DOUBLE);
     wmem_.putLong(THETA_LONG, thetaLong);
   }
 
diff --git a/src/main/java/org/apache/datasketches/theta/DirectQuickSelectSketchR.java b/src/main/java/org/apache/datasketches/theta/DirectQuickSelectSketchR.java
index 79b4e82..52cc536 100644
--- a/src/main/java/org/apache/datasketches/theta/DirectQuickSelectSketchR.java
+++ b/src/main/java/org/apache/datasketches/theta/DirectQuickSelectSketchR.java
@@ -155,7 +155,7 @@ class DirectQuickSelectSketchR extends UpdateSketch {
 
   @Override
   public boolean isEmpty() {
-    return PreambleUtil.isEmpty(wmem_);
+    return PreambleUtil.isEmptySketch(wmem_);
   }
 
   @Override
diff --git a/src/main/java/org/apache/datasketches/theta/ForwardCompatibility.java b/src/main/java/org/apache/datasketches/theta/ForwardCompatibility.java
index 97c5525..733dfc1 100644
--- a/src/main/java/org/apache/datasketches/theta/ForwardCompatibility.java
+++ b/src/main/java/org/apache/datasketches/theta/ForwardCompatibility.java
@@ -100,8 +100,7 @@ final class ForwardCompatibility {
 
     final long[] compactOrderedCache = new long[curCount];
     srcMem.getLongArray(preLongs << 3, compactOrderedCache, 0, curCount);
-    return HeapCompactOrderedSketch
-        .compact(compactOrderedCache, false, seedHash, curCount, thetaLong);
+    return new HeapCompactOrderedSketch(compactOrderedCache, false, seedHash, curCount, thetaLong);
   }
 
   /**
@@ -147,8 +146,7 @@ final class ForwardCompatibility {
       validateInputSize(reqBytesIn, memCap);
       final long[] compactOrderedCache = new long[curCount];
       srcMem.getLongArray(preLongs << 3, compactOrderedCache, 0, curCount);
-      return HeapCompactOrderedSketch
-          .compact(compactOrderedCache, false, seedHash, curCount, thetaLong);
+      return new HeapCompactOrderedSketch(compactOrderedCache, false, seedHash, curCount, thetaLong);
     }
     if (preLongs == 3) { //pre0 + count + theta
       reqBytesIn = (preLongs) << 3; //
@@ -169,8 +167,7 @@ final class ForwardCompatibility {
       validateInputSize(reqBytesIn, memCap);
       final long[] compactOrderedCache = new long[curCount];
       srcMem.getLongArray(preLongs << 3, compactOrderedCache, 0, curCount);
-      return HeapCompactOrderedSketch
-          .compact(compactOrderedCache, false, seedHash, curCount, thetaLong);
+      return new HeapCompactOrderedSketch(compactOrderedCache, false, seedHash, curCount, thetaLong);
     }
     throw new SketchesArgumentException("PreLongs must be 1,2, or 3: " + preLongs);
   }
diff --git a/src/main/java/org/apache/datasketches/theta/HeapAlphaSketch.java b/src/main/java/org/apache/datasketches/theta/HeapAlphaSketch.java
index 21057d2..78ba811 100644
--- a/src/main/java/org/apache/datasketches/theta/HeapAlphaSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/HeapAlphaSketch.java
@@ -23,10 +23,11 @@ import static java.lang.Math.max;
 import static java.lang.Math.min;
 import static java.lang.Math.sqrt;
 import static org.apache.datasketches.HashOperations.STRIDE_MASK;
+import static org.apache.datasketches.Util.LONG_MAX_VALUE_AS_DOUBLE;
 import static org.apache.datasketches.Util.MIN_LG_ARR_LONGS;
 import static org.apache.datasketches.Util.REBUILD_THRESHOLD;
 import static org.apache.datasketches.Util.RESIZE_THRESHOLD;
-import static org.apache.datasketches.theta.PreambleUtil.MAX_THETA_LONG_AS_DOUBLE;
+import static org.apache.datasketches.Util.startingSubMultiple;
 import static org.apache.datasketches.theta.PreambleUtil.extractCurCount;
 import static org.apache.datasketches.theta.PreambleUtil.extractFamilyID;
 import static org.apache.datasketches.theta.PreambleUtil.extractLgArrLongs;
@@ -44,7 +45,6 @@ import org.apache.datasketches.Family;
 import org.apache.datasketches.HashOperations;
 import org.apache.datasketches.ResizeFactor;
 import org.apache.datasketches.SketchesArgumentException;
-import org.apache.datasketches.Util;
 import org.apache.datasketches.memory.Memory;
 import org.apache.datasketches.memory.WritableMemory;
 
@@ -98,15 +98,15 @@ final class HeapAlphaSketch extends HeapUpdateSketch {
 
     final double nomLongs = (1L << lgNomLongs);
     final double alpha = nomLongs / (nomLongs + 1.0);
-    final long split1 = (long) (((p * (alpha + 1.0)) / 2.0) * MAX_THETA_LONG_AS_DOUBLE);
+    final long split1 = (long) (((p * (alpha + 1.0)) / 2.0) * LONG_MAX_VALUE_AS_DOUBLE);
 
     final HeapAlphaSketch has = new HeapAlphaSketch(lgNomLongs, seed, p, rf, alpha, split1);
 
-    final int lgArrLongs = Util.startingSubMultiple(lgNomLongs + 1, rf, MIN_LG_ARR_LONGS);
+    final int lgArrLongs = startingSubMultiple(lgNomLongs + 1, rf, MIN_LG_ARR_LONGS);
     has.lgArrLongs_ = lgArrLongs;
     has.hashTableThreshold_ = setHashTableThreshold(lgNomLongs, lgArrLongs);
     has.curCount_ = 0;
-    has.thetaLong_ = (long)(p * MAX_THETA_LONG_AS_DOUBLE);
+    has.thetaLong_ = (long)(p * LONG_MAX_VALUE_AS_DOUBLE);
     has.empty_ = true; //other flags: bigEndian = readOnly = compact = ordered = false;
     has.cache_ = new long[1 << lgArrLongs];
     return has;
@@ -133,10 +133,10 @@ final class HeapAlphaSketch extends HeapUpdateSketch {
 
     final double nomLongs = (1L << lgNomLongs);
     final double alpha = nomLongs / (nomLongs + 1.0);
-    final long split1 = (long) (((p * (alpha + 1.0)) / 2.0) * MAX_THETA_LONG_AS_DOUBLE);
+    final long split1 = (long) (((p * (alpha + 1.0)) / 2.0) * LONG_MAX_VALUE_AS_DOUBLE);
 
     if ((myRF == ResizeFactor.X1)
-            && (lgArrLongs != Util.startingSubMultiple(lgNomLongs + 1, myRF, MIN_LG_ARR_LONGS))) {
+            && (lgArrLongs != startingSubMultiple(lgNomLongs + 1, myRF, MIN_LG_ARR_LONGS))) {
       throw new SketchesArgumentException("Possible corruption: ResizeFactor X1, but provided "
               + "array too small for sketch size");
     }
@@ -146,7 +146,7 @@ final class HeapAlphaSketch extends HeapUpdateSketch {
     has.hashTableThreshold_ = setHashTableThreshold(lgNomLongs, lgArrLongs);
     has.curCount_ = extractCurCount(srcMem);
     has.thetaLong_ = extractThetaLong(srcMem);
-    has.empty_ = PreambleUtil.isEmpty(srcMem);
+    has.empty_ = PreambleUtil.isEmptySketch(srcMem);
     has.cache_ = new long[1 << lgArrLongs];
     srcMem.getLongArray(preambleLongs << 3, has.cache_, 0, 1 << lgArrLongs); //read in as hash table
     return has;
@@ -168,7 +168,7 @@ final class HeapAlphaSketch extends HeapUpdateSketch {
   public double getEstimate() {
     return (thetaLong_ > split1_)
         ? Sketch.estimate(thetaLong_, curCount_)
-        : (1 << lgNomLongs_) * (MAX_THETA_LONG_AS_DOUBLE / thetaLong_);
+        : (1 << lgNomLongs_) * (LONG_MAX_VALUE_AS_DOUBLE / thetaLong_);
   }
 
   @Override
@@ -247,7 +247,7 @@ final class HeapAlphaSketch extends HeapUpdateSketch {
   @Override
   public final void reset() {
     final int lgArrLongs =
-        Util.startingSubMultiple(lgNomLongs_ + 1, getResizeFactor(), MIN_LG_ARR_LONGS);
+        startingSubMultiple(lgNomLongs_ + 1, getResizeFactor(), MIN_LG_ARR_LONGS);
     if (lgArrLongs == lgArrLongs_) {
       final int arrLongs = cache_.length;
       assert (1 << lgArrLongs_) == arrLongs;
@@ -260,7 +260,7 @@ final class HeapAlphaSketch extends HeapUpdateSketch {
     hashTableThreshold_ = setHashTableThreshold(lgNomLongs_, lgArrLongs_);
     empty_ = true;
     curCount_ = 0;
-    thetaLong_ =  (long)(getP() * MAX_THETA_LONG_AS_DOUBLE);
+    thetaLong_ =  (long)(getP() * LONG_MAX_VALUE_AS_DOUBLE);
     dirty_ = false;
   }
 
diff --git a/src/main/java/org/apache/datasketches/theta/HeapAnotB.java b/src/main/java/org/apache/datasketches/theta/HeapAnotB.java
deleted file mode 100644
index d657daa..0000000
--- a/src/main/java/org/apache/datasketches/theta/HeapAnotB.java
+++ /dev/null
@@ -1,345 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *   http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.datasketches.theta;
-
-import static java.lang.Math.min;
-import static org.apache.datasketches.HashOperations.hashArrayInsert;
-import static org.apache.datasketches.HashOperations.hashSearch;
-import static org.apache.datasketches.theta.CompactSketch.compactCache;
-
-import java.util.Arrays;
-
-import org.apache.datasketches.Util;
-import org.apache.datasketches.memory.Memory;
-import org.apache.datasketches.memory.WritableMemory;
-
-/**
- * @author Lee Rhodes
- * @author Kevin Lang
- */
-final class HeapAnotB extends AnotB {
-  private final short seedHash_;
-  private Sketch a_;
-  private Sketch b_;
-  private long thetaLong_;
-  private boolean empty_;
-  private long[] cache_; // no match set
-  private int curCount_;
-
-  private int lgArrLongsHT_; //for Hash Table only. may not need to be member after refactoring
-  private long[] bHashTable_; //may not need to be member after refactoring.
-
-  /**
-   * Construct a new AnotB SetOperation on the java heap.  Called by SetOperation.Builder.
-   *
-   * @param seed <a href="{@docRoot}/resources/dictionary.html#seed">See seed</a>
-   */
-  HeapAnotB(final long seed) {
-    this(Util.computeSeedHash(seed));
-  }
-
-  /**
-   * Construct a new AnotB SetOperation on the java heap.  Called by PairwiseSetOperation.
-   *
-   * @param seedHash 16 bit hash of the chosen update seed.
-   */
-  HeapAnotB(final short seedHash) {
-    seedHash_ = seedHash;
-    reset();
-  }
-
-  @Override
-  public void update(final Sketch a, final Sketch b) {
-    a_ = a;
-    b_ = b;
-    thetaLong_ = Long.MAX_VALUE;
-    empty_ = true;
-    cache_ = null;
-    curCount_ = 0;
-    lgArrLongsHT_ = 5;
-    bHashTable_ = null;
-    compute();
-  }
-
-  @Override
-  public CompactSketch aNotB(final Sketch a, final Sketch b, final boolean dstOrdered,
-      final WritableMemory dstMem) {
-    update(a, b);
-    return getResult(dstOrdered, dstMem);
-  }
-
-  @Override
-  public CompactSketch getResult() {
-    return getResult(true, null);
-  }
-
-  @Override
-  public CompactSketch getResult(final boolean dstOrdered, final WritableMemory dstMem) {
-    final long[] compactCache = (curCount_ <= 0)
-        ? new long[0]
-        : Arrays.copyOfRange(cache_, 0, curCount_);
-    if (dstOrdered && (curCount_ > 1)) {
-      Arrays.sort(compactCache);
-    }
-    //Create the CompactSketch
-    final CompactSketch comp = createCompactSketch(
-        compactCache, empty_, seedHash_, curCount_, thetaLong_, dstOrdered, dstMem);
-    reset(); //TODO
-    return comp;
-  }
-
-  @Override
-  int getRetainedEntries(final boolean valid) {
-    return curCount_;
-  }
-
-  @Override
-  boolean isEmpty() {
-    return empty_;
-  }
-
-  @Override
-  public boolean isSameResource(final Memory that) {
-    return false;
-  }
-
-  //restricted
-
-  void compute() {
-    final int swA = ((a_ == null) || (a_ instanceof EmptyCompactSketch))
-        ? 0
-        : (a_.isEmpty())
-          ? 1
-          : (a_ instanceof UpdateSketch)
-            ? 4
-            : (a_.isOrdered())
-              ? 3
-              : 2;
-    final int swB = ((b_ == null) || (b_ instanceof EmptyCompactSketch))
-        ? 0
-        : (b_.isEmpty())
-          ? 1
-          : (b_ instanceof UpdateSketch)
-            ? 4
-            : (b_.isOrdered())
-              ? 3
-              : 2;
-    final int sw = (swA * 8) | swB;
-
-    //  NOTES:
-    //    In the table below, A and B refer to the two input sketches in the order A-not-B.
-    //    The Theta rule: min(ThetaA, ThetaB)
-    //    The Empty rule: Whatever the empty state of A is: E(A)
-    //    The Return triple is defined as: (Theta, Count, EmptyFlag).
-    //    bHashTable temporarily stores the values of B.
-    //    A sketch in stored form can be in one of 5 states.
-    //    Null is not actually a state, but is included for completeness.
-    //    Null is interpreted as {Theta = 1.0, count = 0, empty = true}.
-    //    In some cases the empty state may have Theta < 1.0 but it is ignored; count must be zero.
-    //    State:
-    //      0 N Null or instance of EmptyCompactSketch
-    //      1 E Empty bit set
-    //      2 C Compact, not ordered
-    //      3 O Compact Ordered
-    //      4 H Hash-Table
-    //
-    //A    B    swA  swB  Case  Actions
-    //N    N    0    0    0     Return (1.0, 0, T)
-    //N    E    0    1    1     CheckB,  Return (1.0, 0, T)
-    //N    C    0    2    2     CheckB,  Return (1.0, 0, T)
-    //N    O    0    3    3     CheckB,  Return (1.0, 0, T)
-    //N    H    0    4    4     CheckB,  Return (1.0, 0, T)
-    //E    N    1    0    8     CheckA,  Return (1.0, 0, T)
-    //E    E    1    1    9     CheckAB, Return (1.0, 0, T)
-    //E    C    1    2    10    CheckAB, Return (1.0, 0, T)
-    //E    O    1    3    11    CheckAB, Return (1.0, 0, T)
-    //E    H    1    4    12    CheckAB, Return (1.0, 0, T)
-    //C    N    2    0    16    CheckA,  Return (ThA, |A|, F), copyA
-    //C    E    2    1    17    CheckAB, Return (ThA, |A|, F)), copyA
-    //C    C    2    2    18    CheckAB, B -> H; => C,H; scanAllAsearchB()
-    //C    O    2    3    19    CheckAB, B -> H; => C,H; scanAllAsearchB()
-    //C    H    2    4    20    CheckAB, scanAllAsearchB()
-    //O    N    3    0    24    CheckA,  Return (ThA, |A|, F), copyA
-    //O    E    3    1    25    CheckAB, Return (ThA, |A|, F), copyA
-    //O    C    3    2    26    CheckAB, B -> H; => O,H; scanEarlyStopAsearchB()
-    //O    O    3    3    27    CheckAB, B -> H; => O,H; scanEarlyStopAsearchB()
-    //O    H    3    4    28    CheckAB, scanEarlyStopAsearchB()
-    //H    N    4    0    32    CheckA,  Return (ThA, |A|, F), copyA
-    //H    E    4    1    33    CheckAB, Return (ThA, |A|, F), copyA
-    //H    C    4    2    34    CheckAB, B -> H; => H,H; scanAllAsearchB()
-    //H    O    4    3    35    CheckAB, B -> H; => H,H; scanAllAsearchB()
-    //H    H    4    4    36    CheckAB, scanAllAsearchB()
-
-    switch (sw) {
-      case 0 :  //A Null, B Null;    Return (1.0, 0, T)
-        thetaLong_ = Long.MAX_VALUE;
-        empty_ = true;
-        break;
-
-      case 10:   //A Empty, B Compact; CheckAB, Return (1.0, 0, T)
-      case 11:   //A Empty, B Ordered; CheckAB, Return (1.0, 0, T)
-      case 12:   //A Empty, B HashTbl; CheckAB, Return (1.0, 0, T)
-        Util.checkSeedHashes(seedHash_, a_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        //$FALL-THROUGH$
-      case 1:    //A Null, B Empty;   CheckB,  Return (1.0, 0, T)
-      case 2:    //A Null, B Compact; CheckB,  Return (1.0, 0, T)
-      case 3:    //A Null, B Ordered; CheckB,  Return (1.0, 0, T)
-      case 4:    //A Null, B HashTbl; CheckB,  Return (1.0, 0, T)
-        Util.checkSeedHashes(seedHash_, b_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        thetaLong_ = Long.MAX_VALUE;
-        empty_ = true;
-        break;
-
-      case 9:   //A Empty, B Empty;   CheckAB, Return (1.0, 0, T)
-        Util.checkSeedHashes(seedHash_, b_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        //$FALL-THROUGH$
-      case 8:   //A Empty, B Null;    CheckA,  Return (1.0, 0, T)
-        Util.checkSeedHashes(seedHash_, a_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        thetaLong_ = Long.MAX_VALUE;
-        empty_ = true;
-        break;
-
-      case 17:   //A Compact, B Empty; CheckAB, Return (ThA, |A|, F), copyA
-      case 25:   //A Ordered, B Empty; CheckAB, Return (ThA, |A|, F), copyA
-      case 33:  //A HashTbl, B Empty; CheckAB, Return (ThA, |A|, F), copyA
-        Util.checkSeedHashes(seedHash_, b_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        //$FALL-THROUGH$
-      case 16:   //A Compact, B Null;  CheckA,  Return (ThA, |A|, F), copyA
-      case 24:   //A Ordered, B Null;  CheckA,  Return (ThA, |A|, F), copyA
-      case 32:  //A HashTbl, B Null;  CheckA,  Return (ThA, |A|, F), copyA
-        Util.checkSeedHashes(seedHash_, a_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        thetaLong_ = a_.getThetaLong();
-        empty_ = false;
-        curCount_ = a_.getRetainedEntries(true);
-        cache_ = compactCache(a_.getCache(), curCount_, thetaLong_, false);
-        break;
-
-      case 18:   //A Compact, B Compact; CheckAB, B -> H; => C,H; scanAllAsearchB()
-      case 19:   //A Compact, B Ordered; CheckAB, B -> H; => C,H; scanAllAsearchB()
-      case 34:   //A HashTbl, B Compact; CheckAB, B -> H; => H,H; scanAllAsearchB()
-      case 35:  //A HashTbl, B Ordered; CheckAB, B -> H; => H,H; scanAllAsearchB()
-        Util.checkSeedHashes(seedHash_, a_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        Util.checkSeedHashes(seedHash_, b_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        thetaLong_ = min(a_.getThetaLong(), b_.getThetaLong());
-        empty_ = false;
-        convertBtoHT();
-        scanAllAsearchB();
-        break;
-
-      case 26:   //A Ordered, B Compact; CheckAB, B -> H; => O,H; scanEarlyStopAsearchB()
-      case 27:  //A Ordered, B Ordered; CheckAB, B -> H; => O,H; scanEarlyStopAsearchB()
-        Util.checkSeedHashes(seedHash_, a_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        Util.checkSeedHashes(seedHash_, b_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        thetaLong_ = min(a_.getThetaLong(), b_.getThetaLong());
-        empty_ = false;
-        convertBtoHT();
-        scanEarlyStopAsearchB();
-        break;
-
-      case 20:   //A Compact, B HashTbl; CheckAB, scanAllAsearchB()
-      case 36:  //A HashTbl, B HashTbl; CheckAB, scanAllAsearchB()
-        Util.checkSeedHashes(seedHash_, a_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        Util.checkSeedHashes(seedHash_, b_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        thetaLong_ = min(a_.getThetaLong(), b_.getThetaLong());
-        empty_ = false;
-        lgArrLongsHT_ = ((UpdateSketch)b_).getLgArrLongs();
-        bHashTable_ = b_.getCache();
-        scanAllAsearchB();
-        break;
-
-      case 28:  //A Ordered, B HashTbl; CheckAB, scanEarlyStopAsearchB()
-        Util.checkSeedHashes(seedHash_, a_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        Util.checkSeedHashes(seedHash_, b_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        thetaLong_ = min(a_.getThetaLong(), b_.getThetaLong());
-        empty_ = false;
-        lgArrLongsHT_ = ((UpdateSketch)b_).getLgArrLongs();
-        bHashTable_ = b_.getCache();
-        scanEarlyStopAsearchB();
-        break;
-
-      //default: //This cannot happen and cannot be tested
-    }
-  }
-
-  private void convertBtoHT() {
-    final int curCountB = b_.getRetainedEntries(true);
-    lgArrLongsHT_ = computeMinLgArrLongsFromCount(curCountB);
-    bHashTable_ = new long[1 << lgArrLongsHT_];
-    hashArrayInsert(b_.getCache(), bHashTable_, lgArrLongsHT_, thetaLong_);
-  }
-
-  //Sketch A is either unordered compact or hash table
-  private void scanAllAsearchB() {
-    final long[] scanAArr = a_.getCache();
-    final int arrLongsIn = scanAArr.length;
-    cache_ = new long[arrLongsIn];
-    for (int i = 0; i < arrLongsIn; i++ ) {
-      final long hashIn = scanAArr[i];
-      if ((hashIn <= 0L) || (hashIn >= thetaLong_)) { continue; }
-      final int foundIdx = hashSearch(bHashTable_, lgArrLongsHT_, hashIn);
-      if (foundIdx > -1) { continue; }
-      cache_[curCount_++] = hashIn;
-    }
-  }
-
-  //Sketch A is ordered compact, which enables early stop
-  private void scanEarlyStopAsearchB() {
-    final long[] scanAArr = a_.getCache();
-    final int arrLongsIn = scanAArr.length;
-    cache_ = new long[arrLongsIn]; //maybe 2x what is needed, but getRetainedEntries can be slow.
-    for (int i = 0; i < arrLongsIn; i++ ) {
-      final long hashIn = scanAArr[i];
-      if (hashIn <= 0L) { continue; }
-      if (hashIn >= thetaLong_) {
-        break; //early stop assumes that hashes in input sketch are ordered!
-      }
-      final int foundIdx = hashSearch(bHashTable_, lgArrLongsHT_, hashIn);
-      if (foundIdx > -1) { continue; }
-      cache_[curCount_++] = hashIn;
-    }
-  }
-
-  private void reset() {
-    a_ = null;
-    b_ = null;
-    thetaLong_ = Long.MAX_VALUE;
-    empty_ = true;
-    cache_ = null;
-    curCount_ = 0;
-    lgArrLongsHT_ = 5;
-    bHashTable_ = null;
-  }
-
-  @Override
-  long[] getCache() {
-    return null;
-  }
-
-  @Override
-  short getSeedHash() {
-    return seedHash_;
-  }
-
-  @Override
-  long getThetaLong() {
-    return thetaLong_;
-  }
-
-}
diff --git a/src/main/java/org/apache/datasketches/theta/HeapCompactOrderedSketch.java b/src/main/java/org/apache/datasketches/theta/HeapCompactOrderedSketch.java
index 6957843..f359381 100644
--- a/src/main/java/org/apache/datasketches/theta/HeapCompactOrderedSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/HeapCompactOrderedSketch.java
@@ -58,9 +58,8 @@ final class HeapCompactOrderedSketch extends HeapCompactSketch {
   static CompactSketch heapifyInstance(final Memory srcMem, final long seed) {
     final short memSeedHash = checkMemorySeedHash(srcMem, seed);
     final int preLongs = extractPreLongs(srcMem);
-    final boolean empty = PreambleUtil.isEmpty(srcMem); //checks for cap <= 8
+    final boolean empty = PreambleUtil.isEmptySketch(srcMem);
     long thetaLong = Long.MAX_VALUE;
-    //preLongs == 1 handled before this method, so preLongs > 1
     final int curCount = extractCurCount(srcMem);
     final long[] cache = new long[curCount];
     if (preLongs == 2) {
@@ -72,27 +71,11 @@ final class HeapCompactOrderedSketch extends HeapCompactSketch {
     return new HeapCompactOrderedSketch(cache, empty, memSeedHash, curCount, thetaLong);
   }
 
-  /**
-   * Constructs this sketch from correct, valid arguments.
-   * @param cache in compact, ordered form
-   * @param empty The correct <a href="{@docRoot}/resources/dictionary.html#empty">Empty</a>.
-   * @param seedHash The correct
-   * <a href="{@docRoot}/resources/dictionary.html#seedHash">Seed Hash</a>.
-   * @param curCount correct value
-   * @param thetaLong The correct
-   * <a href="{@docRoot}/resources/dictionary.html#thetaLong">thetaLong</a>.
-   * @return a CompactSketch
-   */
-  static CompactSketch compact(final long[] cache, final boolean empty,
-      final short seedHash, final int curCount, final long thetaLong) {
-    return new HeapCompactOrderedSketch(cache, empty, seedHash, curCount, thetaLong);
-  }
-
   //Sketch interface
 
   @Override
   public byte[] toByteArray() {
-    return toByteArray(true);
+    return toByteArray(true); //ordered
   }
 
   @Override
diff --git a/src/main/java/org/apache/datasketches/theta/HeapCompactUnorderedSketch.java b/src/main/java/org/apache/datasketches/theta/HeapCompactUnorderedSketch.java
index d6019a2..d1cd085 100644
--- a/src/main/java/org/apache/datasketches/theta/HeapCompactUnorderedSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/HeapCompactUnorderedSketch.java
@@ -27,7 +27,7 @@ import static org.apache.datasketches.theta.PreambleUtil.extractThetaLong;
 import org.apache.datasketches.memory.Memory;
 
 /**
- * An on-heap, compact, read-only sketch.
+ * An on-heap, compact, unordered, read-only sketch.
  *
  * @author Lee Rhodes
  */
@@ -57,13 +57,11 @@ final class HeapCompactUnorderedSketch extends HeapCompactSketch {
   //Note Empty and SingleItemSketches should be filtered out before we get here.
   static CompactSketch heapifyInstance(final Memory srcMem, final long seed) {
     final short memSeedHash = checkMemorySeedHash(srcMem, seed);
-    final int preLongs = extractPreLongs(srcMem); //must be > 1
-    final boolean empty = PreambleUtil.isEmpty(srcMem); //checks for cap <= 16
-    int curCount = 0;
+    final int preLongs = extractPreLongs(srcMem);
+    final boolean empty = PreambleUtil.isEmptySketch(srcMem);
     long thetaLong = Long.MAX_VALUE;
-    long[] cache = new long[0];
-    curCount = extractCurCount(srcMem);
-    cache = new long[curCount];
+    final int curCount = extractCurCount(srcMem);
+    final long[] cache = new long[curCount];
     if (preLongs == 2) {
       srcMem.getLongArray(16, cache, 0, curCount);
     } else { //preLongs == 3
@@ -73,31 +71,13 @@ final class HeapCompactUnorderedSketch extends HeapCompactSketch {
     return new HeapCompactUnorderedSketch(cache, empty, memSeedHash, curCount, thetaLong);
   }
 
-  /**
-   * Constructs this sketch from correct, valid arguments.
-   * @param cache in compact form
-   * @param empty The correct <a href="{@docRoot}/resources/dictionary.html#empty">Empty</a>.
-   * @param seedHash The correct
-   * <a href="{@docRoot}/resources/dictionary.html#seedHash">Seed Hash</a>.
-   * @param curCount correct value
-   * @param thetaLong The correct
-   * <a href="{@docRoot}/resources/dictionary.html#thetaLong">thetaLong</a>.
-   * @return a CompactSketch
-   */
-  static CompactSketch compact(final long[] cache, final boolean empty,
-      final short seedHash, final int curCount, final long thetaLong) {
-    return new HeapCompactUnorderedSketch(cache, empty, seedHash, curCount, thetaLong);
-  }
-
   //Sketch interface
 
   @Override
   public byte[] toByteArray() {
-    return toByteArray(false);
+    return toByteArray(false); //unordered
   }
 
-  //restricted methods
-
   @Override
   public boolean isOrdered() {
     return false;
diff --git a/src/main/java/org/apache/datasketches/theta/HeapQuickSelectSketch.java b/src/main/java/org/apache/datasketches/theta/HeapQuickSelectSketch.java
index cb40feb..c549018 100644
--- a/src/main/java/org/apache/datasketches/theta/HeapQuickSelectSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/HeapQuickSelectSketch.java
@@ -22,10 +22,11 @@ package org.apache.datasketches.theta;
 import static java.lang.Math.max;
 import static java.lang.Math.min;
 import static org.apache.datasketches.QuickSelect.selectExcludingZeros;
+import static org.apache.datasketches.Util.LONG_MAX_VALUE_AS_DOUBLE;
 import static org.apache.datasketches.Util.MIN_LG_ARR_LONGS;
 import static org.apache.datasketches.Util.REBUILD_THRESHOLD;
 import static org.apache.datasketches.Util.RESIZE_THRESHOLD;
-import static org.apache.datasketches.theta.PreambleUtil.MAX_THETA_LONG_AS_DOUBLE;
+import static org.apache.datasketches.Util.startingSubMultiple;
 import static org.apache.datasketches.theta.PreambleUtil.extractCurCount;
 import static org.apache.datasketches.theta.PreambleUtil.extractFamilyID;
 import static org.apache.datasketches.theta.PreambleUtil.extractLgArrLongs;
@@ -41,7 +42,6 @@ import static org.apache.datasketches.theta.UpdateReturnState.RejectedOverTheta;
 import org.apache.datasketches.Family;
 import org.apache.datasketches.HashOperations;
 import org.apache.datasketches.ResizeFactor;
-import org.apache.datasketches.Util;
 import org.apache.datasketches.memory.Memory;
 import org.apache.datasketches.memory.WritableMemory;
 
@@ -92,10 +92,10 @@ class HeapQuickSelectSketch extends HeapUpdateSketch {
       MY_FAMILY = Family.QUICKSELECT;
     }
 
-    lgArrLongs_ = Util.startingSubMultiple(lgNomLongs + 1, rf, MIN_LG_ARR_LONGS);
+    lgArrLongs_ = startingSubMultiple(lgNomLongs + 1, rf, MIN_LG_ARR_LONGS);
     hashTableThreshold_ = setHashTableThreshold(lgNomLongs, lgArrLongs_);
     curCount_ = 0;
-    thetaLong_ = (long)(p * MAX_THETA_LONG_AS_DOUBLE);
+    thetaLong_ = (long)(p * LONG_MAX_VALUE_AS_DOUBLE);
     empty_ = true; //other flags: bigEndian = readOnly = compact = ordered = false;
     cache_ = new long[1 << lgArrLongs_];
   }
@@ -123,7 +123,7 @@ class HeapQuickSelectSketch extends HeapUpdateSketch {
     final Family family = Family.idToFamily(familyID);
 
     if ((myRF == ResizeFactor.X1)
-            && (lgArrLongs != Util.startingSubMultiple(lgNomLongs + 1, myRF, MIN_LG_ARR_LONGS))) {
+            && (lgArrLongs != startingSubMultiple(lgNomLongs + 1, myRF, MIN_LG_ARR_LONGS))) {
       myRF = ResizeFactor.X2;
     }
 
@@ -133,7 +133,7 @@ class HeapQuickSelectSketch extends HeapUpdateSketch {
     hqss.hashTableThreshold_ = setHashTableThreshold(lgNomLongs, lgArrLongs);
     hqss.curCount_ = extractCurCount(srcMem);
     hqss.thetaLong_ = extractThetaLong(srcMem);
-    hqss.empty_ = PreambleUtil.isEmpty(srcMem);
+    hqss.empty_ = PreambleUtil.isEmptySketch(srcMem);
     hqss.cache_ = new long[1 << lgArrLongs];
     srcMem.getLongArray(preambleLongs << 3, hqss.cache_, 0, 1 << lgArrLongs); //read in as hash table
     return hqss;
@@ -189,7 +189,7 @@ class HeapQuickSelectSketch extends HeapUpdateSketch {
   @Override
   public void reset() {
     final ResizeFactor rf = getResizeFactor();
-    final int lgArrLongsSM = Util.startingSubMultiple(lgNomLongs_ + 1, rf, MIN_LG_ARR_LONGS);
+    final int lgArrLongsSM = startingSubMultiple(lgNomLongs_ + 1, rf, MIN_LG_ARR_LONGS);
     if (lgArrLongsSM == lgArrLongs_) {
       final int arrLongs = cache_.length;
       assert (1 << lgArrLongs_) == arrLongs;
@@ -202,7 +202,7 @@ class HeapQuickSelectSketch extends HeapUpdateSketch {
     hashTableThreshold_ = setHashTableThreshold(lgNomLongs_, lgArrLongs_);
     empty_ = true;
     curCount_ = 0;
-    thetaLong_ =  (long)(getP() * MAX_THETA_LONG_AS_DOUBLE);
+    thetaLong_ =  (long)(getP() * LONG_MAX_VALUE_AS_DOUBLE);
   }
 
   //restricted methods
diff --git a/src/main/java/org/apache/datasketches/theta/IntersectionImpl.java b/src/main/java/org/apache/datasketches/theta/IntersectionImpl.java
index 5e38750..d5478e4 100644
--- a/src/main/java/org/apache/datasketches/theta/IntersectionImpl.java
+++ b/src/main/java/org/apache/datasketches/theta/IntersectionImpl.java
@@ -260,7 +260,7 @@ final class IntersectionImpl extends IntersectionImplR {
         performIntersect(sketchIn);
         break;
       }
-      //default: not possible
+      default: assert false : "Should not happen";
     }
   }
 
diff --git a/src/main/java/org/apache/datasketches/theta/PairwiseSetOperations.java b/src/main/java/org/apache/datasketches/theta/PairwiseSetOperations.java
index c34a560..a73e7eb 100644
--- a/src/main/java/org/apache/datasketches/theta/PairwiseSetOperations.java
+++ b/src/main/java/org/apache/datasketches/theta/PairwiseSetOperations.java
@@ -34,24 +34,30 @@ import org.apache.datasketches.Util;
  * Heap-based or Direct.
  *
  * @author Lee Rhodes
+ * @deprecated  This class has been deprecated as equivalent functionality has been added to the
+ * SetOperation classes: Union, Intersection and AnotB.
  */
+@Deprecated
 public class PairwiseSetOperations {
 
   /**
    * This implements a stateless, pair-wise <i>Intersect</i> operation on sketches
    * that are either Heap-based or Direct.
-   * If both inputs are null an EmptyCompactSketch is returned.
+   * If either inputs are null or empty an EmptyCompactSketch is returned.
    *
    * @param skA The first Sketch argument.
    * @param skB The second Sketch argument.
    * @return the result as an ordered CompactSketch on the heap.
+   * @deprecated Use {@link Intersection#intersect(Sketch, Sketch)} instead, which has more
+   * complete seed handling.
    */
+  @Deprecated
   public static CompactSketch intersect(final Sketch skA, final Sketch skB) {
     if (((skA == null) || (skA instanceof EmptyCompactSketch))
-        && ((skB == null) || (skB instanceof EmptyCompactSketch))) {
+        || ((skB == null) || (skB instanceof EmptyCompactSketch))) {
       return EmptyCompactSketch.getInstance();
     }
-    final short seedHash = (skA == null) ? skB.getSeedHash() : skA.getSeedHash();
+    final short seedHash = skA.getSeedHash();
     final Intersection inter = new IntersectionImpl(seedHash);
     return inter.intersect(skA, skB, true, null);
   }
@@ -64,7 +70,10 @@ public class PairwiseSetOperations {
    * @param skA The first Sketch argument.
    * @param skB The second Sketch argument.
    * @return the result as an ordered CompactSketch on the heap.
+   * @deprecated Use {@link AnotB#aNotB(Sketch, Sketch)} instead, which has more
+   * complete seed handling.
    */
+  @Deprecated
   public static CompactSketch aNotB(final Sketch skA, final Sketch skB) {
     if (((skA == null) || (skA instanceof EmptyCompactSketch))
         && ((skB == null) || (skB instanceof EmptyCompactSketch))) {
@@ -72,13 +81,14 @@ public class PairwiseSetOperations {
     }
     final short seedHash = ((skA == null) || (skA instanceof EmptyCompactSketch))
         ? skB.getSeedHash() : skA.getSeedHash(); // lgtm [java/dereferenced-value-may-be-null]
-    final HeapAnotB anotb = new HeapAnotB(seedHash);
-    return anotb.aNotB(skA, skB, true, null); //handles null arguments just fine.
+    final AnotBimpl anotb = new AnotBimpl(seedHash);
+    return anotb.aNotB(skA, skB, true, null);
   }
 
   /**
    * This implements a stateless, pair-wise union operation on ordered,
    * CompactSketches that are either Heap-based or Direct.
+   * Having the input sketches be compact and ordered enables extremely fast union operation.
    * If both inputs are null an EmptyCompactSketch is returned.
    * If one is null the other is returned, which can be either Heap-based or Direct.
    * This is equivalent to union(skA, skB, k) where k is the default of 4096.
@@ -86,7 +96,10 @@ public class PairwiseSetOperations {
    * @param skA The first ordered, CompactSketch argument.
    * @param skB The second ordered, CompactSketch argument
    * @return the result as an ordered CompactSketch.
+   * @deprecated Please use {@link Union#union(Sketch, Sketch)} instead, which has more
+   * complete seed handling.
    */
+  @Deprecated
   public static CompactSketch union(final CompactSketch skA, final CompactSketch skB) {
     return union(skA, skB, Util.DEFAULT_NOMINAL_ENTRIES);
   }
@@ -104,7 +117,10 @@ public class PairwiseSetOperations {
    * @param skB The second ordered, CompactSketch argument
    * @param k The upper bound of the number of entries to be retained by the sketch
    * @return the result as an ordered CompactSketch.
+   * @deprecated Please use {@link Union#union(Sketch, Sketch)} instead, which has more
+   * complete seed handling.
    */
+  @Deprecated
   @SuppressWarnings("null")
   public static CompactSketch union(final CompactSketch skA, final CompactSketch skB, final int k) {
     //Handle all corner cases with null or empty arguments
diff --git a/src/main/java/org/apache/datasketches/theta/PreambleUtil.java b/src/main/java/org/apache/datasketches/theta/PreambleUtil.java
index 17cae62..a61e16c 100644
--- a/src/main/java/org/apache/datasketches/theta/PreambleUtil.java
+++ b/src/main/java/org/apache/datasketches/theta/PreambleUtil.java
@@ -29,6 +29,7 @@ import java.nio.ByteOrder;
 import org.apache.datasketches.Family;
 import org.apache.datasketches.ResizeFactor;
 import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.Util;
 import org.apache.datasketches.memory.Memory;
 import org.apache.datasketches.memory.WritableMemory;
 
@@ -190,8 +191,6 @@ final class PreambleUtil {
   static final boolean NATIVE_ORDER_IS_BIG_ENDIAN  =
       (ByteOrder.nativeOrder() == ByteOrder.BIG_ENDIAN);
 
-  static final double MAX_THETA_LONG_AS_DOUBLE = Long.MAX_VALUE;
-
   /**
    * Computes the number of bytes required for a non-full sized sketch in hash-table form.
    * This can be used to compute current storage size for heap sketches, or current off-heap memory
@@ -239,7 +238,8 @@ final class PreambleUtil {
 
     //Flags
     final int flags = extractFlags(mem);
-    final String flagsStr = (flags) + ", " + zeroPad(Integer.toBinaryString(flags), 8);
+    final String flagsStr = (flags) + ", 0x" + (Integer.toHexString(flags)) + ", "
+        + zeroPad(Integer.toBinaryString(flags), 8);
     final String nativeOrder = ByteOrder.nativeOrder().toString();
     final boolean bigEndian = (flags & BIG_ENDIAN_FLAG_MASK) > 0;
     final boolean readOnly = (flags & READ_ONLY_FLAG_MASK) > 0;
@@ -274,9 +274,9 @@ final class PreambleUtil {
     }
     //else the same as an empty sketch or singleItem
 
-    final double thetaDbl = thetaLong / MAX_THETA_LONG_AS_DOUBLE;
+    final double thetaDbl = thetaLong / Util.LONG_MAX_VALUE_AS_DOUBLE;
     final String thetaHex = zeroPad(Long.toHexString(thetaLong), 16);
-    final double thetaUDbl = thetaULong / MAX_THETA_LONG_AS_DOUBLE;
+    final double thetaUDbl = thetaULong / Util.LONG_MAX_VALUE_AS_DOUBLE;
     final String thetaUHex = zeroPad(Long.toHexString(thetaULong), 16);
 
     final StringBuilder sb = new StringBuilder();
@@ -462,14 +462,69 @@ final class PreambleUtil {
     wmem.putByte(FLAGS_BYTE, (byte) flags);
   }
 
-  static boolean isEmpty(final Memory mem) {
-    final boolean emptyFlag = (mem.getByte(FLAGS_BYTE) & EMPTY_FLAG_MASK) > 0;
-    final boolean emptyCap = mem.getCapacity() < 16L;
-    return emptyFlag || emptyCap;
+  static boolean isEmptyFlag(final Memory mem) {
+    return ((extractFlags(mem) & EMPTY_FLAG_MASK) > 0);
+  }
+
+  private static final int ALPHA_ID = Family.ALPHA.getID();
+  private static final int QUICKSELECT_ID = Family.QUICKSELECT.getID();
+  private static final int COMPACT_ID = Family.COMPACT.getID();
+  private static final int UNION_ID = Family.UNION.getID();
+
+  static boolean isEmptySketch(final Memory mem) {
+    //CHECK FamID
+    final int famId = extractFamilyID(mem);
+    if ( !((famId == ALPHA_ID) || (famId == QUICKSELECT_ID)
+        || (famId == COMPACT_ID) || (famId == UNION_ID))) {
+      throw new SketchesArgumentException("Not part of the Theta Sketch Family");
+    }
+    //CHECK PreLongs
+    final int preLongs = extractPreLongs(mem);
+    if ((preLongs < 1) || (preLongs > 4)) {
+      throw new SketchesArgumentException("Possible corruption: Illegal preLongs value: " + preLongs);
+    }
+    //CHECK SerVer
+    final int serVer = extractSerVer(mem);
+    if ((serVer < 1) || (serVer > 3)) {
+      throw new SketchesArgumentException("Possible corruption: Illegal serVer value: " + serVer);
+    }
+    if (serVer == 1) { //Prelongs is always 3, no empty flag
+      assert preLongs == 3;
+      return ((extractCurCount(mem) == 0)  && (extractThetaLong(mem) == Long.MAX_VALUE));
+    }
+    // Flags byte: SI, Ordered, Compact, Empty, ReadOnly, LittleEndian = 0XX1X0
+    // Flags mask = 100101 = 0x25; Flags compare = 000100 = 0x4
+    final int flags = extractFlags(mem);
+    final boolean emptyFlag = (flags & 0x25) == EMPTY_FLAG_MASK;
+    if (serVer == 2) {
+      if (preLongs == 1) { return true; }
+      final int curCount = extractCurCount(mem);
+      if (preLongs == 2) {
+        return emptyFlag || (curCount == 0);
+      }
+      final long thetaLong = extractThetaLong(mem);
+      if (preLongs == 3) {
+        return emptyFlag || ((curCount == 0) && (thetaLong == Long.MAX_VALUE));
+      }
+    }
+    if (serVer == 3) {
+      final boolean emptyCap = mem.getCapacity() < 16L;
+      if (preLongs == 1) { return emptyFlag || emptyCap; }
+      final int curCount = extractCurCount(mem);
+      if (preLongs == 2) {
+        return emptyFlag || (curCount == 0);
+      }
+      final long thetaLong = extractThetaLong(mem);
+      if (preLongs <= 4) {
+        return emptyFlag || ((curCount == 0) && (thetaLong == Long.MAX_VALUE));
+      }
+    }
+    assert false : "Should not get here";
+    return true;
   }
 
-  static boolean isSingleItem(final Memory mem) {
-    // Flags byte must be LittleEndian, ReadOnly, Not Empty, Compact, Ordered = 11010 = 0x1A.
+  static boolean isSingleItemSketch(final Memory mem) {
+    // Flags byte: SI, Ordered, Compact, NotEmpty, ReadOnly, LittleEndian = X11010 = 0x1A.
     // Flags mask will be 0x1F.
     // SingleItem flag may not be set due to a historical bug, so we can't depend on it for now.
     // However, if the above flags are correct, preLongs == 1, SerVer >= 3, FamilyID == 3,
diff --git a/src/main/java/org/apache/datasketches/theta/SetOperation.java b/src/main/java/org/apache/datasketches/theta/SetOperation.java
index a397ad0..b4fc4cf 100644
--- a/src/main/java/org/apache/datasketches/theta/SetOperation.java
+++ b/src/main/java/org/apache/datasketches/theta/SetOperation.java
@@ -203,6 +203,17 @@ public abstract class SetOperation {
   }
 
   /**
+   * Returns the maximum number of bytes for the returned CompactSketch, given the maximum
+   * value of nomEntries of the first sketch A of AnotB.
+   * @param maxNomEntries the given value
+   * @return the maximum number of bytes.
+   */
+  public static int getMaxAnotBResultBytes(final int maxNomEntries) {
+    return 24 + (15 * maxNomEntries);
+  }
+
+
+  /**
    * Gets the Family of this SetOperation
    * @return the Family of this SetOperation
    */
@@ -264,19 +275,17 @@ public abstract class SetOperation {
     }
     if (dstMem == null) {
       if (dstOrdered) {
-        return HeapCompactOrderedSketch.compact(compactCache, empty, seedHash, curCount,
-            thetaLong); //converts to SingleItem format if curCount == 1
+        return new HeapCompactOrderedSketch(compactCache, empty, seedHash, curCount, thetaLong);
       } else {
-        return HeapCompactUnorderedSketch.compact(compactCache, empty, seedHash, curCount,
-            thetaLong); //converts to SingleItem if curCount == 1
+        return new HeapCompactUnorderedSketch(compactCache, empty, seedHash, curCount, thetaLong);
       }
     } else {
       if (dstOrdered) {
         return DirectCompactOrderedSketch.compact(compactCache, empty, seedHash, curCount,
-            thetaLong, dstMem); //converts to SingleItem format if curCount == 1
+            thetaLong, dstMem);
       } else {
         return DirectCompactUnorderedSketch.compact(compactCache, empty, seedHash, curCount,
-            thetaLong, dstMem); //converts to SingleItem format if curCount == 1
+            thetaLong, dstMem);
       }
     }
   }
diff --git a/src/main/java/org/apache/datasketches/theta/SetOperationBuilder.java b/src/main/java/org/apache/datasketches/theta/SetOperationBuilder.java
index c4f074f..822d88f 100644
--- a/src/main/java/org/apache/datasketches/theta/SetOperationBuilder.java
+++ b/src/main/java/org/apache/datasketches/theta/SetOperationBuilder.java
@@ -207,11 +207,11 @@ public class SetOperationBuilder {
       }
       case A_NOT_B: {
         if (dstMem == null) {
-          setOp = new HeapAnotB(bSeed);
+          setOp = new AnotBimpl(bSeed);
         }
         else {
           throw new SketchesArgumentException(
-            "AnotB is a stateless operation and cannot be persisted.");
+            "AnotB is only on heap and cannot be persisted.");
         }
         break;
       }
diff --git a/src/main/java/org/apache/datasketches/theta/SingleItemSketch.java b/src/main/java/org/apache/datasketches/theta/SingleItemSketch.java
index d82a5c2..28702ea 100644
--- a/src/main/java/org/apache/datasketches/theta/SingleItemSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/SingleItemSketch.java
@@ -24,12 +24,10 @@ import static org.apache.datasketches.ByteArrayUtil.putLongLE;
 import static org.apache.datasketches.Util.DEFAULT_UPDATE_SEED;
 import static org.apache.datasketches.Util.computeSeedHash;
 import static org.apache.datasketches.hash.MurmurHash3.hash;
-import static org.apache.datasketches.theta.PreambleUtil.MAX_THETA_LONG_AS_DOUBLE;
 import static org.apache.datasketches.theta.PreambleUtil.checkMemorySeedHash;
-import static org.apache.datasketches.theta.PreambleUtil.isSingleItem;
+import static org.apache.datasketches.theta.PreambleUtil.isSingleItemSketch;
 
 import org.apache.datasketches.SketchesArgumentException;
-import org.apache.datasketches.Util;
 import org.apache.datasketches.memory.Memory;
 
 /**
@@ -41,7 +39,7 @@ public final class SingleItemSketch extends CompactSketch {
   private static final long DEFAULT_SEED_HASH = computeSeedHash(DEFAULT_UPDATE_SEED) & 0xFFFFL;
 
   // For backward compatibility, a candidate pre0_ long must have:
-  // Flags byte 5 must be Ordered, Compact, NOT Empty, Read Only, LittleEndian = 11010 = 0x1A.
+  // Flags (byte 5): Ordered, Compact, NOT Empty, Read Only, LittleEndian = 11010 = 0x1A.
   // Flags mask will be 0x1F.
   // SingleItem flag may not be set due to a historical bug, so we can't depend on it for now.
   // However, if the above flags are correct, preLongs == 1, SerVer >= 3, FamilyID == 3,
@@ -80,7 +78,7 @@ public final class SingleItemSketch extends CompactSketch {
    * @return a SingleItemSketch
    */
   public static SingleItemSketch heapify(final Memory srcMem) {
-    return heapify(srcMem, Util.DEFAULT_UPDATE_SEED);
+    return heapify(srcMem, DEFAULT_UPDATE_SEED);
   }
 
   /**
@@ -92,7 +90,7 @@ public final class SingleItemSketch extends CompactSketch {
    */
   public static SingleItemSketch heapify(final Memory srcMem, final long seed) {
     final short seedHashMem = checkMemorySeedHash(srcMem, seed);
-    if (isSingleItem(srcMem)) {
+    if (isSingleItemSketch(srcMem)) {
       return new SingleItemSketch(srcMem.getLong(8), seedHashMem);
     }
     throw new SketchesArgumentException("Input Memory Preamble is not a SingleItemSketch.");
@@ -303,9 +301,9 @@ public final class SingleItemSketch extends CompactSketch {
 
   //Sketch
 
-  @Override
-  public int getCountLessThanTheta(final double theta) {
-    return (hash_ < (theta * MAX_THETA_LONG_AS_DOUBLE)) ? 1 : 0;
+  @Override //much faster
+  public int getCountLessThanThetaLong(final long thetaLong) {
+    return (hash_ < thetaLong) ? 1 : 0;
   }
 
   @Override
diff --git a/src/main/java/org/apache/datasketches/theta/Sketch.java b/src/main/java/org/apache/datasketches/theta/Sketch.java
index 8acfdaf..d61a312 100644
--- a/src/main/java/org/apache/datasketches/theta/Sketch.java
+++ b/src/main/java/org/apache/datasketches/theta/Sketch.java
@@ -22,24 +22,23 @@ package org.apache.datasketches.theta;
 import static org.apache.datasketches.Family.idToFamily;
 import static org.apache.datasketches.HashOperations.count;
 import static org.apache.datasketches.Util.DEFAULT_UPDATE_SEED;
+import static org.apache.datasketches.Util.LONG_MAX_VALUE_AS_DOUBLE;
 import static org.apache.datasketches.Util.LS;
 import static org.apache.datasketches.Util.ceilingPowerOf2;
 import static org.apache.datasketches.Util.zeroPad;
 import static org.apache.datasketches.theta.PreambleUtil.COMPACT_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.FAMILY_BYTE;
 import static org.apache.datasketches.theta.PreambleUtil.FLAGS_BYTE;
-import static org.apache.datasketches.theta.PreambleUtil.MAX_THETA_LONG_AS_DOUBLE;
 import static org.apache.datasketches.theta.PreambleUtil.ORDERED_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.PREAMBLE_LONGS_BYTE;
 import static org.apache.datasketches.theta.PreambleUtil.READ_ONLY_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.SER_VER_BYTE;
-import static org.apache.datasketches.theta.PreambleUtil.isSingleItem;
+import static org.apache.datasketches.theta.PreambleUtil.isSingleItemSketch;
 
 import org.apache.datasketches.BinomialBoundsN;
 import org.apache.datasketches.Family;
 import org.apache.datasketches.SketchesArgumentException;
 import org.apache.datasketches.SketchesStateException;
-import org.apache.datasketches.Util;
 import org.apache.datasketches.memory.Memory;
 import org.apache.datasketches.memory.WritableMemory;
 
@@ -109,11 +108,20 @@ public abstract class Sketch {
   }
 
   /**
-   * Wrap takes the sketch image in Memory and refers to it directly. There is no data copying onto
-   * the java heap.  Only "Direct" Serialization Version 3 (i.e, OpenSource) sketches that have
-   * been explicitly stored as direct objects can be wrapped.
-   * An attempt to "wrap" earlier version sketches will result in a "heapified", normal
-   * Java Heap version of the sketch where all data will be copied to the heap.
+   * Wrap takes the sketch image in Memory and refers to it directly with just a reference.
+   * There is no data copying onto the java heap.  Only "Direct" Serialization Version 3
+   * (i.e, OpenSource) sketches that have been explicitly stored as direct objects can be wrapped.
+   *
+   * <p>The wrap operation enables fast read-only merging and access to all the public read-only API.</p>
+   *
+   * <p>Note: wrapping earlier serial version sketches will result in a on-heap form of the
+   * sketch where all data will be copied to the heap. These early versions were never designed to
+   * "wrap".</p>
+   *
+   * <p>Wrapping any subclass of this class that is empty or contains only a single item will
+   * result in on-heap equivalent forms of empty and single item sketch respectively.
+   * This is actually faster and consumes less overall memory.</p>
+   *
    * @param srcMem an image of a Sketch where the image seed hash matches the given seed hash.
    * <a href="{@docRoot}/resources/dictionary.html#mem">See Memory</a>
    * @param seed <a href="{@docRoot}/resources/dictionary.html#seed">See Update Hash Seed</a>.
@@ -136,10 +144,10 @@ public abstract class Sketch {
       }
       case COMPACT: { //serVer 1, 2, or 3, preLongs = 1, 2, or 3
         if (serVer == 3) {
-          if (PreambleUtil.isEmpty(srcMem)) { //empty flag OR cap < 16 bytes
+          if (PreambleUtil.isEmptySketch(srcMem)) {
             return EmptyCompactSketch.getInstance(srcMem);
           }
-          if (isSingleItem(srcMem)) { //SINGLEITEM?
+          if (isSingleItemSketch(srcMem)) { //SINGLEITEM?
             return SingleItemSketch.heapify(srcMem, seed);
           }
           //not empty & not singleItem
@@ -175,7 +183,7 @@ public abstract class Sketch {
   //Sketch interface
 
   /**
-   * Converts this sketch to an ordered CompactSketch on the Java heap.
+   * Converts this sketch as an ordered CompactSketch on the Java heap.
    *
    * <p>If this sketch is already in compact form this operation returns <i>this</i>.
    *
@@ -210,9 +218,20 @@ public abstract class Sketch {
    * Gets the number of hash values less than the given theta.
    * @param theta the given theta as a double between zero and one.
    * @return the number of hash values less than the given theta.
+   * @deprecated Use {@link #getCountLessThanThetaLong(long)}. It is more accurate.
    */
+  @Deprecated
   public int getCountLessThanTheta(final double theta) {
-    final long thetaLong = (long) (MAX_THETA_LONG_AS_DOUBLE * theta);
+    final long thetaLong = (long) (LONG_MAX_VALUE_AS_DOUBLE * theta);
+    return count(getCache(), thetaLong);
+  }
+
+  /**
+   * Gets the number of hash values less than the given theta expressed as a long.
+   * @param thetaLong the given theta as a long between zero and <i>Long.MAX_VALUE</i>.
+   * @return the number of hash values less than the given thetaLong.
+   */
+  public int getCountLessThanThetaLong(final long thetaLong) {
     return count(getCache(), thetaLong);
   }
 
@@ -317,7 +336,7 @@ public abstract class Sketch {
    * @return the value of theta as a double
    */
   public double getTheta() {
-    return getThetaLong() / MAX_THETA_LONG_AS_DOUBLE;
+    return getThetaLong() / LONG_MAX_VALUE_AS_DOUBLE;
   }
 
   /**
@@ -469,7 +488,7 @@ public abstract class Sketch {
     }
 
     if (sketchSummary) {
-      final double thetaDbl = thetaLong / MAX_THETA_LONG_AS_DOUBLE;
+      final double thetaDbl = thetaLong / LONG_MAX_VALUE_AS_DOUBLE;
       final String thetaHex = zeroPad(Long.toHexString(thetaLong), 16);
       final String thisSimpleName = this.getClass().getSimpleName();
       final int seedHash = Short.toUnsignedInt(getSeedHash());
@@ -524,7 +543,8 @@ public abstract class Sketch {
   //Restricted methods
 
   /**
-   * Gets the internal cache array.
+   * Gets the internal cache array. For on-heap sketches this will return a reference to the actual
+   * cache array. For off-heap sketches this returns a copy.
    * @return the internal cache array.
    */
   abstract long[] getCache();
@@ -643,18 +663,18 @@ public abstract class Sketch {
 
 
   static final double estimate(final long thetaLong, final int curCount) {
-    return curCount * (MAX_THETA_LONG_AS_DOUBLE / thetaLong);
+    return curCount * (LONG_MAX_VALUE_AS_DOUBLE / thetaLong);
   }
 
   static final double lowerBound(final int curCount, final long thetaLong, final int numStdDev,
       final boolean empty) {
-    final double theta = thetaLong / MAX_THETA_LONG_AS_DOUBLE;
+    final double theta = thetaLong / LONG_MAX_VALUE_AS_DOUBLE;
     return BinomialBoundsN.getLowerBound(curCount, theta, numStdDev, empty);
   }
 
   static final double upperBound(final int curCount, final long thetaLong, final int numStdDev,
       final boolean empty) {
-    final double theta = thetaLong / MAX_THETA_LONG_AS_DOUBLE;
+    final double theta = thetaLong / LONG_MAX_VALUE_AS_DOUBLE;
     return BinomialBoundsN.getUpperBound(curCount, theta, numStdDev, empty);
   }
 
@@ -666,7 +686,6 @@ public abstract class Sketch {
    * Instantiates a Heap Sketch from Memory. SerVer 1 & 2 already handled.
    * @param srcMem <a href="{@docRoot}/resources/dictionary.html#mem">See Memory</a>
    * @param seed <a href="{@docRoot}/resources/dictionary.html#seed">See Update Hash Seed</a>.
-   * The seed required to instantiate a non-compact sketch.
    * @return a Sketch
    */
   private static final Sketch heapifyFromMemory(final Memory srcMem, final long seed) {
@@ -694,6 +713,7 @@ public abstract class Sketch {
         return HeapQuickSelectSketch.heapifyInstance(srcMem, seed);
       }
       case COMPACT: {
+
         if (!compactFlag) {
           throw new SketchesArgumentException(
               "Corrupted: COMPACT family sketch image must have compact flag set");
@@ -703,13 +723,13 @@ public abstract class Sketch {
           throw new SketchesArgumentException(
               "Corrupted: COMPACT family sketch image must have Read-Only flag set");
         }
-        if (PreambleUtil.isEmpty(srcMem)) { //emptyFlag OR capacity < 16 bytes.
+        if (PreambleUtil.isEmptySketch(srcMem)) { //emptyFlag OR capacity < 16 bytes.
           return EmptyCompactSketch.getInstance(srcMem);
         }
         if (preLongs == 1) {
-          if (isSingleItem(srcMem)) { //SINGLE ITEM
+          if (isSingleItemSketch(srcMem)) { //SINGLE ITEM
             return SingleItemSketch.heapify(srcMem, seed);
-          } else { //EMPTY Note very old sketches (<2014) have no empty flag.
+          } else { //EMPTY. Note very old sketches ( before 2014) have no empty flag.
             return EmptyCompactSketch.getInstance(srcMem);
           }
         }
diff --git a/src/main/java/org/apache/datasketches/theta/Sketches.java b/src/main/java/org/apache/datasketches/theta/Sketches.java
index 749cfbf..ee73f11 100644
--- a/src/main/java/org/apache/datasketches/theta/Sketches.java
+++ b/src/main/java/org/apache/datasketches/theta/Sketches.java
@@ -284,6 +284,18 @@ public final class Sketches {
     return SetOperation.getMaxIntersectionBytes(nomEntries);
   }
 
+  /**
+   * Ref: {@link SetOperation#getMaxAnotBResultBytes(int)}.
+   * Returns the maximum number of bytes for the returned CompactSketch, given the maximum
+   * value of nomEntries of the first sketch A of AnotB.
+   * @param maxNomEntries the given value
+   * @return the maximum number of bytes.
+   */
+  public static int getMaxAnotBResultBytes(final int maxNomEntries) {
+    return SetOperation.getMaxAnotBResultBytes(maxNomEntries);
+  }
+
+
   //Get estimates and bounds from Memory
 
   /**
diff --git a/src/main/java/org/apache/datasketches/theta/Union.java b/src/main/java/org/apache/datasketches/theta/Union.java
index b8a86fd..1d74a2f 100644
--- a/src/main/java/org/apache/datasketches/theta/Union.java
+++ b/src/main/java/org/apache/datasketches/theta/Union.java
@@ -70,6 +70,36 @@ public abstract class Union extends SetOperation {
   public abstract byte[] toByteArray();
 
   /**
+   * This implements a stateless, pair-wise union operation. The returned sketch will be cutback to
+   * k if required, similar to the regular Union operation.
+   *
+   * <p>Nulls and empty sketches are ignored.</p>
+   *
+   * @param sketchA The first argument
+   * @param sketchB The second argument
+   * @return the result ordered CompactSketch on the heap.
+   */
+  public CompactSketch union(final Sketch sketchA, final Sketch sketchB) {
+    return union(sketchA, sketchB, true, null);
+  }
+
+  /**
+   * This implements a stateless, pair-wise union operation. The returned sketch will be cutback to
+   * k if required, similar to the regular Union operation.
+   *
+   * <p>Nulls and empty sketches are ignored.</p>
+   *
+   * @param sketchA The first argument
+   * @param sketchB The second argument
+   * @param dstOrdered If true, the returned CompactSketch will be ordered.
+   * @param dstMem If not null, the returned CompactSketch will be placed in this WritableMemory.
+   * @return the result CompactSketch.
+   */
+  public abstract CompactSketch union(Sketch sketchA, Sketch sketchB, boolean dstOrdered,
+      WritableMemory dstMem);
+
+
+  /**
    * Perform a Union operation with <i>this</i> union and the given on-heap sketch of the Theta Family.
    * This method is not valid for the older SetSketch, which was prior to Open Source (August, 2015).
    *
diff --git a/src/main/java/org/apache/datasketches/theta/UnionImpl.java b/src/main/java/org/apache/datasketches/theta/UnionImpl.java
index ad093bc..e4d84d8 100644
--- a/src/main/java/org/apache/datasketches/theta/UnionImpl.java
+++ b/src/main/java/org/apache/datasketches/theta/UnionImpl.java
@@ -38,7 +38,7 @@ import static org.apache.datasketches.theta.PreambleUtil.extractSerVer;
 import static org.apache.datasketches.theta.PreambleUtil.extractThetaLong;
 import static org.apache.datasketches.theta.PreambleUtil.extractUnionThetaLong;
 import static org.apache.datasketches.theta.PreambleUtil.insertUnionThetaLong;
-import static org.apache.datasketches.theta.PreambleUtil.isSingleItem;
+import static org.apache.datasketches.theta.PreambleUtil.isSingleItemSketch;
 
 import org.apache.datasketches.Family;
 import org.apache.datasketches.HashOperations;
@@ -136,7 +136,7 @@ final class UnionImpl extends Union {
     final UpdateSketch gadget = HeapQuickSelectSketch.heapifyInstance(srcMem, seed);
     final UnionImpl unionImpl = new UnionImpl(gadget, seed);
     unionImpl.unionThetaLong_ = extractUnionThetaLong(srcMem);
-    unionImpl.unionEmpty_ = PreambleUtil.isEmpty(srcMem);
+    unionImpl.unionEmpty_ = PreambleUtil.isEmptyFlag(srcMem);
     return unionImpl;
   }
 
@@ -153,7 +153,7 @@ final class UnionImpl extends Union {
     final UpdateSketch gadget = DirectQuickSelectSketchR.fastReadOnlyWrap(srcMem, seed);
     final UnionImpl unionImpl = new UnionImpl(gadget, seed);
     unionImpl.unionThetaLong_ = extractUnionThetaLong(srcMem);
-    unionImpl.unionEmpty_ = PreambleUtil.isEmpty(srcMem);
+    unionImpl.unionEmpty_ = PreambleUtil.isEmptyFlag(srcMem);
     return unionImpl;
   }
 
@@ -170,7 +170,7 @@ final class UnionImpl extends Union {
     final UpdateSketch gadget = DirectQuickSelectSketch.fastWritableWrap(srcMem, seed);
     final UnionImpl unionImpl = new UnionImpl(gadget, seed);
     unionImpl.unionThetaLong_ = extractUnionThetaLong(srcMem);
-    unionImpl.unionEmpty_ = PreambleUtil.isEmpty(srcMem);
+    unionImpl.unionEmpty_ = PreambleUtil.isEmptyFlag(srcMem);
     return unionImpl;
   }
 
@@ -187,7 +187,7 @@ final class UnionImpl extends Union {
     final UpdateSketch gadget = DirectQuickSelectSketchR.readOnlyWrap(srcMem, seed);
     final UnionImpl unionImpl = new UnionImpl(gadget, seed);
     unionImpl.unionThetaLong_ = extractUnionThetaLong(srcMem);
-    unionImpl.unionEmpty_ = PreambleUtil.isEmpty(srcMem);
+    unionImpl.unionEmpty_ = PreambleUtil.isEmptyFlag(srcMem);
     return unionImpl;
   }
 
@@ -204,11 +204,17 @@ final class UnionImpl extends Union {
     final UpdateSketch gadget = DirectQuickSelectSketch.writableWrap(srcMem, seed);
     final UnionImpl unionImpl = new UnionImpl(gadget, seed);
     unionImpl.unionThetaLong_ = extractUnionThetaLong(srcMem);
-    unionImpl.unionEmpty_ = PreambleUtil.isEmpty(srcMem);
+    unionImpl.unionEmpty_ = PreambleUtil.isEmptyFlag(srcMem);
     return unionImpl;
   }
 
   @Override
+  public boolean isSameResource(final Memory that) {
+    return (gadget_ instanceof DirectQuickSelectSketchR)
+        ? gadget_.getMemory().isSameResource(that) : false;
+  }
+
+  @Override
   public CompactSketch getResult() {
     return getResult(true, null);
   }
@@ -262,9 +268,11 @@ final class UnionImpl extends Union {
   }
 
   @Override
-  public boolean isSameResource(final Memory that) {
-    return (gadget_ instanceof DirectQuickSelectSketchR)
-        ? gadget_.getMemory().isSameResource(that) : false;
+  public CompactSketch union(final Sketch sketchA, final Sketch sketchB, final boolean dstOrdered,
+      final WritableMemory dstMem) {
+    update(sketchA);
+    update(sketchB);
+    return getResult(dstOrdered, dstMem);
   }
 
   @Override
@@ -367,7 +375,7 @@ final class UnionImpl extends Union {
     final int preLongs = extractPreLongs(skMem);
 
     if (preLongs == 1) {
-      if (isSingleItem(skMem)) {
+      if (isSingleItemSketch(skMem)) {
         final long hash = skMem.getLong(8);
         gadget_.hashUpdate(hash);
         return;
diff --git a/src/main/java/org/apache/datasketches/theta/UpdateSketch.java b/src/main/java/org/apache/datasketches/theta/UpdateSketch.java
index 3134adf..f67324b 100644
--- a/src/main/java/org/apache/datasketches/theta/UpdateSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/UpdateSketch.java
@@ -21,14 +21,16 @@ package org.apache.datasketches.theta;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
 import static org.apache.datasketches.Util.DEFAULT_UPDATE_SEED;
+import static org.apache.datasketches.Util.LONG_MAX_VALUE_AS_DOUBLE;
 import static org.apache.datasketches.Util.MIN_LG_NOM_LONGS;
+import static org.apache.datasketches.Util.checkSeedHashes;
+import static org.apache.datasketches.Util.computeSeedHash;
 import static org.apache.datasketches.hash.MurmurHash3.hash;
 import static org.apache.datasketches.theta.CompactSketch.compactCache;
 import static org.apache.datasketches.theta.CompactSketch.loadCompactMemory;
 import static org.apache.datasketches.theta.PreambleUtil.BIG_ENDIAN_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.COMPACT_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.FAMILY_BYTE;
-import static org.apache.datasketches.theta.PreambleUtil.MAX_THETA_LONG_AS_DOUBLE;
 import static org.apache.datasketches.theta.PreambleUtil.ORDERED_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.PREAMBLE_LONGS_BYTE;
 import static org.apache.datasketches.theta.PreambleUtil.READ_ONLY_FLAG_MASK;
@@ -46,7 +48,6 @@ import static org.apache.datasketches.theta.UpdateReturnState.RejectedNullOrEmpt
 import org.apache.datasketches.Family;
 import org.apache.datasketches.ResizeFactor;
 import org.apache.datasketches.SketchesArgumentException;
-import org.apache.datasketches.Util;
 import org.apache.datasketches.memory.Memory;
 import org.apache.datasketches.memory.WritableMemory;
 
@@ -137,10 +138,14 @@ public abstract class UpdateSketch extends Sketch {
 
   @Override
   public CompactSketch compact(final boolean dstOrdered, final WritableMemory dstMem) {
-    final int curCount = this.getRetainedEntries(true);
-    final boolean empty = isEmpty();
+    return compact(this, dstOrdered, dstMem);
+  }
+
+  static CompactSketch compact(final UpdateSketch usk, final boolean dstOrdered,
+      final WritableMemory dstMem) {
+    final int curCount = usk.getRetainedEntries(true);
+    final boolean empty = usk.isEmpty();
     checkIllegalCurCountAndEmpty(empty, curCount);
-    final long thetaLong = correctThetaOnCompact(empty, curCount, getThetaLong());
 
     if (empty) {
       final EmptyCompactSketch sk = EmptyCompactSketch.getInstance();
@@ -150,20 +155,29 @@ public abstract class UpdateSketch extends Sketch {
       return sk;
     }
     //not empty
+    final long thetaLong = correctThetaOnCompact(empty, curCount, usk.getThetaLong());
+    final short seedHash = usk.getSeedHash();
+    final long[] cache = usk.getCache();
     if ((thetaLong == Long.MAX_VALUE) && (curCount == 1)) {
-      final long[] cache = getCache();
+
       final long[] cacheOut = compactCache(cache, curCount, thetaLong, dstOrdered);
       final long hash = cacheOut[0];
-      final SingleItemSketch sis = new SingleItemSketch(hash, getSeedHash());
+      final SingleItemSketch sis = new SingleItemSketch(hash, usk.getSeedHash());
       if (dstMem != null) {
         dstMem.putByteArray(0, sis.toByteArray(), 0, 16);
       }
-      return new SingleItemSketch(hash, getSeedHash());
+      return sis;
     }
     if (dstMem == null) {
-      return compactHeap(this, dstOrdered, curCount, thetaLong);
+      final long[] cacheOut = CompactSketch.compactCache(cache, curCount, thetaLong, dstOrdered);
+      if (dstOrdered) {
+        return new HeapCompactOrderedSketch(cacheOut, false, seedHash, curCount, thetaLong);
+      } else {
+        return new HeapCompactUnorderedSketch(cacheOut, false, seedHash, curCount, thetaLong);
+      }
+      //return compactHeap(usk, dstOrdered, curCount, thetaLong);
     } else {
-      return compactDirect(this, dstMem, dstOrdered, curCount, thetaLong);
+      return compactDirect(usk, dstMem, dstOrdered, curCount, thetaLong);
     }
   }
 
@@ -198,19 +212,19 @@ public abstract class UpdateSketch extends Sketch {
    * @param thetaLong the value of theta.
    * @return a CompactSketch.
    */
-  static CompactSketch compactDirect(final UpdateSketch sketch,
+  private static CompactSketch compactDirect(final UpdateSketch sketch,
       final WritableMemory dstMem, final boolean ordered, final int curCount, final long thetaLong) {
     final int preLongs = computeCompactPreLongs(thetaLong, false, curCount);
     final short seedHash = sketch.getSeedHash();
     final long[] cache = sketch.getCache();
-    final long[] compactCache = CompactSketch.compactCache(cache, curCount, thetaLong, ordered);
+    final long[] cacheOut = CompactSketch.compactCache(cache, curCount, thetaLong, ordered);
     if (ordered) {
       final byte flags = (byte)(READ_ONLY_FLAG_MASK | COMPACT_FLAG_MASK | ORDERED_FLAG_MASK);
-      loadCompactMemory(compactCache, seedHash, curCount, thetaLong, dstMem, flags, preLongs);
+      loadCompactMemory(cacheOut, seedHash, curCount, thetaLong, dstMem, flags, preLongs);
       return new DirectCompactOrderedSketch(dstMem);
     } else {
       final byte flags = (byte)(READ_ONLY_FLAG_MASK | COMPACT_FLAG_MASK);
-      loadCompactMemory(compactCache, seedHash, curCount, thetaLong, dstMem, flags, preLongs);
+      loadCompactMemory(cacheOut, seedHash, curCount, thetaLong, dstMem, flags, preLongs);
       return new DirectCompactUnorderedSketch(dstMem);
     }
   }
@@ -469,7 +483,7 @@ public abstract class UpdateSketch extends Sketch {
 
     //Check seed hashes
     final short seedHash = checkMemorySeedHash(srcMem, seed);              //byte 6,7
-    Util.checkSeedHashes(seedHash, Util.computeSeedHash(seed));
+    checkSeedHashes(seedHash, computeSeedHash(seed));
 
     //Check mem capacity, lgArrLongs
     final long curCapBytes = srcMem.getCapacity();
@@ -482,7 +496,7 @@ public abstract class UpdateSketch extends Sketch {
     //check Theta, p
     final float p = extractP(srcMem);                                   //bytes 12-15
     final long thetaLong = extractThetaLong(srcMem);                    //bytes 16-23
-    final double theta = thetaLong / MAX_THETA_LONG_AS_DOUBLE;
+    final double theta = thetaLong / LONG_MAX_VALUE_AS_DOUBLE;
     if ((lgArrLongs <= lgNomLongs) && (theta < p) ) {
       throw new SketchesArgumentException(
         "Possible corruption: Theta cannot be < p and lgArrLongs <= lgNomLongs. "
diff --git a/src/main/java/org/apache/datasketches/tuple/AnotB.java b/src/main/java/org/apache/datasketches/tuple/AnotB.java
index cb8cd9c..ff5b37b 100644
--- a/src/main/java/org/apache/datasketches/tuple/AnotB.java
+++ b/src/main/java/org/apache/datasketches/tuple/AnotB.java
@@ -19,15 +19,14 @@
 
 package org.apache.datasketches.tuple;
 
-import static org.apache.datasketches.Util.MIN_LG_NOM_LONGS;
+import static org.apache.datasketches.HashOperations.convertToHashTable;
+import static org.apache.datasketches.HashOperations.hashSearch;
 import static org.apache.datasketches.Util.REBUILD_THRESHOLD;
-import static org.apache.datasketches.Util.ceilingPowerOf2;
 import static org.apache.datasketches.Util.simpleIntLog2;
 
 import java.lang.reflect.Array;
 import java.util.Arrays;
 
-import org.apache.datasketches.HashOperations;
 import org.apache.datasketches.SketchesArgumentException;
 import org.apache.datasketches.theta.HashIterator;
 
@@ -40,7 +39,7 @@ public final class AnotB<S extends Summary> {
   private long thetaLong_ = Long.MAX_VALUE;
   private long[] hashArr_ = null;   //always in compact form, not necessarily sorted
   private S[] summaryArr_ = null; //always in compact form, not necessarily sorted
-  private int count_ = 0;
+  private int curCount_ = 0;
 
   /**
    * Sets the given Tuple sketch as the first argument <i>A</i>. This overwrites the internal state of
@@ -65,13 +64,14 @@ public final class AnotB<S extends Summary> {
     final CompactSketch<S> cskA = (skA instanceof CompactSketch)
         ? (CompactSketch<S>)skA
         : ((QuickSelectSketch<S>)skA).compact();
-    hashArr_ = cskA.getHashArr();
-    summaryArr_ = cskA.getSummaryArr();
-    count_ = cskA.getRetainedEntries();
+    hashArr_ = cskA.getHashArr().clone();
+    summaryArr_ = cskA.getSummaryArr().clone(); //shallow
+    curCount_ = cskA.getRetainedEntries();
   }
 
   /**
    * Performs an <i>AND NOT</i> operation with the existing internal state of this AnotB operator.
+   * Use {@link #getResult(boolean)} to obtain the result.
    *
    * <p>An input argument of null or empty is ignored.</p>
    *
@@ -91,26 +91,27 @@ public final class AnotB<S extends Summary> {
     final long[] hashTableB;
     if (skB instanceof CompactSketch) {
       cskB = (CompactSketch<S>) skB;
-      hashTableB = convertToHashTable(cskB.getHashArr(), countB, thetaLong_);
+      hashTableB =
+          convertToHashTable(cskB.getHashArr(), countB, thetaLong_, REBUILD_THRESHOLD);
     } else {
       qskB = (QuickSelectSketch<S>) skB;
       hashTableB = (thetaLong_ < thetaLongB)
-          ? convertToHashTable(qskB.getHashTable(), countB, thetaLong_)
+          ? convertToHashTable(qskB.getHashTable(), countB, thetaLong_, REBUILD_THRESHOLD)
           : qskB.getHashTable();
     }
 
     //build temporary arrays of skA
-    final long[] tmpHashArrA = new long[count_];
+    final long[] tmpHashArrA = new long[curCount_];
     final Class<S> summaryType = (Class<S>) summaryArr_.getClass().getComponentType();
-    final S[] tmpSummaryArrA = (S[]) Array.newInstance(summaryType, count_);
+    final S[] tmpSummaryArrA = (S[]) Array.newInstance(summaryType, curCount_);
 
     //search for non matches and build temp arrays
     int nonMatches = 0;
-    for (int i = 0; i < count_; i++) {
+    for (int i = 0; i < curCount_; i++) {
       final long hash = hashArr_[i];
       if ((hash != 0) && (hash < thetaLong_)) { //skips hashes of A >= theta
         final int index =
-            HashOperations.hashSearch(hashTableB, simpleIntLog2(hashTableB.length), hash);
+            hashSearch(hashTableB, simpleIntLog2(hashTableB.length), hash);
         if (index == -1) {
           tmpHashArrA[nonMatches] = hash;
           tmpSummaryArrA[nonMatches] = summaryArr_[i];
@@ -120,11 +121,12 @@ public final class AnotB<S extends Summary> {
     }
     hashArr_ = Arrays.copyOfRange(tmpHashArrA, 0, nonMatches);
     summaryArr_ = Arrays.copyOfRange(tmpSummaryArrA, 0, nonMatches);
-    count_ = nonMatches;
+    curCount_ = nonMatches;
   }
 
   /**
    * Performs an <i>AND NOT</i> operation with the existing internal state of this AnotB operator.
+   * Use {@link #getResult(boolean)} to obtain the result.
    *
    * <p>An input argument of null or empty is ignored.</p>
    *
@@ -139,20 +141,21 @@ public final class AnotB<S extends Summary> {
     //Build hashtable and removes hashes of skB >= theta
     final int countB = skB.getRetainedEntries();
     final long[] hashTableB =
-        convertToHashTable(extractThetaHashArray(skB, countB), countB, thetaLong_);
+        convertToHashTable(extractThetaHashArray(skB, countB), countB,
+            thetaLong_, REBUILD_THRESHOLD);
 
     //build temporary arrays of skA
-    final long[] tmpHashArrA = new long[count_];
+    final long[] tmpHashArrA = new long[curCount_];
     final Class<S> summaryType = (Class<S>) summaryArr_.getClass().getComponentType();
-    final S[] tmpSummaryArrA = (S[]) Array.newInstance(summaryType, count_);
+    final S[] tmpSummaryArrA = (S[]) Array.newInstance(summaryType, curCount_);
 
     //search for non matches and build temp arrays
+    final int lgHTBLen = simpleIntLog2(hashTableB.length);
     int nonMatches = 0;
-    for (int i = 0; i < count_; i++) {
+    for (int i = 0; i < curCount_; i++) {
       final long hash = hashArr_[i];
       if ((hash > 0) && (hash < thetaLong_)) { //skips hashes of A >= theta
-        final int index =
-            HashOperations.hashSearch(hashTableB, simpleIntLog2(hashTableB.length), hash);
+        final int index = hashSearch(hashTableB, lgHTBLen, hash);
         if (index == -1) {
           tmpHashArrA[nonMatches] = hash;
           tmpSummaryArrA[nonMatches] = summaryArr_[i];
@@ -162,7 +165,23 @@ public final class AnotB<S extends Summary> {
     }
     hashArr_ = Arrays.copyOfRange(tmpHashArrA, 0, nonMatches);
     summaryArr_ = Arrays.copyOfRange(tmpSummaryArrA, 0, nonMatches);
-    count_ = nonMatches;
+    curCount_ = nonMatches;
+  }
+
+  /**
+   * Gets the result of this stateful operation.
+   * @param reset If true, clears this operator to the empty state after result is returned.
+   * @return the result of this operation as a CompactSketch.
+   */
+  public CompactSketch<S> getResult(final boolean reset) {
+    if (curCount_ == 0) {
+      return new CompactSketch<>(null, null, thetaLong_, empty_);
+    }
+    final CompactSketch<S> result =
+        new CompactSketch<>(Arrays.copyOfRange(hashArr_, 0, curCount_),
+            Arrays.copyOfRange(summaryArr_, 0, curCount_), thetaLong_, empty_);
+    if (reset) { reset(); }
+    return result;
   }
 
   /**
@@ -185,42 +204,32 @@ public final class AnotB<S extends Summary> {
     if ((skA == null) || (skB == null)) {
       throw new SketchesArgumentException("Neither argument may be null");
     }
-    if (skA.isEmpty()) {
-      return new CompactSketch<>(null, null, Long.MAX_VALUE, true);
-    }
-    //skA is not empty
-    final boolean empty = false;
+    if (skA.isEmpty()) { return new CompactSketch<>(null, null, Long.MAX_VALUE, true); }
+    if (skB.isEmpty()) { return skA.compact(); }
+    //Both skA & skB are not empty
+    //Load skA into tmp registers
+    boolean empty = false;
     final long thetaLongA = skA.getThetaLong();
     final CompactSketch<S> cskA = (skA instanceof CompactSketch)
         ? (CompactSketch<S>)skA
         : ((QuickSelectSketch<S>)skA).compact();
-    final long[] hashArrA = cskA.getHashArr();
-    final S[] summaryArrA = cskA.getSummaryArr();
+    final long[] hashArrA = cskA.getHashArr().clone();
+    final S[] summaryArrA = cskA.getSummaryArr().clone();
     final int countA = cskA.getRetainedEntries();
 
-    if (skB.isEmpty()) {
-      return new CompactSketch<>(hashArrA, summaryArrA, thetaLongA, empty);
-    }
-    //skB is not empty
+    //Compare with skB
     final long thetaLongB = skB.getThetaLong();
     final long thetaLong = Math.min(thetaLongA, thetaLongB);
     final int countB = skB.getRetainedEntries();
-    //
-    CompactSketch<S> cskB = null;
-    QuickSelectSketch<S> qskB = null;
 
     //Build/rebuild hashtable and removes hashes of skB >= thetaLong
-    final long[] hashTableB;
-    if (skB instanceof CompactSketch) {
-      cskB = (CompactSketch<S>) skB;
-      hashTableB = convertToHashTable(cskB.getHashArr(), countB, thetaLong);
-    } else {
-      qskB = (QuickSelectSketch<S>) skB;
-      hashTableB = convertToHashTable(qskB.getHashTable(), countB, thetaLong);
-      cskB = qskB.compact();
-    }
+    final long[] hashes = (skB instanceof CompactSketch)
+        ? ((CompactSketch<S>) skB).getHashArr()
+        : ((QuickSelectSketch<S>) skB).getHashTable();
 
-    //build temporary arrays of skA
+    final long[] hashTableB = convertToHashTable(hashes, countB, thetaLong, REBUILD_THRESHOLD);
+
+    //build temporary hash array and summary array for values from skA
     final long[] tmpHashArrA = new long[countA];
     final Class<S> summaryType = (Class<S>) summaryArrA.getClass().getComponentType();
     final S[] tmpSummaryArrA = (S[]) Array.newInstance(summaryType, countA);
@@ -231,7 +240,7 @@ public final class AnotB<S extends Summary> {
       final long hash = hashArrA[i];
       if ((hash != 0) && (hash < thetaLong)) { //skips hashes of A >= theta
         final int index =
-            HashOperations.hashSearch(hashTableB, simpleIntLog2(hashTableB.length), hash);
+            hashSearch(hashTableB, simpleIntLog2(hashTableB.length), hash);
         if (index == -1) {
           tmpHashArrA[nonMatches] = hash;
           tmpSummaryArrA[nonMatches] = summaryArrA[i];
@@ -239,6 +248,8 @@ public final class AnotB<S extends Summary> {
         }
       }
     }
+    //final compaction
+    empty = ((nonMatches == 0) && (thetaLong == Long.MAX_VALUE));
     final long[] hashArrOut = Arrays.copyOfRange(tmpHashArrA, 0, nonMatches);
     final S[] summaryArrOut = Arrays.copyOfRange(tmpSummaryArrA, 0, nonMatches);
     final CompactSketch<S> result =
@@ -289,7 +300,7 @@ public final class AnotB<S extends Summary> {
 
     //Build/rebuild hashtable and removes hashes of skB >= thetaLong
     final long[] hashTableB = //the following convert works for all theta sketches
-        convertToHashTable(extractThetaHashArray(skB, countB), countB, thetaLong);
+        convertToHashTable(extractThetaHashArray(skB, countB), countB, thetaLong, REBUILD_THRESHOLD);
 
     //build temporary arrays of skA for matching
     final long[] tmpHashArrA = new long[countA];
@@ -302,7 +313,7 @@ public final class AnotB<S extends Summary> {
       final long hash = hashArrA[i];
       if ((hash != 0) && (hash < thetaLong)) { //skips hashes of A >= theta
         final int index =
-            HashOperations.hashSearch(hashTableB, simpleIntLog2(hashTableB.length), hash);
+            hashSearch(hashTableB, simpleIntLog2(hashTableB.length), hash);
         if (index == -1) {
           tmpHashArrA[nonMatches] = hash;
           tmpSummaryArrA[nonMatches] = summaryArrA[i];
@@ -318,22 +329,6 @@ public final class AnotB<S extends Summary> {
   }
 
   /**
-   * Gets the result of this operation.
-   * @param reset if true, clears this operator to the empty state after result is returned.
-   * @return the result of this operation as a CompactSketch.
-   */
-  public CompactSketch<S> getResult(final boolean reset) {
-    if (count_ == 0) {
-      return new CompactSketch<>(null, null, thetaLong_, empty_);
-    }
-    final CompactSketch<S> result =
-        new CompactSketch<>(Arrays.copyOfRange(hashArr_, 0, count_),
-            Arrays.copyOfRange(summaryArr_, 0, count_), thetaLong_, empty_);
-    if (reset) { reset(); }
-    return result;
-  }
-
-  /**
    * Resets this sketch back to the empty state.
    */
   public void reset() {
@@ -341,7 +336,7 @@ public final class AnotB<S extends Summary> {
     thetaLong_ = Long.MAX_VALUE;
     hashArr_ = null;
     summaryArr_ = null;
-    count_ = 0;
+    curCount_ = 0;
   }
 
   private static long[] extractThetaHashArray(
@@ -357,17 +352,6 @@ public final class AnotB<S extends Summary> {
     return hashArr;
   }
 
-  private static long[] convertToHashTable(final long[] hashArr, final int count, final long thetaLong) {
-    final int size = Math.max(
-      ceilingPowerOf2((int) Math.ceil(count / REBUILD_THRESHOLD)),
-      1 << MIN_LG_NOM_LONGS
-    );
-    final long[] hashTable = new long[size];
-    HashOperations.hashArrayInsert(
-        hashArr, hashTable, Integer.numberOfTrailingZeros(size), thetaLong);
-    return hashTable;
-  }
-
   //Deprecated methods
 
   /**
@@ -379,9 +363,7 @@ public final class AnotB<S extends Summary> {
    *
    * @param skA The incoming sketch for the first argument
    * @param skB The incoming sketch for the second argument
-   * @deprecated After release 2.0.0. Instead please use {@link #aNotB(Sketch, Sketch)}
-   * or a combination of {@link #setA(Sketch)} and
-   * {@link #notB(Sketch)} with {@link #getResult(boolean)}.
+   * @deprecated Instead please use {@link #aNotB(Sketch, Sketch)}.
    */
   @SuppressWarnings("unchecked")
   @Deprecated
@@ -406,12 +388,12 @@ public final class AnotB<S extends Summary> {
 
       if (skB instanceof CompactSketch) {
         csk = (CompactSketch<S>) skB;
-        hashTableB = convertToHashTable(csk.getHashArr(), count, thetaLong_);
+        hashTableB = convertToHashTable(csk.getHashArr(), count, thetaLong_, REBUILD_THRESHOLD);
         summaryType = (Class<S>) csk.getSummaryArr().getClass().getComponentType();
         lgHashTableSize = Integer.numberOfTrailingZeros(hashTableB.length);
       } else {
         qsk = (QuickSelectSketch<S>) skB;
-        hashTableB = convertToHashTable(qsk.getHashTable(), count, thetaLong_);
+        hashTableB = convertToHashTable(qsk.getHashTable(), count, thetaLong_, REBUILD_THRESHOLD);
         summaryType = (Class<S>) qsk.getSummaryTable().getClass().getComponentType();
         lgHashTableSize = Integer.numberOfTrailingZeros(hashTableB.length);
       }
@@ -424,11 +406,11 @@ public final class AnotB<S extends Summary> {
         final long hash = itrA.getHash();
         final S summary = itrA.getSummary();
         if ((hash <= 0) || (hash >= thetaLong_)) { continue; }
-        final int index = HashOperations.hashSearch(hashTableB, lgHashTableSize, hash);
+        final int index = hashSearch(hashTableB, lgHashTableSize, hash);
         if (index == -1) {
-          hashArr_[count_] = hash;
-          summaryArr_[count_] = summary;
-          count_++;
+          hashArr_[curCount_] = hash;
+          summaryArr_[curCount_] = summary;
+          curCount_++;
         }
       }
     }
@@ -438,18 +420,16 @@ public final class AnotB<S extends Summary> {
    * Gets the result of this operation. This clears the state of this operator after the result is
    * returned.
    * @return the result of this operation as a CompactSketch
-   * @deprecated Only used with deprecated {@link #update(Sketch,Sketch)}.
-   * Instead use {@link #aNotB(Sketch, Sketch)} or a combination of {@link #setA(Sketch)} and
-   * {@link #notB(Sketch)} with {@link #getResult(boolean)}.
+   * @deprecated Instead use {@link #getResult(boolean)}.
    */
   @Deprecated
   public CompactSketch<S> getResult() {
-    if (count_ == 0) {
+    if (curCount_ == 0) {
       return new CompactSketch<>(null, null, thetaLong_, empty_);
     }
     final CompactSketch<S> result =
-        new CompactSketch<>(Arrays.copyOfRange(hashArr_, 0, count_),
-            Arrays.copyOfRange(summaryArr_, 0, count_), thetaLong_, empty_);
+        new CompactSketch<>(Arrays.copyOfRange(hashArr_, 0, curCount_),
+            Arrays.copyOfRange(summaryArr_, 0, curCount_), thetaLong_, empty_);
     reset();
     return result;
   }
@@ -461,16 +441,17 @@ public final class AnotB<S extends Summary> {
    */
   private void loadCompactedArrays(final Sketch<S> sketch) {
     final CompactSketch<S> csk;
+    // assuming only two types: CompactSketch and QuickSelectSketch
     if (sketch instanceof CompactSketch) {
       csk = (CompactSketch<S>)sketch;
       hashArr_ = csk.getHashArr().clone();
       summaryArr_ = csk.getSummaryArr().clone();
-    } else { // assuming only two types: CompactSketch and QuickSelectSketch
+    } else {
       csk = ((QuickSelectSketch<S>)sketch).compact();
       hashArr_ = csk.getHashArr();
       summaryArr_ = csk.getSummaryArr();
     }
-    count_ = sketch.getRetainedEntries();
+    curCount_ = sketch.getRetainedEntries();
   }
 
 }
diff --git a/src/main/java/org/apache/datasketches/tuple/CompactSketch.java b/src/main/java/org/apache/datasketches/tuple/CompactSketch.java
index 837165f..07d350f 100644
--- a/src/main/java/org/apache/datasketches/tuple/CompactSketch.java
+++ b/src/main/java/org/apache/datasketches/tuple/CompactSketch.java
@@ -121,6 +121,11 @@ public class CompactSketch<S extends Summary> extends Sketch<S> {
     }
   }
 
+  @Override
+  public CompactSketch<S> compact() {
+    return this;
+  }
+
   long[] getHashArr() {
     return hashArr_;
   }
diff --git a/src/main/java/org/apache/datasketches/tuple/QuickSelectSketch.java b/src/main/java/org/apache/datasketches/tuple/QuickSelectSketch.java
index f0a67a6..423d05a 100644
--- a/src/main/java/org/apache/datasketches/tuple/QuickSelectSketch.java
+++ b/src/main/java/org/apache/datasketches/tuple/QuickSelectSketch.java
@@ -271,6 +271,7 @@ class QuickSelectSketch<S extends Summary> extends Sketch<S> {
    * Converts the current state of the sketch into a compact sketch
    * @return compact sketch
    */
+  @Override
   @SuppressWarnings("unchecked")
   public CompactSketch<S> compact() {
     if (getRetainedEntries() == 0) {
diff --git a/src/main/java/org/apache/datasketches/tuple/Sketch.java b/src/main/java/org/apache/datasketches/tuple/Sketch.java
index 36e01d9..02b7bf5 100644
--- a/src/main/java/org/apache/datasketches/tuple/Sketch.java
+++ b/src/main/java/org/apache/datasketches/tuple/Sketch.java
@@ -23,6 +23,7 @@ import static org.apache.datasketches.Util.LS;
 
 import org.apache.datasketches.BinomialBoundsN;
 
+
 /**
  * This is an equivalent to org.apache.datasketches.theta.Sketch with
  * addition of a user-defined Summary object associated with every unique entry
@@ -39,6 +40,15 @@ public abstract class Sketch<S extends Summary> {
   Sketch() {}
 
   /**
+   * Converts this sketch to a CompactSketch on the Java heap.
+   *
+   * <p>If this sketch is already in compact form this operation returns <i>this</i>.
+   *
+   * @return this sketch as a CompactSketch on the Java heap.
+   */
+  public abstract CompactSketch<S> compact();
+
+  /**
    * Estimates the cardinality of the set (number of unique values presented to the sketch)
    * @return best estimate of the number of unique values
    */
diff --git a/src/test/java/org/apache/datasketches/theta/HeapAnotBTest.java b/src/test/java/org/apache/datasketches/theta/AnotBimplTest.java
similarity index 99%
rename from src/test/java/org/apache/datasketches/theta/HeapAnotBTest.java
rename to src/test/java/org/apache/datasketches/theta/AnotBimplTest.java
index 2f1ae23..cc47dd6 100644
--- a/src/test/java/org/apache/datasketches/theta/HeapAnotBTest.java
+++ b/src/test/java/org/apache/datasketches/theta/AnotBimplTest.java
@@ -34,7 +34,7 @@ import org.testng.annotations.Test;
  * @author Lee Rhodes
  */
 @SuppressWarnings("javadoc")
-public class HeapAnotBTest {
+public class AnotBimplTest {
 
   @Test
   public void checkExactAnotB_AvalidNoOverlap() {
@@ -372,7 +372,7 @@ public class HeapAnotBTest {
   @Test
   public void checkGetFamily() {
     //cheap trick
-    HeapAnotB anotb = new HeapAnotB(Util.DEFAULT_UPDATE_SEED);
+    AnotBimpl anotb = new AnotBimpl(Util.DEFAULT_UPDATE_SEED);
     assertEquals(anotb.getFamily(), Family.A_NOT_B);
   }
 
diff --git a/src/test/java/org/apache/datasketches/theta/ForwardCompatibilityTest.java b/src/test/java/org/apache/datasketches/theta/ForwardCompatibilityTest.java
index a900a59..29dfd09 100644
--- a/src/test/java/org/apache/datasketches/theta/ForwardCompatibilityTest.java
+++ b/src/test/java/org/apache/datasketches/theta/ForwardCompatibilityTest.java
@@ -113,7 +113,7 @@ public class ForwardCompatibilityTest {
     WritableMemory srcMemW = WritableMemory.allocate(16);
     srcMem.copyTo(0, srcMemW, 0, 16);
     PreambleUtil.setEmpty(srcMemW); //Force
-    assertTrue(PreambleUtil.isEmpty(srcMemW));
+    assertTrue(PreambleUtil.isEmptySketch(srcMemW));
     srcMemW.putInt(8, 0); //corrupt curCount = 0
 
     Sketch sketch = Sketch.heapify(srcMemW);
@@ -136,7 +136,7 @@ public class ForwardCompatibilityTest {
     WritableMemory srcMemW = WritableMemory.allocate(24);
     srcMem.copyTo(0, srcMemW, 0, 24);
     PreambleUtil.setEmpty(srcMemW); //Force
-    assertTrue(PreambleUtil.isEmpty(srcMemW));
+    assertTrue(PreambleUtil.isEmptySketch(srcMemW));
     srcMemW.putInt(8, 0); //corrupt curCount = 0
     srcMemW.putLong(16, Long.MAX_VALUE); //corrupt to make it look empty
 
diff --git a/src/test/java/org/apache/datasketches/theta/PairwiseSetOperationsTest.java b/src/test/java/org/apache/datasketches/theta/PairwiseSetOperationsTest.java
index dfbb866..5c1b76f 100644
--- a/src/test/java/org/apache/datasketches/theta/PairwiseSetOperationsTest.java
+++ b/src/test/java/org/apache/datasketches/theta/PairwiseSetOperationsTest.java
@@ -171,7 +171,7 @@ public class PairwiseSetOperationsTest {
       CompactSketch csk2 = usk2.compact(true, null);
 
       Sketch rsk = PairwiseSetOperations.aNotB(csk1, csk2);
-      double result1 = rsk.getEstimate();
+      double result1 = rsk.getEstimate(); //null ptr
 
       aNotB.update(csk1, csk2);
       CompactSketch csk3 = aNotB.getResult(true, null);
diff --git a/src/test/java/org/apache/datasketches/theta/PreambleUtilTest.java b/src/test/java/org/apache/datasketches/theta/PreambleUtilTest.java
index b4f4fed..dc8a49f 100644
--- a/src/test/java/org/apache/datasketches/theta/PreambleUtilTest.java
+++ b/src/test/java/org/apache/datasketches/theta/PreambleUtilTest.java
@@ -46,7 +46,7 @@ import static org.apache.datasketches.theta.PreambleUtil.insertSeedHash;
 import static org.apache.datasketches.theta.PreambleUtil.insertSerVer;
 import static org.apache.datasketches.theta.PreambleUtil.insertThetaLong;
 import static org.apache.datasketches.theta.PreambleUtil.insertUnionThetaLong;
-import static org.apache.datasketches.theta.PreambleUtil.isEmpty;
+import static org.apache.datasketches.theta.PreambleUtil.isEmptyFlag;
 import static org.apache.datasketches.theta.PreambleUtil.setEmpty;
 import static org.apache.datasketches.theta.SetOperation.getMaxUnionBytes;
 import static org.testng.Assert.assertEquals;
@@ -54,14 +54,13 @@ import static org.testng.Assert.assertFalse;
 import static org.testng.Assert.assertTrue;
 import static org.testng.Assert.fail;
 
-import org.testng.Assert;
-import org.testng.annotations.Test;
-
-import org.apache.datasketches.memory.Memory;
-import org.apache.datasketches.memory.WritableMemory;
 import org.apache.datasketches.Family;
 import org.apache.datasketches.SketchesArgumentException;
 import org.apache.datasketches.Util;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+import org.testng.Assert;
+import org.testng.annotations.Test;
 
 /**
  * @author Lee Rhodes
@@ -158,52 +157,64 @@ public class PreambleUtilTest {
   @Test
   public void checkInsertsAndExtracts() {
     byte[] arr = new byte[32];
-    WritableMemory mem = WritableMemory.wrap(arr);
+    WritableMemory wmem = WritableMemory.wrap(arr);
 
     int v = 0;
-    insertPreLongs(mem, ++v);
-    assertEquals(extractPreLongs(mem), v);
+    insertPreLongs(wmem, ++v);
+    assertEquals(extractPreLongs(wmem), v);
+    insertPreLongs(wmem, 0);
 
-    insertLgResizeFactor(mem, 3); //limited to 2 bits
-    assertEquals(extractLgResizeFactor(mem), 3);
+    insertLgResizeFactor(wmem, 3); //limited to 2 bits
+    assertEquals(extractLgResizeFactor(wmem), 3);
+    insertLgResizeFactor(wmem, 0);
 
-    insertSerVer(mem, ++v);
-    assertEquals(extractSerVer(mem), v);
+    insertSerVer(wmem, ++v);
+    assertEquals(extractSerVer(wmem), v);
+    insertSerVer(wmem, 0);
 
-    insertFamilyID(mem, ++v);
-    assertEquals(extractFamilyID(mem), v);
+    insertFamilyID(wmem, ++v);
+    assertEquals(extractFamilyID(wmem), v);
+    insertFamilyID(wmem, 0);
 
-    insertLgNomLongs(mem, ++v);
-    assertEquals(extractLgNomLongs(mem), v);
+    insertLgNomLongs(wmem, ++v);
+    assertEquals(extractLgNomLongs(wmem), v);
+    insertLgNomLongs(wmem, 0);
 
-    insertLgArrLongs(mem, ++v);
-    assertEquals(extractLgArrLongs(mem), v);
+    insertLgArrLongs(wmem, ++v);
+    assertEquals(extractLgArrLongs(wmem), v);
+    insertLgArrLongs(wmem, 0);
 
-    insertFlags(mem, 3);
-    assertEquals(extractFlags(mem), 3);
-    assertEquals(extractLgResizeRatioV1(mem), 3); //also at byte 5, limited to 2 bits
+    insertFlags(wmem, 3);
+    assertEquals(extractFlags(wmem), 3);
+    assertEquals(extractLgResizeRatioV1(wmem), 3); //also at byte 5, limited to 2 bits
+    insertFlags(wmem, 0);
 
-    insertSeedHash(mem, ++v);
-    assertEquals(extractSeedHash(mem), v);
-    assertEquals(extractFlagsV1(mem), v); //also at byte 6
+    insertSeedHash(wmem, ++v);
+    assertEquals(extractSeedHash(wmem), v);
+    assertEquals(extractFlagsV1(wmem), v); //also at byte 6
+    insertSeedHash(wmem, 0);
 
-    insertCurCount(mem, ++v);
-    assertEquals(extractCurCount(mem), v);
+    insertCurCount(wmem, ++v);
+    assertEquals(extractCurCount(wmem), v);
+    insertCurCount(wmem, 0);
 
-    insertP(mem, (float) 1.0);
-    assertEquals(extractP(mem), (float) 1.0);
+    insertP(wmem, (float) 1.0);
+    assertEquals(extractP(wmem), (float) 1.0);
+    insertP(wmem, (float) 0.0);
 
-    insertThetaLong(mem, ++v);
-    assertEquals(extractThetaLong(mem), v);
+    insertThetaLong(wmem, ++v);
+    assertEquals(extractThetaLong(wmem), v);
+    insertThetaLong(wmem, 0L);
 
-    insertUnionThetaLong(mem, ++v);
-    assertEquals(extractUnionThetaLong(mem), v);
+    insertUnionThetaLong(wmem, ++v);
+    assertEquals(extractUnionThetaLong(wmem), v);
+    insertUnionThetaLong(wmem, 0L);
 
-    setEmpty(mem);
-    assertTrue(isEmpty(mem));
+    setEmpty(wmem);
+    assertTrue(isEmptyFlag(wmem));
 
-    clearEmpty(mem);
-    assertFalse(isEmpty(mem));
+    clearEmpty(wmem);
+    assertFalse(isEmptyFlag(wmem));
   }
 
   @Test
diff --git a/src/test/java/org/apache/datasketches/theta/PairwiseCornerCasesTest.java b/src/test/java/org/apache/datasketches/theta/SetOpsCornerCasesTest.java
similarity index 54%
rename from src/test/java/org/apache/datasketches/theta/PairwiseCornerCasesTest.java
rename to src/test/java/org/apache/datasketches/theta/SetOpsCornerCasesTest.java
index 96b9573..0095a47 100644
--- a/src/test/java/org/apache/datasketches/theta/PairwiseCornerCasesTest.java
+++ b/src/test/java/org/apache/datasketches/theta/SetOpsCornerCasesTest.java
@@ -19,186 +19,224 @@
 
 package org.apache.datasketches.theta;
 
-import static org.apache.datasketches.theta.PairwiseCornerCasesTest.State.EMPTY;
-import static org.apache.datasketches.theta.PairwiseCornerCasesTest.State.EST_HEAP;
-import static org.apache.datasketches.theta.PairwiseCornerCasesTest.State.EST_MEMORY_UNORDERED;
-import static org.apache.datasketches.theta.PairwiseCornerCasesTest.State.EXACT;
-import static org.apache.datasketches.theta.PairwiseCornerCasesTest.State.NULL;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EMPTY;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EST_HEAP;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EST_MEMORY_UNORDERED;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.EXACT;
+import static org.apache.datasketches.theta.SetOpsCornerCasesTest.State.NULL;
 import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertNull;
 
 import java.util.Random;
 
+import org.apache.datasketches.memory.WritableMemory;
 import org.testng.Assert;
 import org.testng.annotations.Test;
 
-import org.apache.datasketches.memory.WritableMemory;
-
-@SuppressWarnings("javadoc")
-public class PairwiseCornerCasesTest {
+@SuppressWarnings({"javadoc","deprecation"})
+public class SetOpsCornerCasesTest {
 
+  /*******************************************/
   Random rand = new Random(9001); //deterministic
 
   @Test
-  public void checkSetOps() {
+  public void checkSetOpsRandom() {
     int hiA = 0, loB = 0, hiB = 0;
     for (int i = 0; i < 1000; i++) {
-      hiA = 0 + rand.nextInt(128);
+      hiA = rand.nextInt(128);      //skA fed values between 0 and 127
       loB = rand.nextInt(64);
-      hiB = 0 + loB + rand.nextInt(64);
-      compareSetOps(64, 0, hiA, loB, hiB);
+      hiB = loB + rand.nextInt(64); //skB fed up to 63 values starting at loB
+      compareSetOpsRandom(64, 0, hiA, loB, hiB);
     }
   }
 
-  private static void compareSetOps(int k, int loA, int hiA, int loB, int hiB) {
-    UpdateSketch skA = Sketches.updateSketchBuilder().setNominalEntries(k).build();
-    UpdateSketch skB = Sketches.updateSketchBuilder().setNominalEntries(k).build();
-    Union union = Sketches.setOperationBuilder().setNominalEntries(k).buildUnion();
-    Intersection inter = Sketches.setOperationBuilder().buildIntersection();
-    AnotB aNotB = Sketches.setOperationBuilder().buildANotB();
+  private static void compareSetOpsRandom(int k, int loA, int hiA, int loB, int hiB) {
+    UpdateSketch tskA = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+    UpdateSketch tskB = Sketches.updateSketchBuilder().setNominalEntries(k).build();
 
-    for (int i = loA; i < hiA; i++) {
-      skA.update(i);
-    }
+    for (int i = loA; i < hiA; i++) { tskA.update(i); }
+    for (int i = loB; i < hiB; i++) { tskB.update(i); }
 
-    for (int i = loB; i < hiB; i++) {
-      skB.update(i);
-    }
+    CompactSketch rcskStdU = doStdUnion(tskA, tskB, k, null);
+    CompactSketch rcskPwU = doPwUnion(tskA, tskB, k);
+    checkCornerCase(rcskPwU, rcskStdU);
+
+    CompactSketch rcskStdPairU = doStdPairUnion(tskA, tskB, k, null);
+    checkCornerCase(rcskStdPairU, rcskStdU);
+
+    CompactSketch rcskStdI = doStdIntersection(tskA, tskB, null);
+    CompactSketch rcskPwI = doPwIntersection(tskA, tskB);
+    checkCornerCase(rcskPwI, rcskStdI);
 
-    union.update(skA);
-    union.update(skB);
-    CompactSketch comp = union.getResult(true, null);
-    double unionEst = comp.getEstimate();
-
-    CompactSketch cskA = skA.compact();
-    CompactSketch cskB = skB.compact();
-    CompactSketch pwComp = PairwiseSetOperations.union(cskA, cskB, k);
-    double pwUnionEst = pwComp.getEstimate();
-    Assert.assertEquals(pwUnionEst, unionEst, 0.0);
-
-    inter.update(skA);
-    inter.update(skB);
-    comp = inter.getResult(true, null);
-    double interEst = comp.getEstimate();
-    cskA = skA.compact();
-    cskB = skB.compact();
-    pwComp = PairwiseSetOperations.intersect(cskA, cskB);
-    double pwInterEst = pwComp.getEstimate();
-    Assert.assertEquals(pwInterEst, interEst, 0.0);
-
-    aNotB.update(skA, skB);
-    comp = aNotB.getResult(true, null);
-    double aNbEst = comp.getEstimate();
-    cskA = skA.compact();
-    cskB = skB.compact();
-    pwComp = PairwiseSetOperations.aNotB(cskA, cskB);
-    double pwAnBEst = pwComp.getEstimate();
-    Assert.assertEquals(pwAnBEst, aNbEst);
+    CompactSketch rcskStdPairI = doStdPairIntersection(tskA, tskB, null);
+    checkCornerCase(rcskStdPairI, rcskStdI);
+
+    CompactSketch rcskStdAnotB = doStdAnotB(tskA, tskB, null);
+    CompactSketch rcskPwAnotB = doPwAnotB(tskA, tskB);
+    checkCornerCase(rcskPwAnotB, rcskStdAnotB);
+
+    CompactSketch rcskStdStatefulAnotB = doStdStatefulAnotB(tskA, tskB, null);
+    checkCornerCase(rcskStdStatefulAnotB, rcskStdAnotB);
   }
 
+  /*******************************************/
+
   @Test
-  //Check all corner cases against standard Union, except unordered, which is not allowed
+  //Check all corner cases against standard Union, Intersection, and AnotB.
+  //The unordered case is not tested
   public void compareCornerCases() {
     int k = 64;
     for (State stateA : State.values()) {
       for (State stateB : State.values()) {
         if ((stateA == EST_MEMORY_UNORDERED) || (stateB == EST_MEMORY_UNORDERED)) { continue; }
         cornerCaseChecks(stateA, stateB, k);
+        //cornerCaseChecksMemory(stateA, stateB, k);
       }
     }
   }
 
   @Test
-  public void checkNull_THLT1_CNT0_FALSE() {
-    cornerCaseChecks(State.NULL, State.THLT1_CNT0_FALSE, 64);
+  public void checkExactNull() {
+    cornerCaseChecksMemory(State.EXACT, State.NULL, 64);
+  }
+
+  private static void cornerCaseChecksMemory(State stateA, State stateB, int k) {
+    println("StateA: " + stateA + ", StateB: " + stateB);
+    CompactSketch tcskA = generate(stateA, k);
+    CompactSketch tcskB = generate(stateB, k);
+
+    WritableMemory wmem = WritableMemory.allocate(SetOperation.getMaxUnionBytes(k));
+
+    CompactSketch rcskStdU = doStdUnion(tcskA, tcskB, k, null);
+    CompactSketch rcskPwU = doPwUnion(tcskA, tcskB, k);
+    checkCornerCase(rcskPwU, rcskStdU);
+
+    rcskStdU = doStdUnion(tcskA, tcskB, k, wmem);
+    CompactSketch rcskStdPairU = doStdPairUnion(tcskA, tcskB, k, wmem);
+    checkCornerCase(rcskStdPairU, rcskStdU);
+
+    wmem = WritableMemory.allocate(SetOperation.getMaxIntersectionBytes(k));
+
+    CompactSketch rcskStdI = doStdIntersection(tcskA, tcskB, null);
+    CompactSketch rcskPwI = doPwIntersection(tcskA, tcskB);
+    checkCornerCase(rcskPwI, rcskStdI);
+
+    rcskStdI = doStdIntersection(tcskA, tcskB, wmem);
+    CompactSketch rcskStdPairI = doStdPairIntersection(tcskA, tcskB, wmem);
+    checkCornerCase(rcskStdPairI, rcskStdI);
+
+    wmem = WritableMemory.allocate(SetOperation.getMaxAnotBResultBytes(k));
+
+    CompactSketch rcskStdAnotB = doStdAnotB(tcskA, tcskB, null);
+    CompactSketch rcskPwAnotB = doPwAnotB(tcskA, tcskB);
+    checkCornerCase(rcskPwAnotB, rcskStdAnotB);
+
+    rcskStdAnotB = doStdAnotB(tcskA, tcskB, wmem);
+    CompactSketch rcskStdStatefulAnotB = doStdStatefulAnotB(tcskA, tcskB, wmem);
+    checkCornerCase(rcskStdStatefulAnotB, rcskStdAnotB);
   }
 
   private static void cornerCaseChecks(State stateA, State stateB, int k) {
     println("StateA: " + stateA + ", StateB: " + stateB);
-    CompactSketch cskA = generate(stateA, k);
-    CompactSketch cskB = generate(stateB, k);
+    CompactSketch tcskA = generate(stateA, k);
+    CompactSketch tcskB = generate(stateB, k);
+
+    CompactSketch rcskStdU = doStdUnion(tcskA, tcskB, k, null);
+    CompactSketch rcskPwU = doPwUnion(tcskA, tcskB, k);
+    checkCornerCase(rcskPwU, rcskStdU);
+
+    CompactSketch rcskStdPairU = doStdPairUnion(tcskA, tcskB, k, null);
+    checkCornerCase(rcskStdPairU, rcskStdU);
+
+    CompactSketch rcskStdI = doStdIntersection(tcskA, tcskB, null);
+    CompactSketch rcskPwI = doPwIntersection(tcskA, tcskB);
+    checkCornerCase(rcskPwI, rcskStdI);
+
+    CompactSketch rcskStdPairI = doStdPairIntersection(tcskA, tcskB, null);
+    checkCornerCase(rcskStdPairI, rcskStdI);
+
+    CompactSketch rcskStdAnotB = doStdAnotB(tcskA, tcskB, null);
+    CompactSketch rcskPwAnotB = doPwAnotB(tcskA, tcskB);
+    checkCornerCase(rcskPwAnotB, rcskStdAnotB);
+
+    CompactSketch rcskStdStatefulAnotB = doStdStatefulAnotB(tcskA, tcskB, null);
+    checkCornerCase(rcskStdStatefulAnotB, rcskStdAnotB);
+  }
+
+  private static CompactSketch doStdUnion(Sketch tskA, Sketch tskB, int k, WritableMemory wmem) {
     Union union = Sketches.setOperationBuilder().setNominalEntries(k).buildUnion();
+    union.update(tskA);
+    union.update(tskB);
+    return union.getResult(true, wmem);
+  }
+
+  private static CompactSketch doStdPairUnion(Sketch tskA, Sketch tskB, int k, WritableMemory wmem) {
+    Union union = Sketches.setOperationBuilder().setNominalEntries(k).buildUnion();
+    return union.union(tskA, tskB, true, wmem);
+  }
+
+  private static CompactSketch doStdIntersection(Sketch tskA, Sketch tskB, WritableMemory wmem) {
     Intersection inter = Sketches.setOperationBuilder().buildIntersection();
-    AnotB aNotB = Sketches.setOperationBuilder().buildANotB();
-
-    //UNION
-    union.update(cskA);
-    union.update(cskB);
-    CompactSketch comp = union.getResult(true, null);
-    double stdEst = comp.getEstimate();
-    boolean stdEmpty = comp.isEmpty();
-    double stdTheta = comp.getTheta();
-    int stdEnt = comp.getRetainedEntries(true);
-
-    CompactSketch pwComp = PairwiseSetOperations.union(cskA, cskB, k);
-    double pwEst = pwComp.getEstimate();
-    boolean pwEmpty = pwComp.isEmpty();
-    double pwTheta = pwComp.getTheta();
-    int pwEnt = pwComp.getRetainedEntries(true);
-
-    if ((stateA == NULL) && (stateB == NULL)) {
-      Assert.assertEquals(pwEst,  0.0, 0.0);
-      Assert.assertEquals(stdEst, 0.0, 0.0);
-    } else {
-      Assert.assertEquals(pwEst, stdEst, 0.0);
-    }
-    assert pwEmpty == stdEmpty;
-    Assert.assertEquals(pwEmpty, stdEmpty);
-    Assert.assertEquals(pwTheta, stdTheta, 0.0);
-    Assert.assertEquals(pwEnt, stdEnt);
-
-    //INTERSECT
-    inter.update(cskA);
-    inter.update(cskB);
-    comp = inter.getResult(true, null);
-    stdEst = comp.getEstimate();
-    stdEmpty = comp.isEmpty();
-    stdTheta = comp.getTheta();
-    stdEnt = comp.getRetainedEntries(true);
-
-    pwComp = PairwiseSetOperations.intersect(cskA, cskB);
-    pwEst = pwComp.getEstimate();
-    pwEmpty = pwComp.isEmpty();
-    pwTheta = pwComp.getTheta();
-    pwEnt = pwComp.getRetainedEntries(true);
-
-    if ((stateA == NULL) && (stateB == NULL)) {
-      Assert.assertEquals(pwEst,  0.0, 0.0);
-      Assert.assertEquals(stdEst, 0.0, 0.0);
-    } else {
-      Assert.assertEquals(pwEst, stdEst, 0.0);
-    }
-    Assert.assertEquals(pwEmpty, stdEmpty);
-    Assert.assertEquals(pwTheta, stdTheta, 0.0);
-    Assert.assertEquals(pwEnt, stdEnt);
+    inter.update(tskA);
+    inter.update(tskB);
+    return inter.getResult(true, wmem);
+  }
 
-    //A NOT B
-    aNotB.update(cskA, cskB);
-    comp = aNotB.getResult(true, null);
-    stdEst =comp.getEstimate();
-    stdEmpty = comp.isEmpty();
-    stdTheta = comp.getTheta();
-    stdEnt = comp.getRetainedEntries(true);
-
-    pwComp = PairwiseSetOperations.aNotB(cskA, cskB);
-    pwEst = pwComp.getEstimate();
-    pwEmpty = pwComp.isEmpty();
-    pwTheta = pwComp.getTheta();
-    pwEnt = pwComp.getRetainedEntries(true);
-
-    if ((stateA == NULL) && (stateB == NULL)) {
-      Assert.assertEquals(pwEst,  0.0, 0.0);
-      Assert.assertEquals(stdEst, 0.0, 0.0);
-    } else {
-      Assert.assertEquals(pwEst, stdEst, 0.0);
-    }
-    Assert.assertEquals(pwEmpty, stdEmpty);
-    Assert.assertEquals(pwTheta, stdTheta, 0.0);
-    Assert.assertEquals(pwEnt, stdEnt);
+  private static CompactSketch doStdPairIntersection(Sketch tskA, Sketch tskB, WritableMemory wmem) {
+    Intersection inter = Sketches.setOperationBuilder().buildIntersection();
+    return inter.intersect(tskA, tskB, true, wmem);
+  }
+
+  private static CompactSketch doStdAnotB(Sketch tskA, Sketch tskB, WritableMemory wmem) {
+    AnotB anotb = Sketches.setOperationBuilder().buildANotB();
+    return anotb.aNotB(tskA, tskB, true, wmem);
   }
 
+  private static CompactSketch doStdStatefulAnotB(Sketch tskA, Sketch tskB, WritableMemory wmem) {
+    AnotB anotb = Sketches.setOperationBuilder().buildANotB();
+    anotb.setA(tskA);
+    anotb.notB(tskB);
+    anotb.getResult(false);
+    return anotb.getResult(true, wmem, true);
+  }
+
+  private static CompactSketch doPwUnion(Sketch tskA, Sketch tskB, int k) {
+    CompactSketch tcskA, tcskB;
+    if (tskA == null) { tcskA = null; }
+    else { tcskA = (tskA instanceof CompactSketch) ? (CompactSketch) tskA : tskA.compact(); }
+    if (tskB == null) { tcskB = null; }
+    else { tcskB = (tskB instanceof CompactSketch) ? (CompactSketch) tskB : tskB.compact(); }
+    return PairwiseSetOperations.union(tcskA, tcskB, k);
+  }
+
+  private static CompactSketch doPwIntersection(Sketch tskA, Sketch tskB) {
+    return PairwiseSetOperations.intersect(tskA, tskB);
+  }
+
+  private static CompactSketch doPwAnotB(Sketch tskA, Sketch tskB) {
+    return PairwiseSetOperations.aNotB(tskA, tskB);
+  }
+
+  private static void checkCornerCase(Sketch rskA, Sketch rskB) {
+    double estA = rskA.getEstimate();
+    double estB = rskB.getEstimate();
+    boolean emptyA = rskA.isEmpty();
+    boolean emptyB = rskB.isEmpty();
+    long thetaLongA = rskA.getThetaLong();
+    long thetaLongB = rskB.getThetaLong();
+    int countA = rskA.getRetainedEntries(true);
+    int countB = rskB.getRetainedEntries(true);
+    Assert.assertEquals(estB, estA, 0.0);
+    Assert.assertEquals(emptyB, emptyA);
+    Assert.assertEquals(thetaLongB, thetaLongA);
+    Assert.assertEquals(countB, countA);
+    Assert.assertEquals(rskA.getClass().getSimpleName(), rskB.getClass().getSimpleName());
+  }
+
+  /*******************************************/
+
   @Test
-  public void checkNotOrdered() {
+  public void checkUnionNotOrdered() {
     int k = 64;
     CompactSketch skNull = generate(NULL, k);
     CompactSketch skEmpty = generate(EMPTY, k);
@@ -237,18 +275,20 @@ public class PairwiseCornerCasesTest {
   public void checkSeedHash() {
     int k = 64;
     UpdateSketch tmp1 = Sketches.updateSketchBuilder().setNominalEntries(k).setSeed(123).build();
-    tmp1.update(1); tmp1.update(3);
-    CompactSketch skSmallSeed2 = tmp1.compact(true, null);
+    tmp1.update(1);
+    tmp1.update(3);
+    CompactSketch skSmallSeed2A = tmp1.compact(true, null);
 
     UpdateSketch tmp2 = Sketches.updateSketchBuilder().setNominalEntries(k).setSeed(123).build();
-    tmp2.update(1); tmp2.update(2);
+    tmp2.update(1);
+    tmp2.update(2);
     CompactSketch skSmallSeed2B = tmp2.compact(true, null);
 
     CompactSketch skExact = generate(EXACT, k);
     CompactSketch skHeap = generate(EST_HEAP, 2 * k);
     //Intersect
     try {
-      PairwiseSetOperations.intersect(skExact, skSmallSeed2);
+      PairwiseSetOperations.intersect(skExact, skSmallSeed2A);
       Assert.fail();
     } catch (Exception e) { } //pass
     try {
@@ -265,7 +305,7 @@ public class PairwiseCornerCasesTest {
     } catch (Exception e) { } //pass
     //A NOT B
     try {
-      PairwiseSetOperations.aNotB(skExact, skSmallSeed2);
+      PairwiseSetOperations.aNotB(skExact, skSmallSeed2A);
       Assert.fail();
     } catch (Exception e) { } //pass
     try {
@@ -282,7 +322,7 @@ public class PairwiseCornerCasesTest {
     } catch (Exception e) { } //pass
     //Union
     try {
-      PairwiseSetOperations.union(skExact, skSmallSeed2);
+      PairwiseSetOperations.union(skExact, skSmallSeed2A);
       Assert.fail();
     } catch (Exception e) { } //pass
     try {
@@ -297,11 +337,10 @@ public class PairwiseCornerCasesTest {
       PairwiseSetOperations.union(skHeap, skSmallSeed2B);
       Assert.fail();
     } catch (Exception e) { } //pass
-
   }
 
   @Test
-  public void checkReduceToK() {
+  public void checkPwUnionReduceToK() {
     int k = 16;
     CompactSketch skNull = generate(NULL, k);
     CompactSketch skEmpty = generate(EMPTY, k);
@@ -320,6 +359,9 @@ public class PairwiseCornerCasesTest {
     Assert.assertEquals(csk.getRetainedEntries(), k);
   }
 
+
+
+
   @Test
   public void printlnTest() {
     println("PRINTING: "+this.getClass().getName());
@@ -333,7 +375,7 @@ public class PairwiseCornerCasesTest {
   }
 
   @Test
-  public void checkGenerate() {
+  public void checkGenerator() {
     int k = 16;
     CompactSketch csk;
 
@@ -349,6 +391,15 @@ public class PairwiseCornerCasesTest {
     assertEquals(csk.hasMemory(), false);
     assertEquals(csk.isOrdered(), true);
 
+    csk = generate(State.SINGLE, k);
+    assertEquals(csk.isEmpty(), false);
+    assertEquals(csk.isEstimationMode(), false);
+    assertEquals(csk.getRetainedEntries(), 1);
+    assertEquals(csk.getThetaLong(), Long.MAX_VALUE);
+    assertEquals(csk.isDirect(), false);
+    assertEquals(csk.hasMemory(), false);
+    assertEquals(csk.isOrdered(), true);
+
     csk = generate(State.EXACT, k);
     assertEquals(csk.isEmpty(), false);
     assertEquals(csk.isEstimationMode(), false);
@@ -395,7 +446,7 @@ public class PairwiseCornerCasesTest {
     assertEquals(csk.isOrdered(), false);
   }
 
-  enum State {NULL, EMPTY, EXACT, EST_HEAP, THLT1_CNT0_FALSE, THEQ1_CNT0_TRUE, EST_MEMORY_UNORDERED}
+  enum State {NULL, EMPTY, SINGLE, EXACT, EST_HEAP, THLT1_CNT0_FALSE, THEQ1_CNT0_TRUE, EST_MEMORY_UNORDERED}
 
   private static CompactSketch generate(State state, int k) {
     UpdateSketch sk = null;
@@ -410,6 +461,12 @@ public class PairwiseCornerCasesTest {
         csk = Sketches.updateSketchBuilder().setNominalEntries(k).build().compact(true, null);
         break;
       }
+      case SINGLE : { //results in SingleItemSketches most of the time
+        sk = Sketches.updateSketchBuilder().setNominalEntries(k).build();
+        sk.update(1);
+        csk = sk.compact(true, null);
+        break;
+      }
       case EXACT : {
         sk = Sketches.updateSketchBuilder().setNominalEntries(k).build();
         for (int i = 0; i < k; i++) {
diff --git a/src/test/java/org/apache/datasketches/theta/SingleItemSketchTest.java b/src/test/java/org/apache/datasketches/theta/SingleItemSketchTest.java
index 5f12e97..23a0a37 100644
--- a/src/test/java/org/apache/datasketches/theta/SingleItemSketchTest.java
+++ b/src/test/java/org/apache/datasketches/theta/SingleItemSketchTest.java
@@ -22,7 +22,6 @@ package org.apache.datasketches.theta;
 import static org.apache.datasketches.Util.DEFAULT_UPDATE_SEED;
 import static org.apache.datasketches.Util.computeSeedHash;
 import static org.apache.datasketches.hash.MurmurHash3.hash;
-import static org.apache.datasketches.theta.PreambleUtil.MAX_THETA_LONG_AS_DOUBLE;
 import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertFalse;
 import static org.testng.Assert.assertNull;
@@ -123,7 +122,6 @@ public class SingleItemSketchTest {
   @Test
   public void checkSketchInterface() {
     SingleItemSketch sis = SingleItemSketch.create(1);
-    assertEquals(sis.getCountLessThanTheta(1.0), 1);
     assertEquals(sis.getCurrentBytes(true), 16);
     assertEquals(sis.getEstimate(), 1.0);
     assertEquals(sis.getLowerBound(1), 1.0);
@@ -136,13 +134,14 @@ public class SingleItemSketchTest {
   }
 
   @Test
-  public void checkLessThanTheta() {
+  public void checkLessThanThetaLong() {
     for (int i = 0; i < 10; i++) {
       long[] data = { i };
       long h = hash(data, DEFAULT_UPDATE_SEED)[0] >>> 1;
-      double theta = h / MAX_THETA_LONG_AS_DOUBLE;
       SingleItemSketch sis = SingleItemSketch.create(i);
-      assertEquals(sis.getCountLessThanTheta(0.5), (theta < 0.5) ? 1 : 0);
+      long halfMax = Long.MAX_VALUE >> 1;
+      int count = sis.getCountLessThanThetaLong(halfMax);
+      assertEquals(count, (h < halfMax) ? 1 : 0);
     }
   }
 


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


[incubator-datasketches-java] 03/03: Interim changes, all tests work.

Posted by le...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

leerho pushed a commit to branch Refactor_Theta_Tuple
in repository https://gitbox.apache.org/repos/asf/incubator-datasketches-java.git

commit bc77a086f7b772b73873a18dbe3bfac051cb048b
Author: Lee Rhodes <le...@users.noreply.github.com>
AuthorDate: Fri Jun 26 14:11:17 2020 -0700

    Interim changes, all tests work.
---
 .../org/apache/datasketches/theta/AnotBimpl.java   | 177 +++++++++++----------
 .../datasketches/theta/CompactOperations.java      | 156 +++++++++++++-----
 .../apache/datasketches/theta/CompactSketch.java   |  63 --------
 .../theta/DirectCompactOrderedSketch.java          |  46 ------
 .../datasketches/theta/DirectCompactSketch.java    | 102 ++++--------
 .../theta/DirectCompactUnorderedSketch.java        |  44 -----
 .../theta/DirectQuickSelectSketchR.java            |   2 +-
 .../apache/datasketches/theta/HeapAlphaSketch.java |   2 +-
 .../theta/HeapCompactOrderedSketch.java            |  12 +-
 .../datasketches/theta/HeapCompactSketch.java      |   1 +
 .../theta/HeapCompactUnorderedSketch.java          |  39 +----
 .../datasketches/theta/HeapQuickSelectSketch.java  |   2 +-
 .../datasketches/theta/IntersectionImplR.java      |   8 +-
 .../datasketches/theta/PairwiseSetOperations.java  |  19 ++-
 .../apache/datasketches/theta/PreambleUtil.java    |  71 ---------
 .../apache/datasketches/theta/SetOperation.java    |  52 +-----
 .../datasketches/theta/SetOperationBuilder.java    |   2 +-
 .../datasketches/theta/SingleItemSketch.java       |  48 ++++--
 .../java/org/apache/datasketches/theta/Sketch.java |  55 +++----
 .../org/apache/datasketches/theta/UnionImpl.java   |  10 +-
 .../apache/datasketches/theta/UpdateSketch.java    |   6 +-
 .../apache/datasketches/theta/AnotBimplTest.java   |   2 +-
 .../datasketches/theta/CompactSketchTest.java      | 116 ++++++++++++--
 .../ConcurrentDirectQuickSelectSketchTest.java     |  14 +-
 .../theta/ConcurrentHeapQuickSelectSketchTest.java |  17 +-
 .../theta/DirectQuickSelectSketchTest.java         |  13 +-
 .../org/apache/datasketches/theta/EmptyTest.java   |   7 +-
 .../theta/ForwardCompatibilityTest.java            |  25 +--
 .../datasketches/theta/HeapAlphaSketchTest.java    |  11 +-
 .../theta/HeapQuickSelectSketchTest.java           |  11 +-
 .../theta/PairwiseSetOperationsTest.java           |   2 +-
 .../datasketches/theta/SetOperationTest.java       |  11 +-
 .../datasketches/theta/SetOpsCornerCasesTest.java  |   3 +-
 .../datasketches/theta/SingleItemSketchTest.java   |   9 +-
 .../org/apache/datasketches/theta/SketchTest.java  |   9 +-
 .../tuple/adouble/AdoubleAnotBTest.java            |   2 +
 .../tuple/adouble/AdoubleIntersectionTest.java     |   1 +
 .../tuple/aninteger/IntegerSketchTest.java         |   3 +-
 .../tuple/strings/ArrayOfStringsSketchTest.java    |   5 +-
 tools/FindBugsExcludeFilter.xml                    |   3 +-
 40 files changed, 519 insertions(+), 662 deletions(-)

diff --git a/src/main/java/org/apache/datasketches/theta/AnotBimpl.java b/src/main/java/org/apache/datasketches/theta/AnotBimpl.java
index 702ae4f..576af7b 100644
--- a/src/main/java/org/apache/datasketches/theta/AnotBimpl.java
+++ b/src/main/java/org/apache/datasketches/theta/AnotBimpl.java
@@ -41,11 +41,11 @@ import org.apache.datasketches.memory.WritableMemory;
  * @author Kevin Lang
  */
 final class AnotBimpl extends AnotB {
-  private final short seedHashR_;
-  private boolean emptyR_;
-  private long thetaLongR_;
-  private long[] hashArrR_; //compact array w curCountR_ entries
-  private int curCountR_;
+  private final short seedHash_;
+  private boolean empty_;
+  private long thetaLong_;
+  private long[] hashArr_; //compact array w curCount_ entries
+  private int curCount_;
 
   //Remove all 4 of these with deprecated
   private Sketch skA_;
@@ -68,7 +68,7 @@ final class AnotBimpl extends AnotB {
    * @param seedHash 16 bit hash of the chosen update seed.
    */
   AnotBimpl(final short seedHash) {
-    seedHashR_ = seedHash;
+    seedHash_ = seedHash;
     reset();
   }
 
@@ -84,23 +84,23 @@ final class AnotBimpl extends AnotB {
       return;
     }
     //skA is not empty
-    checkSeedHashes(seedHashR_, skA.getSeedHash());
-    emptyR_ = false;
-    thetaLongR_ = skA.getThetaLong();
+    checkSeedHashes(seedHash_, skA.getSeedHash());
+    empty_ = false;
+    thetaLong_ = skA.getThetaLong();
     final CompactSketch cskA = (skA instanceof CompactSketch)
         ? (CompactSketch) skA
         : ((UpdateSketch) skA).compact();
-    hashArrR_ = skA.isDirect() ? cskA.getCache() : cskA.getCache().clone();
-    curCountR_ = cskA.getRetainedEntries(true);
+    hashArr_ = skA.isDirect() ? cskA.getCache() : cskA.getCache().clone();
+    curCount_ = cskA.getRetainedEntries(true);
   }
 
   @Override
   public void notB(final Sketch skB) {
-    if (emptyR_ || (skB == null) || skB.isEmpty()) { return; }
+    if (empty_ || (skB == null) || skB.isEmpty()) { return; }
     //skB is not empty
-    checkSeedHashes(seedHashR_, skB.getSeedHash());
+    checkSeedHashes(seedHash_, skB.getSeedHash());
     final long thetaLongB = skB.getThetaLong();
-    thetaLongR_ = Math.min(thetaLongR_, thetaLongB);
+    thetaLong_ = Math.min(thetaLong_, thetaLongB);
 
     //Build hashtable and removes hashes of skB >= theta
     final int countB = skB.getRetainedEntries();
@@ -109,24 +109,24 @@ final class AnotBimpl extends AnotB {
     final long[] hashTableB;
     if (skB instanceof CompactSketch) {
       cskB = (CompactSketch) skB;
-      hashTableB = convertToHashTable(cskB.getCache(), countB, thetaLongR_, REBUILD_THRESHOLD);
+      hashTableB = convertToHashTable(cskB.getCache(), countB, thetaLong_, REBUILD_THRESHOLD);
     } else {
       uskB = (UpdateSketch) skB;
-      hashTableB = (thetaLongR_ < thetaLongB)
-          ? convertToHashTable(uskB.getCache(), countB, thetaLongR_, REBUILD_THRESHOLD)
+      hashTableB = (thetaLong_ < thetaLongB)
+          ? convertToHashTable(uskB.getCache(), countB, thetaLong_, REBUILD_THRESHOLD)
           : uskB.getCache();
       cskB = uskB.compact();
     }
 
     //build temporary arrays of skA
-    final long[] tmpHashArrA = new long[curCountR_];
+    final long[] tmpHashArrA = new long[curCount_];
 
     //search for non matches and build temp arrays
     final int lgHTBLen = simpleIntLog2(hashTableB.length);
     int nonMatches = 0;
-    for (int i = 0; i < curCountR_; i++) {
-      final long hash = hashArrR_[i];
-      if ((hash != 0) && (hash < thetaLongR_)) { //skips hashes of A >= theta
+    for (int i = 0; i < curCount_; i++) {
+      final long hash = hashArr_[i];
+      if ((hash != 0) && (hash < thetaLong_)) { //skips hashes of A >= theta
         final int index = hashSearch(hashTableB, lgHTBLen, hash);
         if (index == -1) {
           tmpHashArrA[nonMatches] = hash;
@@ -134,9 +134,9 @@ final class AnotBimpl extends AnotB {
         }
       }
     }
-    hashArrR_ = Arrays.copyOfRange(tmpHashArrA, 0, nonMatches);
-    curCountR_ = nonMatches;
-    emptyR_ = (nonMatches == 0) && (thetaLongR_ == Long.MAX_VALUE);
+    hashArr_ = Arrays.copyOfRange(tmpHashArrA, 0, nonMatches);
+    curCount_ = nonMatches;
+    empty_ = (nonMatches == 0) && (thetaLong_ == Long.MAX_VALUE);
   }
 
   @Override
@@ -148,7 +148,7 @@ final class AnotBimpl extends AnotB {
   public CompactSketch getResult(final boolean dstOrdered, final WritableMemory dstMem,
       final boolean reset) {
     final CompactSketch result =
-        getResult(hashArrR_, emptyR_, seedHashR_, curCountR_, thetaLongR_, dstOrdered, dstMem);
+        getResult(hashArr_, empty_, seedHash_, curCount_, thetaLong_, dstOrdered, dstMem);
     if (reset) { reset(); }
     return result;
   }
@@ -204,8 +204,8 @@ final class AnotBimpl extends AnotB {
     if ((skB == null) || skB.isEmpty()) { return skA.compact(dstOrdered, dstMem); }
     final short seedHashA = skA.getSeedHash();
     final short seedHashB = skB.getSeedHash();
-    checkSeedHashes(seedHashA, seedHashR_);
-    checkSeedHashes(seedHashB, seedHashR_);
+    checkSeedHashes(seedHashA, seedHash_);
+    checkSeedHashes(seedHashB, seedHash_);
 
     //Both skA & skB are not empty
     //Load skA into local tmp registers
@@ -245,18 +245,18 @@ final class AnotBimpl extends AnotB {
     empty = ((nonMatches == 0) && (thetaLong == Long.MAX_VALUE));
     final long[] hashArrOut = Arrays.copyOfRange(tmpHashArrA, 0, nonMatches);
     final CompactSketch result =
-        AnotBimpl.getResult(hashArrOut, empty, seedHashR_, nonMatches, thetaLong, dstOrdered, dstMem);
+        AnotBimpl.getResult(hashArrOut, empty, seedHash_, nonMatches, thetaLong, dstOrdered, dstMem);
     return result;
   }
 
   @Override
   int getRetainedEntries(final boolean valid) {
-    return curCountR_;
+    return curCount_;
   }
 
   @Override
   boolean isEmpty() {
-    return emptyR_;
+    return empty_;
   }
 
   @Override
@@ -266,41 +266,42 @@ final class AnotBimpl extends AnotB {
 
   //Deprecated methods
 
+  @Deprecated
   @Override
   public void update(final Sketch a, final Sketch b) {
     skA_ = a;
     skB_ = b;
-    thetaLongR_ = Long.MAX_VALUE;
-    emptyR_ = true;
-    hashArrR_ = null;
-    curCountR_ = 0;
+    thetaLong_ = Long.MAX_VALUE; //*
+    empty_ = true; //*
+    hashArr_ = null; //*
+    curCount_ = 0; //*
     lgArrLongsHT_ = 5;
     bHashTable_ = null;
     compute();
   }
 
+  @Deprecated
   @Override
   public CompactSketch getResult() {
     return getResult(true, null);
   }
 
+  @Deprecated
   @Override
   public CompactSketch getResult(final boolean dstOrdered, final WritableMemory dstMem) {
-    final long[] compactCache = (curCountR_ <= 0)
+    final long[] compactCache = (curCount_ <= 0)
         ? new long[0]
-        : Arrays.copyOfRange(hashArrR_, 0, curCountR_);
-    if (dstOrdered && (curCountR_ > 1)) {
+        : Arrays.copyOfRange(hashArr_, 0, curCount_);
+    if (dstOrdered && (curCount_ > 1)) {
       Arrays.sort(compactCache);
     }
     //Create the CompactSketch
-    final CompactSketch comp = createCompactSketch(
-        compactCache, emptyR_, seedHashR_, curCountR_, thetaLongR_, dstOrdered, dstMem);
-    reset(); //TODO
-    return comp;
+    final CompactSketch csk = CompactOperations.componentsToCompact(
+        thetaLong_, curCount_, seedHash_, empty_, true, dstOrdered, dstOrdered, dstMem, compactCache);
+    reset();
+    return csk;
   }
 
-
-
   //restricted
 
   void compute() {
@@ -370,86 +371,86 @@ final class AnotBimpl extends AnotB {
 
     switch (sw) {
       case 0 :  //A Null, B Null;    Return (1.0, 0, T)
-        thetaLongR_ = Long.MAX_VALUE;
-        emptyR_ = true;
+        thetaLong_ = Long.MAX_VALUE;
+        empty_ = true;
         break;
 
       case 10:   //A Empty, B Compact; CheckAB, Return (1.0, 0, T)
       case 11:   //A Empty, B Ordered; CheckAB, Return (1.0, 0, T)
       case 12:   //A Empty, B HashTbl; CheckAB, Return (1.0, 0, T)
-        checkSeedHashes(seedHashR_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        checkSeedHashes(seedHash_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
         //$FALL-THROUGH$
       case 1:    //A Null, B Empty;   CheckB,  Return (1.0, 0, T)
       case 2:    //A Null, B Compact; CheckB,  Return (1.0, 0, T)
       case 3:    //A Null, B Ordered; CheckB,  Return (1.0, 0, T)
       case 4:    //A Null, B HashTbl; CheckB,  Return (1.0, 0, T)
-        checkSeedHashes(seedHashR_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        thetaLongR_ = Long.MAX_VALUE;
-        emptyR_ = true;
+        checkSeedHashes(seedHash_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        thetaLong_ = Long.MAX_VALUE;
+        empty_ = true;
         break;
 
       case 9:   //A Empty, B Empty;   CheckAB, Return (1.0, 0, T)
-        checkSeedHashes(seedHashR_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        checkSeedHashes(seedHash_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
         //$FALL-THROUGH$
       case 8:   //A Empty, B Null;    CheckA,  Return (1.0, 0, T)
-        checkSeedHashes(seedHashR_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        thetaLongR_ = Long.MAX_VALUE;
-        emptyR_ = true;
+        checkSeedHashes(seedHash_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        thetaLong_ = Long.MAX_VALUE;
+        empty_ = true;
         break;
 
       case 17:   //A Compact, B Empty; CheckAB, Return (ThA, |A|, F), copyA
       case 25:   //A Ordered, B Empty; CheckAB, Return (ThA, |A|, F), copyA
       case 33:  //A HashTbl, B Empty; CheckAB, Return (ThA, |A|, F), copyA
-        checkSeedHashes(seedHashR_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        checkSeedHashes(seedHash_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
         //$FALL-THROUGH$
       case 16:   //A Compact, B Null;  CheckA,  Return (ThA, |A|, F), copyA
       case 24:   //A Ordered, B Null;  CheckA,  Return (ThA, |A|, F), copyA
       case 32:  //A HashTbl, B Null;  CheckA,  Return (ThA, |A|, F), copyA
-        checkSeedHashes(seedHashR_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        thetaLongR_ = skA_.getThetaLong();
-        emptyR_ = false;
-        curCountR_ = skA_.getRetainedEntries(true);
-        hashArrR_ = CompactOperations.compactCache(skA_.getCache(), curCountR_, thetaLongR_, false);
+        checkSeedHashes(seedHash_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        thetaLong_ = skA_.getThetaLong();
+        empty_ = false;
+        curCount_ = skA_.getRetainedEntries(true);
+        hashArr_ = CompactOperations.compactCache(skA_.getCache(), curCount_, thetaLong_, false);
         break;
 
       case 18:   //A Compact, B Compact; CheckAB, B -> H; => C,H; scanAllAsearchB()
       case 19:   //A Compact, B Ordered; CheckAB, B -> H; => C,H; scanAllAsearchB()
       case 34:   //A HashTbl, B Compact; CheckAB, B -> H; => H,H; scanAllAsearchB()
       case 35:  //A HashTbl, B Ordered; CheckAB, B -> H; => H,H; scanAllAsearchB()
-        checkSeedHashes(seedHashR_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        checkSeedHashes(seedHashR_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        thetaLongR_ = min(skA_.getThetaLong(), skB_.getThetaLong());
-        emptyR_ = false;
+        checkSeedHashes(seedHash_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        checkSeedHashes(seedHash_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        thetaLong_ = min(skA_.getThetaLong(), skB_.getThetaLong());
+        empty_ = false;
         convertBtoHT();
         scanAllAsearchB();
         break;
 
       case 26:   //A Ordered, B Compact; CheckAB, B -> H; => O,H; scanEarlyStopAsearchB()
       case 27:  //A Ordered, B Ordered; CheckAB, B -> H; => O,H; scanEarlyStopAsearchB()
-        checkSeedHashes(seedHashR_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        checkSeedHashes(seedHashR_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        thetaLongR_ = min(skA_.getThetaLong(), skB_.getThetaLong());
-        emptyR_ = false;
+        checkSeedHashes(seedHash_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        checkSeedHashes(seedHash_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        thetaLong_ = min(skA_.getThetaLong(), skB_.getThetaLong());
+        empty_ = false;
         convertBtoHT();
         scanEarlyStopAsearchB();
         break;
 
       case 20:   //A Compact, B HashTbl; CheckAB, scanAllAsearchB()
       case 36:  //A HashTbl, B HashTbl; CheckAB, scanAllAsearchB()
-        checkSeedHashes(seedHashR_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        checkSeedHashes(seedHashR_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        thetaLongR_ = min(skA_.getThetaLong(), skB_.getThetaLong());
-        emptyR_ = false;
+        checkSeedHashes(seedHash_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        checkSeedHashes(seedHash_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        thetaLong_ = min(skA_.getThetaLong(), skB_.getThetaLong());
+        empty_ = false;
         lgArrLongsHT_ = ((UpdateSketch)skB_).getLgArrLongs();
         bHashTable_ = skB_.getCache();
         scanAllAsearchB();
         break;
 
       case 28:  //A Ordered, B HashTbl; CheckAB, scanEarlyStopAsearchB()
-        checkSeedHashes(seedHashR_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        checkSeedHashes(seedHashR_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
-        thetaLongR_ = min(skA_.getThetaLong(), skB_.getThetaLong());
-        emptyR_ = false;
+        checkSeedHashes(seedHash_, skA_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        checkSeedHashes(seedHash_, skB_.getSeedHash());//lgtm [java/dereferenced-value-may-be-null]
+        thetaLong_ = min(skA_.getThetaLong(), skB_.getThetaLong());
+        empty_ = false;
         lgArrLongsHT_ = ((UpdateSketch)skB_).getLgArrLongs();
         bHashTable_ = skB_.getCache();
         scanEarlyStopAsearchB();
@@ -463,20 +464,20 @@ final class AnotBimpl extends AnotB {
     final int curCountB = skB_.getRetainedEntries(true);
     lgArrLongsHT_ = computeMinLgArrLongsFromCount(curCountB);
     bHashTable_ = new long[1 << lgArrLongsHT_];
-    hashArrayInsert(skB_.getCache(), bHashTable_, lgArrLongsHT_, thetaLongR_);
+    hashArrayInsert(skB_.getCache(), bHashTable_, lgArrLongsHT_, thetaLong_);
   }
 
   //Sketch A is either unordered compact or hash table
   private void scanAllAsearchB() {
     final long[] scanAArr = skA_.getCache();
     final int arrLongsIn = scanAArr.length;
-    hashArrR_ = new long[arrLongsIn];
+    hashArr_ = new long[arrLongsIn];
     for (int i = 0; i < arrLongsIn; i++ ) {
       final long hashIn = scanAArr[i];
-      if ((hashIn <= 0L) || (hashIn >= thetaLongR_)) { continue; }
+      if ((hashIn <= 0L) || (hashIn >= thetaLong_)) { continue; }
       final int foundIdx = hashSearch(bHashTable_, lgArrLongsHT_, hashIn);
       if (foundIdx > -1) { continue; }
-      hashArrR_[curCountR_++] = hashIn;
+      hashArr_[curCount_++] = hashIn;
     }
   }
 
@@ -484,26 +485,26 @@ final class AnotBimpl extends AnotB {
   private void scanEarlyStopAsearchB() {
     final long[] scanAArr = skA_.getCache();
     final int arrLongsIn = scanAArr.length;
-    hashArrR_ = new long[arrLongsIn]; //maybe 2x what is needed, but getRetainedEntries can be slow.
+    hashArr_ = new long[arrLongsIn]; //maybe 2x what is needed, but getRetainedEntries can be slow.
     for (int i = 0; i < arrLongsIn; i++ ) {
       final long hashIn = scanAArr[i];
       if (hashIn <= 0L) { continue; }
-      if (hashIn >= thetaLongR_) {
+      if (hashIn >= thetaLong_) {
         break; //early stop assumes that hashes in input sketch are ordered!
       }
       final int foundIdx = hashSearch(bHashTable_, lgArrLongsHT_, hashIn);
       if (foundIdx > -1) { continue; }
-      hashArrR_[curCountR_++] = hashIn;
+      hashArr_[curCount_++] = hashIn;
     }
   }
 
   private void reset() {
     skA_ = null;
     skB_ = null;
-    thetaLongR_ = Long.MAX_VALUE;
-    emptyR_ = true;
-    hashArrR_ = null;
-    curCountR_ = 0;
+    thetaLong_ = Long.MAX_VALUE;
+    empty_ = true;
+    hashArr_ = null;
+    curCount_ = 0;
     lgArrLongsHT_ = 5;
     bHashTable_ = null;
   }
@@ -515,12 +516,12 @@ final class AnotBimpl extends AnotB {
 
   @Override
   short getSeedHash() {
-    return seedHashR_;
+    return seedHash_;
   }
 
   @Override
   long getThetaLong() {
-    return thetaLongR_;
+    return thetaLong_;
   }
 
 }
diff --git a/src/main/java/org/apache/datasketches/theta/CompactOperations.java b/src/main/java/org/apache/datasketches/theta/CompactOperations.java
index 47030fd..6c7067e 100644
--- a/src/main/java/org/apache/datasketches/theta/CompactOperations.java
+++ b/src/main/java/org/apache/datasketches/theta/CompactOperations.java
@@ -46,6 +46,7 @@ import java.util.Arrays;
 
 import org.apache.datasketches.Family;
 import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.SketchesStateException;
 import org.apache.datasketches.memory.Memory;
 import org.apache.datasketches.memory.WritableMemory;
 
@@ -60,35 +61,37 @@ final class CompactOperations {
       final long thetaLong,
       final int curCount,
       final short seedHash,
+      final boolean srcEmpty,
       final boolean srcCompact,
-      boolean srcOrdered,
+      final boolean srcOrdered,
       final boolean dstOrdered,
       final WritableMemory dstMem,
       final long[] hashArr) //may not be compacted, ordered or unordered
-
   {
     final boolean direct = dstMem != null;
-    final boolean empty = (curCount == 0) && (thetaLong == Long.MAX_VALUE);
+    final boolean empty = srcEmpty || ((curCount == 0) && (thetaLong == Long.MAX_VALUE));
     final boolean single = (curCount == 1) && (thetaLong == Long.MAX_VALUE);
     final long[] hashArrOut;
     if (!srcCompact) {
       hashArrOut = CompactOperations.compactCache(hashArr, curCount, thetaLong, dstOrdered);
-      srcOrdered = true;
     } else {
       hashArrOut = hashArr;
     }
     if (!srcOrdered && dstOrdered && !empty && !single) {
       Arrays.sort(hashArrOut);
     }
+    //Note: for empty and single we always output the ordered form.
+    final boolean dstOrderedOut = (empty || single) ? true : dstOrdered;
     if (direct) {
       final int preLongs = computeCompactPreLongs(thetaLong, empty, curCount);
       int flags = READ_ONLY_FLAG_MASK | COMPACT_FLAG_MASK; //always LE
       flags |=  empty ? EMPTY_FLAG_MASK : 0;
-      flags |= dstOrdered ? ORDERED_FLAG_MASK : 0;
+      flags |= dstOrderedOut ? ORDERED_FLAG_MASK : 0;
       flags |= single ? SINGLEITEM_FLAG_MASK : 0;
+
       final Memory mem =
-          loadCompactMemory(hashArr, seedHash, curCount, thetaLong, dstMem, (byte)flags, preLongs);
-      if (dstOrdered) {
+          loadCompactMemory(hashArrOut, seedHash, curCount, thetaLong, dstMem, (byte)flags, preLongs);
+      if (dstOrderedOut) {
         return new DirectCompactOrderedSketch(mem);
       } else {
         return new DirectCompactUnorderedSketch(mem);
@@ -100,7 +103,7 @@ final class CompactOperations {
       if (single) {
         return new SingleItemSketch(hashArrOut[0], seedHash);
       }
-      if (dstOrdered) {
+      if (dstOrderedOut) {
         return new HeapCompactOrderedSketch(hashArrOut, empty, seedHash, curCount, thetaLong);
       } else {
         return new HeapCompactUnorderedSketch(hashArrOut, empty, seedHash, curCount, thetaLong);
@@ -108,45 +111,117 @@ final class CompactOperations {
     }
   }
 
-  @SuppressWarnings("unused")
+  /**
+   * Heapify or convert a source Theta Sketch Memory image into a heap or target Memory CompactSketch.
+   * This assumes hashSeed is OK; serVer = 3.
+   * @param srcMem the given input source Memory image
+   * @param dstOrdered the desired ordering of the resulting CompactSketch
+   * @param dstMem Used for the target CompactSketch if it is Direct.
+   * @return a CompactSketch of the correct form.
+   */
+  @SuppressWarnings("unused") //to replace CompactSketch.anyMemoryToCompactHeap
   static CompactSketch memoryToCompact(
       final Memory srcMem,
       final boolean dstOrdered,
       final WritableMemory dstMem)
   {
-    //extract Pre0 fields
-    final int preLongs = extractPreLongs(srcMem);
-    final int serVer = extractSerVer(srcMem);
-    final int famId = extractFamilyID(srcMem);
-    final int lgArrLongs = extractLgArrLongs(srcMem);
-    final int flags = extractFlags(srcMem);
-    final short seedHash = (short) extractSeedHash(srcMem);
+    //extract Pre0 fields and Flags from srcMem
+    final int srcPreLongs = extractPreLongs(srcMem);
+    final int srcSerVer = extractSerVer(srcMem); //not used
+    final int srcFamId = extractFamilyID(srcMem);
+    final Family srcFamily = Family.idToFamily(srcFamId);
+    final int srcLgArrLongs = extractLgArrLongs(srcMem);
+    final int srcFlags = extractFlags(srcMem);
+    final short srcSeedHash = (short) extractSeedHash(srcMem);
 
+    //srcFlags
+    final boolean srcReadOnlyFlag = (srcFlags & READ_ONLY_FLAG_MASK) > 0;
+    final boolean srcEmptyFlag = (srcFlags & EMPTY_FLAG_MASK) > 0;
+    final boolean srcCompactFlag = (srcFlags & COMPACT_FLAG_MASK) > 0;
+    final boolean srcOrderedFlag = (srcFlags & ORDERED_FLAG_MASK) > 0;
+    //final boolean srcSingleFlag = (srcFlags & SINGLEITEM_FLAG_MASK) > 0;
 
+    final boolean single =
+        SingleItemSketch.otherCheckForSingleItem(srcPreLongs, srcSerVer, srcFamId, srcFlags);
 
-    final int curCount = extractCurCount(srcMem);
-    final long thetaLong = extractThetaLong(srcMem);
+    //extract pre1 and pre2 fields
+    final int curCount = single ? 1 : (srcPreLongs > 1) ? extractCurCount(srcMem) : 0;
+    final long thetaLong = (srcPreLongs > 2) ? extractThetaLong(srcMem) : Long.MAX_VALUE;
 
-    final boolean empty = (flags & EMPTY_FLAG_MASK) > 0;
-    final boolean srcCompact = (flags & COMPACT_FLAG_MASK) > 0;
-    final boolean srcOrdered = (flags & ORDERED_FLAG_MASK) > 0;
-    final boolean single = (flags & SINGLEITEM_FLAG_MASK) > 0;
-    if (!srcOrdered) {
+    //do some basic checks ...
+    if (srcEmptyFlag)  { assert (curCount == 0) && (thetaLong == Long.MAX_VALUE); }
+    if (single) { assert (curCount == 1) && (thetaLong == Long.MAX_VALUE); }
+    checkFamilyAndFlags(srcFamId, srcCompactFlag, srcReadOnlyFlag);
 
+    //dispatch empty and single cases
+    //Note: for empty and single we always output the ordered form.
+    final boolean dstOrderedOut = (srcEmptyFlag || single) ? true : dstOrdered;
+    if (srcEmptyFlag) {
+      if (dstMem != null) {
+        dstMem.putByteArray(0, EmptyCompactSketch.EMPTY_COMPACT_SKETCH_ARR, 0, 8);
+        return new DirectCompactOrderedSketch(dstMem);
+      } else {
+        return EmptyCompactSketch.getInstance();
+      }
+    }
+    if (single) {
+      final long hash = srcMem.getLong(srcPreLongs << 3);
+      final SingleItemSketch sis = new SingleItemSketch(hash, srcSeedHash);
+      if (dstMem != null) {
+        dstMem.putByteArray(0, sis.toByteArray(),0, 16);
+        return new DirectCompactOrderedSketch(dstMem);
+      } else { //heap
+        return sis;
+      }
     }
-    final long[] hashArr ;
 
-    //do checks ...
-    final boolean direct = dstMem != null;
-    if (empty)  { assert (curCount == 0) && (thetaLong == Long.MAX_VALUE); }
-    if (single) { assert (curCount == 1) && (thetaLong == Long.MAX_VALUE); }
-    if (direct) {
+    //extract hashArr > 1
+    final long[] hashArr;
+    if (srcCompactFlag) {
+      hashArr = new long[curCount];
+      srcMem.getLongArray(srcPreLongs << 3, hashArr, 0, curCount);
+    } else { //estimating, thus hashTable form
+      final int srcCacheLen = 1 << srcLgArrLongs;
+      final long[] tempHashArr = new long[srcCacheLen];
+      srcMem.getLongArray(srcPreLongs << 3, tempHashArr, 0, srcCacheLen);
+      hashArr = compactCache(tempHashArr, curCount, thetaLong, dstOrderedOut);
+    }
+
+    //load the destination.
+    if (dstMem != null) {
+      final Memory tgtMem = loadCompactMemory(hashArr, srcSeedHash, curCount, thetaLong, dstMem,
+          (byte)srcFlags, srcPreLongs);
+      if (dstOrderedOut) {
+        return new DirectCompactOrderedSketch(tgtMem);
+      } else {
+        return new DirectCompactUnorderedSketch(tgtMem);
+      }
 
     } else { //heap
-      //dispatch empty and single
-      //dispatch other
+      if (dstOrderedOut) {
+        return new HeapCompactOrderedSketch(hashArr, srcEmptyFlag, srcSeedHash, curCount, thetaLong);
+      } else {
+        return new HeapCompactUnorderedSketch(hashArr, srcEmptyFlag, srcSeedHash, curCount, thetaLong);
+      }
     }
-    return null;
+  }
+
+  private static final void checkFamilyAndFlags(
+      final int srcFamId,
+      final boolean srcCompactFlag,
+      final boolean srcReadOnlyFlag) {
+    final Family srcFamily = Family.idToFamily(srcFamId);
+    if (srcCompactFlag) {
+      if ((srcFamily == Family.COMPACT) && srcReadOnlyFlag) { return; }
+    } else {
+      if (srcFamily == Family.ALPHA) { return; }
+      if (srcFamily == Family.QUICKSELECT) { return; }
+    }
+    throw new SketchesArgumentException(
+        "Possible Corruption: Family does not match flags: Family: "
+            + srcFamily.toString()
+            + ", Compact Flag: " + srcCompactFlag
+            + ", ReadOnly Flag: " + srcReadOnlyFlag);
   }
 
   //All arguments must be valid and correct including flags.
@@ -200,15 +275,16 @@ final class CompactOperations {
   }
 
   /**
-   * Compact the given array. The source cache can be a hash table with interstitial zeros or
-   * "dirty" values, which are hash values greater than theta. These can be generated by the
-   * Alpha sketch.
+   * Copies then compacts, cleans, and may sort the resulting array.
+   * The source cache can be a hash table with interstitial zeros or
+   * "dirty" values, which are hash values greater than theta.
+   * These can be generated by the Alpha sketch.
    * @param srcCache anything
    * @param curCount must be correct
    * @param thetaLong The correct
    * <a href="{@docRoot}/resources/dictionary.html#thetaLong">thetaLong</a>.
    * @param dstOrdered true if output array must be sorted
-   * @return the compacted array
+   * @return the compacted array.
    */
   static final long[] compactCache(final long[] srcCache, final int curCount,
       final long thetaLong, final boolean dstOrdered) {
@@ -223,15 +299,15 @@ final class CompactOperations {
       if ((v <= 0L) || (v >= thetaLong) ) { continue; } //ignoring zeros or dirty values
       cacheOut[j++] = v;
     }
-    assert curCount == j;
+    if (j < curCount) {
+      throw new SketchesStateException(
+          "Possible Corruption: curCount parameter is incorrect.");
+    }
     if (dstOrdered && (curCount > 1)) {
       Arrays.sort(cacheOut);
     }
     return cacheOut;
   }
 
-
-
-
 }
 
diff --git a/src/main/java/org/apache/datasketches/theta/CompactSketch.java b/src/main/java/org/apache/datasketches/theta/CompactSketch.java
index dd667d6..faafd5c 100644
--- a/src/main/java/org/apache/datasketches/theta/CompactSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/CompactSketch.java
@@ -19,19 +19,7 @@
 
 package org.apache.datasketches.theta;
 
-import static org.apache.datasketches.theta.PreambleUtil.COMPACT_FLAG_MASK;
-import static org.apache.datasketches.theta.PreambleUtil.EMPTY_FLAG_MASK;
-import static org.apache.datasketches.theta.PreambleUtil.SINGLEITEM_FLAG_MASK;
-import static org.apache.datasketches.theta.PreambleUtil.extractCurCount;
-import static org.apache.datasketches.theta.PreambleUtil.extractFlags;
-import static org.apache.datasketches.theta.PreambleUtil.extractLgArrLongs;
-import static org.apache.datasketches.theta.PreambleUtil.extractPreLongs;
-import static org.apache.datasketches.theta.PreambleUtil.extractThetaLong;
-
-import java.util.Arrays;
-
 import org.apache.datasketches.Family;
-import org.apache.datasketches.memory.Memory;
 import org.apache.datasketches.memory.WritableMemory;
 
 /**
@@ -67,55 +55,4 @@ public abstract class CompactSketch extends Sketch {
     return true;
   }
 
-  //restricted methods
-
-  /**
-   * Heapifies the given source Memory with seedHash. We assume that the destination sketch type has
-   * been determined to be Compact and that the memory image is valid and the seedHash is correct.
-   * @param srcMem <a href="{@docRoot}/resources/dictionary.html#mem">See Memory</a>
-   * @param seedHash <a href="{@docRoot}/resources/dictionary.html#seedHash">See Seed Hash</a>.
-   * @return a CompactSketch
-   */
-  static CompactSketch You(final Memory srcMem, final short seedHash,
-      final boolean dstOrdered) {
-    final int flags = extractFlags(srcMem);
-    final boolean empty = (flags & EMPTY_FLAG_MASK) > 0;
-    if (empty) { return EmptyCompactSketch.getInstance(); }
-    //EMPTY FLAG is FALSE
-    final int preLongs = extractPreLongs(srcMem);
-    final boolean single = (flags & SINGLEITEM_FLAG_MASK) > 0;
-
-    if (preLongs == 1) {
-      if (single) {
-        return new SingleItemSketch(srcMem.getLong(8), seedHash);
-      } else {
-        return EmptyCompactSketch.getInstance();
-      }
-    }
-    //preLongs > 1
-    final int curCount = extractCurCount(srcMem);
-    final long thetaLong = (preLongs > 2) ? extractThetaLong(srcMem) : Long.MAX_VALUE;
-    final boolean srcCompact = (flags & COMPACT_FLAG_MASK) > 0;
-    final long[] hashArrOut;
-    if (srcCompact) {
-      if ((curCount == 0) && (thetaLong == Long.MAX_VALUE)) {
-        return EmptyCompactSketch.getInstance();
-      }
-      if ((curCount == 1) && (thetaLong == Long.MAX_VALUE)) {
-        //TODO
-      }
-      hashArrOut = new long[curCount];
-      srcMem.getLongArray(8 * preLongs, hashArrOut, 0, curCount);
-      if (dstOrdered) { Arrays.sort(hashArrOut); }
-    } else { //src is hashTable
-      final int lgArrLongs = extractLgArrLongs(srcMem);
-      final long[] hashArr = new long[1 << lgArrLongs];
-      srcMem.getLongArray(8 * preLongs, hashArr, 0, 1 << lgArrLongs);
-      hashArrOut = CompactOperations.compactCache(hashArr, curCount, thetaLong, dstOrdered);
-    }
-    return dstOrdered
-        ? new HeapCompactOrderedSketch(hashArrOut, empty, seedHash, curCount, thetaLong)
-        : new HeapCompactUnorderedSketch(hashArrOut, empty, seedHash, curCount, thetaLong);
-  }
-
 }
diff --git a/src/main/java/org/apache/datasketches/theta/DirectCompactOrderedSketch.java b/src/main/java/org/apache/datasketches/theta/DirectCompactOrderedSketch.java
index 2fc662b..e2c9de5 100644
--- a/src/main/java/org/apache/datasketches/theta/DirectCompactOrderedSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/DirectCompactOrderedSketch.java
@@ -19,15 +19,9 @@
 
 package org.apache.datasketches.theta;
 
-import static org.apache.datasketches.theta.PreambleUtil.COMPACT_FLAG_MASK;
-import static org.apache.datasketches.theta.PreambleUtil.EMPTY_FLAG_MASK;
-import static org.apache.datasketches.theta.PreambleUtil.ORDERED_FLAG_MASK;
-import static org.apache.datasketches.theta.PreambleUtil.READ_ONLY_FLAG_MASK;
-import static org.apache.datasketches.theta.PreambleUtil.SINGLEITEM_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.checkMemorySeedHash;
 
 import org.apache.datasketches.memory.Memory;
-import org.apache.datasketches.memory.WritableMemory;
 
 /**
  * An off-heap (Direct), compact, ordered, read-only sketch.  This sketch can only be associated
@@ -58,46 +52,6 @@ final class DirectCompactOrderedSketch extends DirectCompactSketch {
     return new DirectCompactOrderedSketch(srcMem);
   }
 
-  /**
-   * Constructs this sketch from correct, valid components.
-   * @param cache in compact, ordered form
-   * @param empty The correct <a href="{@docRoot}/resources/dictionary.html#empty">Empty</a>.
-   * @param seedHash The correct
-   * <a href="{@docRoot}/resources/dictionary.html#seedHash">Seed Hash</a>.
-   * @param curCount correct value
-   * @param thetaLong The correct
-   * <a href="{@docRoot}/resources/dictionary.html#thetaLong">thetaLong</a>.
-   * @param dstMem the given destination Memory. This clears it before use.
-   * @return a DirectCompactOrderedSketch
-   */
-  static DirectCompactOrderedSketch compact(
-      final long[] cache,
-      final boolean empty,
-      final short seedHash,
-      final int curCount,
-      final long thetaLong,
-      final WritableMemory dstMem) {
-    final int preLongs = CompactOperations.computeCompactPreLongs(thetaLong, empty, curCount);
-    int flags = READ_ONLY_FLAG_MASK | COMPACT_FLAG_MASK | ORDERED_FLAG_MASK;
-    flags |= empty ? EMPTY_FLAG_MASK : 0;
-    flags |= (curCount == 1) ? SINGLEITEM_FLAG_MASK : 0;
-    CompactOperations.loadCompactMemory(cache, seedHash, curCount, thetaLong, dstMem, (byte)flags, preLongs);
-    return new DirectCompactOrderedSketch(dstMem);
-  }
-
-  @Override //ordered, on-heap
-  public CompactSketch compact() {
-    //TODO
-    return null;
-  }
-
-  @Override
-  public CompactSketch compact(final boolean dstOrdered, final WritableMemory wmem) {
-    //TODO
-    return null;
-  }
-
-
   @Override
   public boolean isOrdered() {
     return true;
diff --git a/src/main/java/org/apache/datasketches/theta/DirectCompactSketch.java b/src/main/java/org/apache/datasketches/theta/DirectCompactSketch.java
index bc62a09..6b65640 100644
--- a/src/main/java/org/apache/datasketches/theta/DirectCompactSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/DirectCompactSketch.java
@@ -23,8 +23,10 @@ import static org.apache.datasketches.theta.PreambleUtil.extractCurCount;
 import static org.apache.datasketches.theta.PreambleUtil.extractPreLongs;
 import static org.apache.datasketches.theta.PreambleUtil.extractSeedHash;
 import static org.apache.datasketches.theta.PreambleUtil.extractThetaLong;
+import static org.apache.datasketches.theta.SingleItemSketch.otherCheckForSingleItem;
 
 import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
 
 /**
  * Parent class of the Direct Compact Sketches.
@@ -40,82 +42,39 @@ abstract class DirectCompactSketch extends CompactSketch {
 
   //Sketch
 
-//  @Override //ordered, on-heap
-//  public CompactSketch compact() {
-//    //TODO
-//    return null;
-//  }
-//
-//  @Override
-//  public CompactSketch compact(final boolean dstOrdered, final WritableMemory wmem) {
-//    final int srcFlags = extractFlags(mem_);
-//    final boolean srcOrdered = (srcFlags & ORDERED_FLAG_MASK) > 0;
-//    final int srcPreLongs = extractPreLongs(mem_);
-//    final int srcCurCount = (srcPreLongs == 1) ? 0 : extractCurCount(mem_);
-//    final long srcThetaLong = (srcPreLongs <= 2) ? Long.MAX_VALUE : extractThetaLong(mem_);
-//    final int bytes = (srcPreLongs + srcCurCount) << 3;
-//    if (srcCurCount == 0) {
-//      if (srcThetaLong == Long.MAX_VALUE) {
-//        //this sets the ordered to true independent of the dstOrdered request
-//        return EmptyCompactSketch.getInstance().compact(true, wmem);
-//      } else {
-//        assert srcPreLongs == 3 : "Theta < 1.0, thus PreLong must be 3: " + srcPreLongs;
-//        mem_.copyTo(0, wmem, 0, srcPreLongs << 3);
-//        if (dstOrdered) {
-//          return new DirectCompactOrderedSketch(wmem);
-//        } else {
-//          return new DirectCompactUnorderedSketch(wmem);
-//        }
-//      }
-//    }
-//    if (srcCurCount == 1) {
-//      if (srcThetaLong == Long.MAX_VALUE) {
-//        //TODO
-//      }
-//    }
-//    if (!srcOrdered && dstOrdered) { //sort this src mem and place in wmem
-//      if (srcCurCount == 0) {
-//        final long thetaLong = extractThetaLong(mem_);
-//        if (thetaLong == Long.MAX_VALUE) {
-//          //TODO
-//        }
-//      } else {
-//        final byte[] srcBytes = new byte[bytes];
-//        mem_.getByteArray(0, srcBytes, 0, bytes);
-//        wmem.putByteArray(0, srcBytes, 0, bytes);
-//        final byte dstFlags = (byte) (srcFlags & ORDERED_FLAG_MASK);
-//        wmem.putByte(FLAGS_BYTE, dstFlags);
-//      }
-//
-//    } else {
-//      mem_.copyTo(0, wmem, 0, bytes);
-//    }
-//
-//    return null;  //TODO
-//  }
-
-  //overidden by EmptyCompactSketch and SingleItemSketch
+  @Override
+  public CompactSketch compact() {
+    return compact(true, null);
+  }
+
+  @Override
+  public CompactSketch compact(final boolean dstOrdered, final WritableMemory dstMem) {
+    return CompactOperations.memoryToCompact(mem_, dstOrdered, dstMem);
+  }
+
   @Override
   public int getCurrentBytes(final boolean compact) { //compact is ignored here
-    final int preLongs = getCurrentPreambleLongs(true);
-    //preLongs > 1
-    final int curCount = extractCurCount(mem_);
+    if (otherCheckForSingleItem(mem_)) { return 16; }
+    final int preLongs = extractPreLongs(mem_);
+    final int curCount = (preLongs == 1) ? 0 : extractCurCount(mem_);
     return (preLongs + curCount) << 3;
   }
 
   @Override
   public double getEstimate() {
-    final int curCount = extractCurCount(mem_);
+    if (otherCheckForSingleItem(mem_)) { return 1; }
     final int preLongs = extractPreLongs(mem_);
+    final int curCount = (preLongs == 1) ? 0 : extractCurCount(mem_);
     final long thetaLong = (preLongs > 2) ? extractThetaLong(mem_) : Long.MAX_VALUE;
     return Sketch.estimate(thetaLong, curCount);
   }
 
-  //overidden by EmptyCompactSketch and SingleItemSketch
   @Override
   public int getRetainedEntries(final boolean valid) { //compact is always valid
-    //preLongs > 1
-    return extractCurCount(mem_);
+    if (otherCheckForSingleItem(mem_)) { return 1; }
+    final int preLongs = extractPreLongs(mem_);
+    final int curCount = (preLongs == 1) ? 0 : extractCurCount(mem_);
+    return curCount;
   }
 
   @Override
@@ -136,7 +95,10 @@ abstract class DirectCompactSketch extends CompactSketch {
 
   @Override
   public boolean isEmpty() {
-    return PreambleUtil.isEmptySketch(mem_);
+    final boolean emptyFlag = PreambleUtil.isEmptyFlag(mem_);
+    final long thetaLong = getThetaLong();
+    final int curCount = getRetainedEntries(true);
+    return emptyFlag || ((curCount == 0) && (thetaLong == Long.MAX_VALUE));
   }
 
   @Override
@@ -149,28 +111,26 @@ abstract class DirectCompactSketch extends CompactSketch {
     return new MemoryHashIterator(mem_, getRetainedEntries(), getThetaLong());
   }
 
-  @Override //order is already determined.
+  @Override
   public byte[] toByteArray() {
     final int curCount = getRetainedEntries(true);
     Sketch.checkIllegalCurCountAndEmpty(isEmpty(), curCount);
-    final int preLongs = getCurrentPreambleLongs(true);
+    final int preLongs = extractPreLongs(mem_);
     final int outBytes = (curCount + preLongs) << 3;
     final byte[] byteArrOut = new byte[outBytes];
-    mem_.getByteArray(0, byteArrOut, 0, outBytes); //copies the whole thing
+    mem_.getByteArray(0, byteArrOut, 0, outBytes);
     return byteArrOut;
   }
 
   //restricted methods
 
-
-
-
   @Override
   long[] getCache() {
-    final int curCount = getRetainedEntries(true);
+    if (otherCheckForSingleItem(mem_)) { return new long[] { mem_.getLong(8) }; }
+    final int preLongs = extractPreLongs(mem_);
+    final int curCount = (preLongs == 1) ? 0 : extractCurCount(mem_);
     if (curCount > 0) {
       final long[] cache = new long[curCount];
-      final int preLongs = getCurrentPreambleLongs(true);
       mem_.getLongArray(preLongs << 3, cache, 0, curCount);
       return cache;
     }
diff --git a/src/main/java/org/apache/datasketches/theta/DirectCompactUnorderedSketch.java b/src/main/java/org/apache/datasketches/theta/DirectCompactUnorderedSketch.java
index f8743ef..15b4433 100644
--- a/src/main/java/org/apache/datasketches/theta/DirectCompactUnorderedSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/DirectCompactUnorderedSketch.java
@@ -19,14 +19,9 @@
 
 package org.apache.datasketches.theta;
 
-import static org.apache.datasketches.theta.PreambleUtil.COMPACT_FLAG_MASK;
-import static org.apache.datasketches.theta.PreambleUtil.EMPTY_FLAG_MASK;
-import static org.apache.datasketches.theta.PreambleUtil.READ_ONLY_FLAG_MASK;
-import static org.apache.datasketches.theta.PreambleUtil.SINGLEITEM_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.checkMemorySeedHash;
 
 import org.apache.datasketches.memory.Memory;
-import org.apache.datasketches.memory.WritableMemory;
 
 /**
  * An off-heap (Direct), compact, unordered, read-only sketch.  This sketch can only be associated
@@ -57,45 +52,6 @@ final class DirectCompactUnorderedSketch extends DirectCompactSketch {
     return new DirectCompactUnorderedSketch(srcMem);
   }
 
-  /**
-   * Constructs this sketch from correct, valid components.
-   * @param cache in compact, ordered form
-   * @param empty The correct <a href="{@docRoot}/resources/dictionary.html#empty">Empty</a>.
-   * @param seedHash The correct
-   * <a href="{@docRoot}/resources/dictionary.html#seedHash">Seed Hash</a>.
-   * @param curCount correct value
-   * @param thetaLong The correct
-   * <a href="{@docRoot}/resources/dictionary.html#thetaLong">thetaLong</a>.
-   * @param dstMem the given destination Memory. This clears it before use.
-   * @return a DirectCompactUnorderedSketch
-   */
-  static DirectCompactUnorderedSketch compact(
-      final long[] cache,
-      final boolean empty,
-      final short seedHash,
-      final int curCount,
-      final long thetaLong,
-      final WritableMemory dstMem) {
-    final int preLongs = CompactOperations.computeCompactPreLongs(thetaLong, empty, curCount);
-    int flags = READ_ONLY_FLAG_MASK | COMPACT_FLAG_MASK;
-    flags |= empty ? EMPTY_FLAG_MASK : 0;
-    flags |= (curCount == 1) ? SINGLEITEM_FLAG_MASK : 0;
-    CompactOperations.loadCompactMemory(cache, seedHash, curCount, thetaLong, dstMem, (byte)flags, preLongs);
-    return new DirectCompactUnorderedSketch(dstMem);
-  }
-
-  @Override //ordered, on-heap
-  public CompactSketch compact() {
-    //TODO
-    return null;
-  }
-
-  @Override
-  public CompactSketch compact(final boolean dstOrdered, final WritableMemory wmem) {
-    //TODO
-    return null;
-  }
-
   @Override
   public boolean isOrdered() {
     return false;
diff --git a/src/main/java/org/apache/datasketches/theta/DirectQuickSelectSketchR.java b/src/main/java/org/apache/datasketches/theta/DirectQuickSelectSketchR.java
index 7900451..97eb82a 100644
--- a/src/main/java/org/apache/datasketches/theta/DirectQuickSelectSketchR.java
+++ b/src/main/java/org/apache/datasketches/theta/DirectQuickSelectSketchR.java
@@ -155,7 +155,7 @@ class DirectQuickSelectSketchR extends UpdateSketch {
 
   @Override
   public boolean isEmpty() {
-    return PreambleUtil.isEmptySketch(wmem_);
+    return PreambleUtil.isEmptyFlag(wmem_);
   }
 
   @Override
diff --git a/src/main/java/org/apache/datasketches/theta/HeapAlphaSketch.java b/src/main/java/org/apache/datasketches/theta/HeapAlphaSketch.java
index 32a73c4..d41beb4 100644
--- a/src/main/java/org/apache/datasketches/theta/HeapAlphaSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/HeapAlphaSketch.java
@@ -146,7 +146,7 @@ final class HeapAlphaSketch extends HeapUpdateSketch {
     has.hashTableThreshold_ = setHashTableThreshold(lgNomLongs, lgArrLongs);
     has.curCount_ = extractCurCount(srcMem);
     has.thetaLong_ = extractThetaLong(srcMem);
-    has.empty_ = PreambleUtil.isEmptySketch(srcMem);
+    has.empty_ = PreambleUtil.isEmptyFlag(srcMem);
     has.cache_ = new long[1 << lgArrLongs];
     srcMem.getLongArray(preambleLongs << 3, has.cache_, 0, 1 << lgArrLongs); //read in as hash table
     return has;
diff --git a/src/main/java/org/apache/datasketches/theta/HeapCompactOrderedSketch.java b/src/main/java/org/apache/datasketches/theta/HeapCompactOrderedSketch.java
index 48c38fb..dbaf823 100644
--- a/src/main/java/org/apache/datasketches/theta/HeapCompactOrderedSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/HeapCompactOrderedSketch.java
@@ -45,16 +45,16 @@ final class HeapCompactOrderedSketch extends HeapCompactSketch {
 
   //Sketch interface
 
-  @Override //ordered, on-heap
+  @Override
   public CompactSketch compact() {
-    //TODO
-    return null;
+    return this;
   }
 
   @Override
-  public CompactSketch compact(final boolean dstOrdered, final WritableMemory wmem) {
-    //TODO
-    return null;
+  public CompactSketch compact(final boolean dstOrdered, final WritableMemory dstMem) {
+    return CompactOperations.componentsToCompact(
+    getThetaLong(), getRetainedEntries(), getSeedHash(), isEmpty(),
+    true, true, dstOrdered, dstMem, getCache());
   }
 
   @Override
diff --git a/src/main/java/org/apache/datasketches/theta/HeapCompactSketch.java b/src/main/java/org/apache/datasketches/theta/HeapCompactSketch.java
index b78002d..74dff7d 100644
--- a/src/main/java/org/apache/datasketches/theta/HeapCompactSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/HeapCompactSketch.java
@@ -124,6 +124,7 @@ abstract class HeapCompactSketch extends CompactSketch {
     return seedHash_;
   }
 
+  //only called from sub classes
   byte[] toByteArray(final boolean ordered) {
     Sketch.checkIllegalCurCountAndEmpty(empty_, curCount_);
     final int bytes = getCurrentBytes(true);
diff --git a/src/main/java/org/apache/datasketches/theta/HeapCompactUnorderedSketch.java b/src/main/java/org/apache/datasketches/theta/HeapCompactUnorderedSketch.java
index 4fc35c3..6b59a02 100644
--- a/src/main/java/org/apache/datasketches/theta/HeapCompactUnorderedSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/HeapCompactUnorderedSketch.java
@@ -19,12 +19,6 @@
 
 package org.apache.datasketches.theta;
 
-import static org.apache.datasketches.theta.PreambleUtil.checkMemorySeedHash;
-import static org.apache.datasketches.theta.PreambleUtil.extractCurCount;
-import static org.apache.datasketches.theta.PreambleUtil.extractPreLongs;
-import static org.apache.datasketches.theta.PreambleUtil.extractThetaLong;
-
-import org.apache.datasketches.memory.Memory;
 import org.apache.datasketches.memory.WritableMemory;
 
 /**
@@ -49,41 +43,18 @@ final class HeapCompactUnorderedSketch extends HeapCompactSketch {
     super(cache, empty, seedHash, curCount, thetaLong);
   }
 
-  /**
-   * Heapifies the given source Memory with seed
-   * @param srcMem <a href="{@docRoot}/resources/dictionary.html#mem">See Memory</a>
-   * @param seed <a href="{@docRoot}/resources/dictionary.html#seed">See Update Hash Seed</a>.
-   * @return a CompactSketch
-   */
-  //Note Empty and SingleItemSketches should be filtered out before we get here.
-  static CompactSketch heapifyInstance(final Memory srcMem, final long seed) {
-    final short memSeedHash = checkMemorySeedHash(srcMem, seed);
-    final int preLongs = extractPreLongs(srcMem);
-    final boolean empty = PreambleUtil.isEmptySketch(srcMem);
-    long thetaLong = Long.MAX_VALUE;
-    final int curCount = extractCurCount(srcMem);
-    final long[] cache = new long[curCount];
-    if (preLongs == 2) {
-      srcMem.getLongArray(16, cache, 0, curCount);
-    } else { //preLongs == 3
-      srcMem.getLongArray(24, cache, 0, curCount);
-      thetaLong = extractThetaLong(srcMem);
-    }
-    return new HeapCompactUnorderedSketch(cache, empty, memSeedHash, curCount, thetaLong);
-  }
-
   //Sketch interface
 
   @Override //ordered, on-heap
   public CompactSketch compact() {
-    //TODO
-    return null;
+    return compact(true, null);
   }
 
   @Override
-  public CompactSketch compact(final boolean dstOrdered, final WritableMemory wmem) {
-    //TODO
-    return null;
+  public CompactSketch compact(final boolean dstOrdered, final WritableMemory dstMem) {
+    return CompactOperations.componentsToCompact(
+      getThetaLong(), getRetainedEntries(), getSeedHash(), isEmpty(),
+      true, false, dstOrdered, dstMem, getCache());
   }
 
   @Override
diff --git a/src/main/java/org/apache/datasketches/theta/HeapQuickSelectSketch.java b/src/main/java/org/apache/datasketches/theta/HeapQuickSelectSketch.java
index 7fe3e9e..1090d9e 100644
--- a/src/main/java/org/apache/datasketches/theta/HeapQuickSelectSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/HeapQuickSelectSketch.java
@@ -133,7 +133,7 @@ class HeapQuickSelectSketch extends HeapUpdateSketch {
     hqss.hashTableThreshold_ = setHashTableThreshold(lgNomLongs, lgArrLongs);
     hqss.curCount_ = extractCurCount(srcMem);
     hqss.thetaLong_ = extractThetaLong(srcMem);
-    hqss.empty_ = PreambleUtil.isEmptySketch(srcMem);
+    hqss.empty_ = PreambleUtil.isEmptyFlag(srcMem);
     hqss.cache_ = new long[1 << lgArrLongs];
     srcMem.getLongArray(preambleLongs << 3, hqss.cache_, 0, 1 << lgArrLongs); //read in as hash table
     return hqss;
diff --git a/src/main/java/org/apache/datasketches/theta/IntersectionImplR.java b/src/main/java/org/apache/datasketches/theta/IntersectionImplR.java
index 0cc7517..6b79a24 100644
--- a/src/main/java/org/apache/datasketches/theta/IntersectionImplR.java
+++ b/src/main/java/org/apache/datasketches/theta/IntersectionImplR.java
@@ -159,8 +159,8 @@ class IntersectionImplR extends Intersection {
 
     if (curCount_ == 0) {
       compactCacheR = new long[0];
-      return createCompactSketch(
-          compactCacheR, empty_, seedHash_, curCount_, thetaLong_, dstOrdered, dstMem);
+      return CompactOperations.componentsToCompact(
+          thetaLong_, curCount_, seedHash_, empty_, true, false, dstOrdered, dstMem, compactCacheR);
     }
     //else curCount > 0
     final long[] hashTable;
@@ -174,8 +174,8 @@ class IntersectionImplR extends Intersection {
     compactCacheR = compactCachePart(hashTable, lgArrLongs_, curCount_, thetaLong_, dstOrdered);
 
     //Create the CompactSketch
-    return createCompactSketch(
-        compactCacheR, empty_, seedHash_, curCount_, thetaLong_, dstOrdered, dstMem);
+    return CompactOperations.componentsToCompact(
+        thetaLong_, curCount_, seedHash_, empty_, true, dstOrdered, dstOrdered, dstMem, compactCacheR);
   }
 
   @Override
diff --git a/src/main/java/org/apache/datasketches/theta/PairwiseSetOperations.java b/src/main/java/org/apache/datasketches/theta/PairwiseSetOperations.java
index a73e7eb..3ffa6ac 100644
--- a/src/main/java/org/apache/datasketches/theta/PairwiseSetOperations.java
+++ b/src/main/java/org/apache/datasketches/theta/PairwiseSetOperations.java
@@ -19,12 +19,14 @@
 
 package org.apache.datasketches.theta;
 
-import static org.apache.datasketches.theta.SetOperation.createCompactSketch;
+import static org.apache.datasketches.Util.DEFAULT_NOMINAL_ENTRIES;
+import static org.apache.datasketches.Util.DEFAULT_UPDATE_SEED;
+import static org.apache.datasketches.Util.checkSeedHashes;
+import static org.apache.datasketches.Util.computeSeedHash;
 
 import java.util.Arrays;
 
 import org.apache.datasketches.SketchesArgumentException;
-import org.apache.datasketches.Util;
 
 /**
  * Set Operations where the arguments are presented in pairs as in <i>C = Op(A,B)</i>. These are
@@ -101,7 +103,7 @@ public class PairwiseSetOperations {
    */
   @Deprecated
   public static CompactSketch union(final CompactSketch skA, final CompactSketch skB) {
-    return union(skA, skB, Util.DEFAULT_NOMINAL_ENTRIES);
+    return union(skA, skB, DEFAULT_NOMINAL_ENTRIES);
   }
 
   /**
@@ -216,7 +218,10 @@ public class PairwiseSetOperations {
     } else {
       outArr = Arrays.copyOf(outCache, curCount); //copy only valid items
     }
-    return createCompactSketch(outArr, false, skA.getSeedHash(), curCount, thetaLong, true, null);
+    final short seedHash = computeSeedHash(DEFAULT_UPDATE_SEED);
+    final boolean srcEmpty = (curCount == 0) && (thetaLong == Long.MAX_VALUE);
+    return CompactOperations.componentsToCompact(
+        thetaLong, curCount, seedHash, srcEmpty, true, true, true, null, outArr);
   }
 
   private static CompactSketch maybeCutback(final CompactSketch csk, final int k) {
@@ -228,7 +233,9 @@ public class PairwiseSetOperations {
       thetaLong = cache[k];
       final long[] arr = Arrays.copyOf(cache, k);
       curCount = k;
-      return createCompactSketch(arr, empty, csk.getSeedHash(), curCount, thetaLong, true, null);
+      final short seedHash = computeSeedHash(DEFAULT_UPDATE_SEED);
+      return CompactOperations.componentsToCompact(
+          thetaLong, curCount, seedHash, empty, true, false, true, null, cache);
     }
     return csk;
   }
@@ -242,7 +249,7 @@ public class PairwiseSetOperations {
   private static short seedHashesCheck(final Sketch skA, final Sketch skB) {
     final short seedHashA = skA.getSeedHash(); //lgtm [java/dereferenced-value-may-be-null]
     final short seedHashB = skB.getSeedHash(); //lgtm [java/dereferenced-value-may-be-null]
-    return Util.checkSeedHashes(seedHashA, seedHashB);
+    return checkSeedHashes(seedHashA, seedHashB);
   }
 
 }
diff --git a/src/main/java/org/apache/datasketches/theta/PreambleUtil.java b/src/main/java/org/apache/datasketches/theta/PreambleUtil.java
index a61e16c..8baf634 100644
--- a/src/main/java/org/apache/datasketches/theta/PreambleUtil.java
+++ b/src/main/java/org/apache/datasketches/theta/PreambleUtil.java
@@ -466,77 +466,6 @@ final class PreambleUtil {
     return ((extractFlags(mem) & EMPTY_FLAG_MASK) > 0);
   }
 
-  private static final int ALPHA_ID = Family.ALPHA.getID();
-  private static final int QUICKSELECT_ID = Family.QUICKSELECT.getID();
-  private static final int COMPACT_ID = Family.COMPACT.getID();
-  private static final int UNION_ID = Family.UNION.getID();
-
-  static boolean isEmptySketch(final Memory mem) {
-    //CHECK FamID
-    final int famId = extractFamilyID(mem);
-    if ( !((famId == ALPHA_ID) || (famId == QUICKSELECT_ID)
-        || (famId == COMPACT_ID) || (famId == UNION_ID))) {
-      throw new SketchesArgumentException("Not part of the Theta Sketch Family");
-    }
-    //CHECK PreLongs
-    final int preLongs = extractPreLongs(mem);
-    if ((preLongs < 1) || (preLongs > 4)) {
-      throw new SketchesArgumentException("Possible corruption: Illegal preLongs value: " + preLongs);
-    }
-    //CHECK SerVer
-    final int serVer = extractSerVer(mem);
-    if ((serVer < 1) || (serVer > 3)) {
-      throw new SketchesArgumentException("Possible corruption: Illegal serVer value: " + serVer);
-    }
-    if (serVer == 1) { //Prelongs is always 3, no empty flag
-      assert preLongs == 3;
-      return ((extractCurCount(mem) == 0)  && (extractThetaLong(mem) == Long.MAX_VALUE));
-    }
-    // Flags byte: SI, Ordered, Compact, Empty, ReadOnly, LittleEndian = 0XX1X0
-    // Flags mask = 100101 = 0x25; Flags compare = 000100 = 0x4
-    final int flags = extractFlags(mem);
-    final boolean emptyFlag = (flags & 0x25) == EMPTY_FLAG_MASK;
-    if (serVer == 2) {
-      if (preLongs == 1) { return true; }
-      final int curCount = extractCurCount(mem);
-      if (preLongs == 2) {
-        return emptyFlag || (curCount == 0);
-      }
-      final long thetaLong = extractThetaLong(mem);
-      if (preLongs == 3) {
-        return emptyFlag || ((curCount == 0) && (thetaLong == Long.MAX_VALUE));
-      }
-    }
-    if (serVer == 3) {
-      final boolean emptyCap = mem.getCapacity() < 16L;
-      if (preLongs == 1) { return emptyFlag || emptyCap; }
-      final int curCount = extractCurCount(mem);
-      if (preLongs == 2) {
-        return emptyFlag || (curCount == 0);
-      }
-      final long thetaLong = extractThetaLong(mem);
-      if (preLongs <= 4) {
-        return emptyFlag || ((curCount == 0) && (thetaLong == Long.MAX_VALUE));
-      }
-    }
-    assert false : "Should not get here";
-    return true;
-  }
-
-  static boolean isSingleItemSketch(final Memory mem) {
-    // Flags byte: SI, Ordered, Compact, NotEmpty, ReadOnly, LittleEndian = X11010 = 0x1A.
-    // Flags mask will be 0x1F.
-    // SingleItem flag may not be set due to a historical bug, so we can't depend on it for now.
-    // However, if the above flags are correct, preLongs == 1, SerVer >= 3, FamilyID == 3,
-    // and the hash seed matches (not done here), it is virtually guaranteed that we have a
-    // SingleItem Sketch.
-    final boolean preLongs = extractPreLongs(mem) == 1;
-    final boolean serVer = extractSerVer(mem) >= 3;
-    final boolean famId = extractFamilyID(mem) == Family.COMPACT.getID();
-    final boolean flags =  (extractFlags(mem) & 0x1F) == 0x1A; //no SI, yet
-    return preLongs && serVer && famId && flags;
-  }
-
   /**
    * Checks Memory for capacity to hold the preamble and returns the extracted preLongs.
    * @param mem the given Memory
diff --git a/src/main/java/org/apache/datasketches/theta/SetOperation.java b/src/main/java/org/apache/datasketches/theta/SetOperation.java
index b4fc4cf..89eed19 100644
--- a/src/main/java/org/apache/datasketches/theta/SetOperation.java
+++ b/src/main/java/org/apache/datasketches/theta/SetOperation.java
@@ -27,7 +27,6 @@ import static org.apache.datasketches.Util.REBUILD_THRESHOLD;
 import static org.apache.datasketches.Util.ceilingPowerOf2;
 import static org.apache.datasketches.theta.PreambleUtil.FAMILY_BYTE;
 import static org.apache.datasketches.theta.PreambleUtil.SER_VER_BYTE;
-import static org.apache.datasketches.theta.Sketch.checkIllegalCurCountAndEmpty;
 
 import org.apache.datasketches.Family;
 import org.apache.datasketches.SketchesArgumentException;
@@ -75,11 +74,11 @@ public abstract class SetOperation {
    * <a href="{@docRoot}/resources/dictionary.html#mem">See Memory</a>
    * @param seed <a href="{@docRoot}/resources/dictionary.html#seed">See Update Hash Seed</a>.
    * @return a Heap-based SetOperation from the given Memory
-   */ //TODO Do we need to add a stateful AnotB here?
+   */
   public static SetOperation heapify(final Memory srcMem, final long seed) {
     final byte famID = srcMem.getByte(FAMILY_BYTE);
     final Family family = idToFamily(famID);
-    switch (family) {
+    switch (family) { //TODO Do we need to add the stateful AnotB ?
       case UNION : {
         return UnionImpl.heapifyInstance(srcMem, seed);
       }
@@ -115,7 +114,7 @@ public abstract class SetOperation {
    * <a href="{@docRoot}/resources/dictionary.html#mem">See Memory</a>
    * @param seed <a href="{@docRoot}/resources/dictionary.html#seed">See Update Hash Seed</a>.
    * @return a SetOperation backed by the given Memory
-   */ //TODO Do we need to add a stateful AnotB here?
+   */
   public static SetOperation wrap(final Memory srcMem, final long seed) {
     final byte famID = srcMem.getByte(FAMILY_BYTE);
     final Family family = idToFamily(famID);
@@ -123,7 +122,7 @@ public abstract class SetOperation {
     if (serVer != 3) {
       throw new SketchesArgumentException("SerVer must be 3: " + serVer);
     }
-    switch (family) {
+    switch (family) { //TODO Do we need to add the stateful AnotB ?
       case UNION : {
         return UnionImpl.wrapInstance(srcMem, seed);
       }
@@ -247,49 +246,6 @@ public abstract class SetOperation {
   //intentionally not made public because behavior will be confusing to end user.
   abstract boolean isEmpty();
 
-  //used only by the set operations
-  static final CompactSketch createCompactSketch(
-      final long[] compactCache,
-      boolean empty,
-      final short seedHash,
-      final int curCount,
-      final long thetaLong,
-      final boolean dstOrdered,
-      final WritableMemory dstMem) {
-    checkIllegalCurCountAndEmpty(empty, curCount);
-    empty = correctEmptyOnSetResult(curCount, thetaLong);
-    if (empty) {
-      final EmptyCompactSketch sk = EmptyCompactSketch.getInstance();
-      if (dstMem != null) {
-        dstMem.putByteArray(0, sk.toByteArray(), 0, 8);
-      }
-      return sk;
-    }
-    //Not Empty
-    if ((thetaLong == Long.MAX_VALUE) && (curCount == 1)) {
-      final SingleItemSketch sis = new SingleItemSketch(compactCache[0], seedHash);
-      if ((dstMem != null) && (dstMem.getCapacity() >= 16)) {
-        dstMem.putByteArray(0, sis.toByteArray(), 0, 16);
-      }
-      return sis;
-    }
-    if (dstMem == null) {
-      if (dstOrdered) {
-        return new HeapCompactOrderedSketch(compactCache, empty, seedHash, curCount, thetaLong);
-      } else {
-        return new HeapCompactUnorderedSketch(compactCache, empty, seedHash, curCount, thetaLong);
-      }
-    } else {
-      if (dstOrdered) {
-        return DirectCompactOrderedSketch.compact(compactCache, empty, seedHash, curCount,
-            thetaLong, dstMem);
-      } else {
-        return DirectCompactUnorderedSketch.compact(compactCache, empty, seedHash, curCount,
-            thetaLong, dstMem);
-      }
-    }
-  }
-
   /**
    * Computes minimum lgArrLongs from a current count.
    * @param count the given current count
diff --git a/src/main/java/org/apache/datasketches/theta/SetOperationBuilder.java b/src/main/java/org/apache/datasketches/theta/SetOperationBuilder.java
index 822d88f..b78ac3b 100644
--- a/src/main/java/org/apache/datasketches/theta/SetOperationBuilder.java
+++ b/src/main/java/org/apache/datasketches/theta/SetOperationBuilder.java
@@ -210,7 +210,7 @@ public class SetOperationBuilder {
           setOp = new AnotBimpl(bSeed);
         }
         else {
-          throw new SketchesArgumentException(
+          throw new SketchesArgumentException( //TODO we should be able to do this now.
             "AnotB is only on heap and cannot be persisted.");
         }
         break;
diff --git a/src/main/java/org/apache/datasketches/theta/SingleItemSketch.java b/src/main/java/org/apache/datasketches/theta/SingleItemSketch.java
index 38bf862..5356dbc 100644
--- a/src/main/java/org/apache/datasketches/theta/SingleItemSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/SingleItemSketch.java
@@ -24,9 +24,14 @@ import static org.apache.datasketches.ByteArrayUtil.putLongLE;
 import static org.apache.datasketches.Util.DEFAULT_UPDATE_SEED;
 import static org.apache.datasketches.Util.computeSeedHash;
 import static org.apache.datasketches.hash.MurmurHash3.hash;
+import static org.apache.datasketches.theta.PreambleUtil.SINGLEITEM_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.checkMemorySeedHash;
-import static org.apache.datasketches.theta.PreambleUtil.isSingleItemSketch;
+import static org.apache.datasketches.theta.PreambleUtil.extractFamilyID;
+import static org.apache.datasketches.theta.PreambleUtil.extractFlags;
+import static org.apache.datasketches.theta.PreambleUtil.extractPreLongs;
+import static org.apache.datasketches.theta.PreambleUtil.extractSerVer;
 
+import org.apache.datasketches.Family;
 import org.apache.datasketches.SketchesArgumentException;
 import org.apache.datasketches.memory.Memory;
 import org.apache.datasketches.memory.WritableMemory;
@@ -89,24 +94,24 @@ final class SingleItemSketch extends CompactSketch {
    */ //does not override Sketch
   public static SingleItemSketch heapify(final Memory srcMem, final long seed) {
     final short seedHashMem = checkMemorySeedHash(srcMem, seed);
-    if (isSingleItemSketch(srcMem)) {
-      return new SingleItemSketch(srcMem.getLong(8), seedHashMem);
-    }
-    throw new SketchesArgumentException("Input Memory Preamble is not a SingleItemSketch.");
+    final boolean singleItem = otherCheckForSingleItem(srcMem);
+    if (singleItem) { return new SingleItemSketch(srcMem.getLong(8), seedHashMem); }
+    throw new SketchesArgumentException("Input Memory is not a SingleItemSketch.");
   }
 
   @Override
   public CompactSketch compact() {
-    final long[] hashArr = getCache();
-    final short seedHash = getSeedHash();
-    return new HeapCompactOrderedSketch(hashArr, false, seedHash, 1, Long.MAX_VALUE);
+    return this;
   }
 
   @Override
   public CompactSketch compact(final boolean dstOrdered, final WritableMemory dstMem) {
-    dstMem.putLong(0, pre0_);
-    dstMem.putLong(8, hash_);
-    return new DirectCompactOrderedSketch(dstMem);
+    if (dstMem == null) { return this; }
+    else {
+      dstMem.putLong(0, pre0_);
+      dstMem.putLong(8, hash_);
+      return new DirectCompactOrderedSketch(dstMem);
+    }
   }
 
   //Create methods using the default seed
@@ -404,4 +409,25 @@ final class SingleItemSketch extends CompactSketch {
     return (short) (pre0_ >>> 48);
   }
 
+  static final boolean otherCheckForSingleItem(final Memory mem) {
+    return otherCheckForSingleItem(extractPreLongs(mem), extractSerVer(mem),
+        extractFamilyID(mem), extractFlags(mem) );
+  }
+
+  static final boolean otherCheckForSingleItem(final int preLongs, final int serVer,
+      final int famId, final int flags) {
+    // Flags byte: SI=X, Ordered=T, Compact=T, Empty=F, ReadOnly=T, BigEndian=F = X11010 = 0x1A.
+    // Flags mask will be 0x1F.
+    // SingleItem flag may not be set due to a historical bug, so we can't depend on it for now.
+    // However, if the above flags are correct, preLongs == 1, SerVer >= 3, FamilyID == 3,
+    // and the hash seed matches (not done here), it is virtually guaranteed that we have a
+    // SingleItem Sketch.
+    final boolean numPreLongs = preLongs == 1;
+    final boolean numSerVer = serVer >= 3;
+    final boolean numFamId = famId == Family.COMPACT.getID();
+    final boolean numFlags =  (flags & 0x1F) == 0x1A; //no SI, yet
+    final boolean singleFlag = (flags & SINGLEITEM_FLAG_MASK) > 0;
+    return (numPreLongs && numSerVer && numFamId && numFlags) || singleFlag;
+  }
+
 }
diff --git a/src/main/java/org/apache/datasketches/theta/Sketch.java b/src/main/java/org/apache/datasketches/theta/Sketch.java
index c54d521..42f4724 100644
--- a/src/main/java/org/apache/datasketches/theta/Sketch.java
+++ b/src/main/java/org/apache/datasketches/theta/Sketch.java
@@ -27,13 +27,14 @@ import static org.apache.datasketches.Util.LS;
 import static org.apache.datasketches.Util.ceilingPowerOf2;
 import static org.apache.datasketches.Util.zeroPad;
 import static org.apache.datasketches.theta.PreambleUtil.COMPACT_FLAG_MASK;
+import static org.apache.datasketches.theta.PreambleUtil.EMPTY_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.FAMILY_BYTE;
 import static org.apache.datasketches.theta.PreambleUtil.FLAGS_BYTE;
 import static org.apache.datasketches.theta.PreambleUtil.ORDERED_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.PREAMBLE_LONGS_BYTE;
 import static org.apache.datasketches.theta.PreambleUtil.READ_ONLY_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.SER_VER_BYTE;
-import static org.apache.datasketches.theta.PreambleUtil.isSingleItemSketch;
+import static org.apache.datasketches.theta.SingleItemSketch.otherCheckForSingleItem;
 
 import org.apache.datasketches.BinomialBoundsN;
 import org.apache.datasketches.Family;
@@ -142,12 +143,12 @@ public abstract class Sketch {
               "Corrupted: " + family + " family image: must have SerVer = 3 and preLongs = 3");
         }
       }
-      case COMPACT: { //serVer 1, 2, or 3, preLongs = 1, 2, or 3
+      case COMPACT: { //serVer 1, 2, 3; preLongs = 1, 2, or 3
         if (serVer == 3) {
-          if (PreambleUtil.isEmptySketch(srcMem)) {
+          if (PreambleUtil.isEmptyFlag(srcMem)) {
             return EmptyCompactSketch.getInstance(srcMem);
           }
-          if (isSingleItemSketch(srcMem)) { //SINGLEITEM?
+          if (otherCheckForSingleItem(srcMem)) { //SINGLEITEM?
             return SingleItemSketch.heapify(srcMem, seed);
           }
           //not empty & not singleItem
@@ -184,26 +185,33 @@ public abstract class Sketch {
   //Sketch interface
 
   /**
-   * Converts this sketch as an ordered CompactSketch on the Java heap.
+   * Converts this sketch to a ordered CompactSketch on the Java heap.
    *
-   * <p>If this sketch is already in compact form this operation returns <i>this</i>.
+   * <p>If this sketch is already in the proper form, this method returns <i>this</i>,
+   * otherwise, this method returns a new CompactSketch of the proper form.
+   *
+   * <p>A CompactSketch is always immutable.</p>
    *
    * @return this sketch as an ordered CompactSketch on the Java heap.
    */
   public abstract CompactSketch compact();
 
   /**
-   * Convert this sketch to a CompactSketch in the chosen form.
+   * Convert this sketch to a new CompactSketch of the chosen order and direct or on the heap.
+   *
+   * <p>If this sketch is already in the proper form, this operation returns <i>this</i>,
+   * otherwise, this method returns a new CompactSketch of the proper form.
    *
-   * <p>If this sketch is already in compact form this operation returns <i>this</i>.
+   * <p>If this sketch is a type of UpdateSketch, the compacting process converts the hash table
+   * of the UpdateSketch to a simple list of the valid hash values.
+   * Any hash values of zero or equal-to or greater than theta will be discarded.
+   * The number of valid values remaining in the CompactSketch depends on a number of factors,
+   * but may be larger or smaller than <i>Nominal Entries</i> (or <i>k</i>).
+   * It will never exceed 2<i>k</i>.
+   * If it is critical to always limit the size to no more than <i>k</i>,
+   * then <i>rebuild()</i> should be called on the UpdateSketch prior to calling this method.</p>
    *
-   * <p>Otherwise, this compacting process converts the hash table form of an UpdateSketch to
-   * a simple list of the valid hash values from the hash table.  Any hash values equal to or
-   * greater than theta will be discarded.  The number of valid values remaining in the
-   * Compact Sketch depends on a number of factors, but may be larger or smaller than
-   * <i>Nominal Entries</i> (or <i>k</i>). It will never exceed 2<i>k</i>.  If it is critical
-   * to always limit the size to no more than <i>k</i>, then <i>rebuild()</i> should be called
-   * on the UpdateSketch prior to this.
+   * <p>A CompactSketch is always immutable.</p>
    *
    * @param dstOrdered
    * <a href="{@docRoot}/resources/dictionary.html#dstOrdered">See Destination Ordered</a>
@@ -650,7 +658,7 @@ public abstract class Sketch {
    * @param curCount the given curCount
    * @param thetaLong the given thetaLong
    * @return thetaLong
-   */
+   */ //This handles #4 above
   static final long correctThetaOnCompact(final boolean empty, final int curCount,
       final long thetaLong) {
     return (empty && (curCount == 0) && (thetaLong < Long.MAX_VALUE)) ? Long.MAX_VALUE : thetaLong;
@@ -707,19 +715,12 @@ public abstract class Sketch {
         return HeapQuickSelectSketch.heapifyInstance(srcMem, seed);
       }
       case COMPACT: {
+        final boolean empty = (flags & EMPTY_FLAG_MASK) != 0;
+        if (!empty) { PreambleUtil.checkMemorySeedHash(srcMem, seed); }
         final boolean srcOrdered = (flags & ORDERED_FLAG_MASK) != 0;
-        if (!compactFlag) {
-          throw new SketchesArgumentException(
-              "Corrupted: COMPACT family sketch image must have compact flag set");
-        }
-        final boolean readOnly = (flags & READ_ONLY_FLAG_MASK) != 0;
-        if (!readOnly) {
-          throw new SketchesArgumentException(
-              "Corrupted: COMPACT family sketch image must have Read-Only flag set");
-        }
-        final short memSeedHash = PreambleUtil.checkMemorySeedHash(srcMem, seed);
-        return CompactSketch.anyMemoryToCompactHeap(srcMem, memSeedHash, srcOrdered);
+        return CompactOperations.memoryToCompact(srcMem, srcOrdered, null);
       } //end of Compact
+
       default: {
         throw new SketchesArgumentException(
             "Sketch cannot heapify family: " + family + " as a Sketch");
diff --git a/src/main/java/org/apache/datasketches/theta/UnionImpl.java b/src/main/java/org/apache/datasketches/theta/UnionImpl.java
index bf0dc7d..a1b20bc 100644
--- a/src/main/java/org/apache/datasketches/theta/UnionImpl.java
+++ b/src/main/java/org/apache/datasketches/theta/UnionImpl.java
@@ -37,7 +37,7 @@ import static org.apache.datasketches.theta.PreambleUtil.extractSerVer;
 import static org.apache.datasketches.theta.PreambleUtil.extractThetaLong;
 import static org.apache.datasketches.theta.PreambleUtil.extractUnionThetaLong;
 import static org.apache.datasketches.theta.PreambleUtil.insertUnionThetaLong;
-import static org.apache.datasketches.theta.PreambleUtil.isSingleItemSketch;
+import static org.apache.datasketches.theta.SingleItemSketch.otherCheckForSingleItem;
 
 import org.apache.datasketches.Family;
 import org.apache.datasketches.HashOperations;
@@ -243,8 +243,10 @@ final class UnionImpl extends Union {
     final long[] compactCacheOut =
         CompactOperations.compactCache(gadgetCacheCopy, curCountOut, minThetaLong, dstOrdered);
     final boolean empty = gadget_.isEmpty() && unionEmpty_;
-    return createCompactSketch(
-        compactCacheOut, empty, seedHash_, curCountOut, minThetaLong, dstOrdered, dstMem);
+    final short seedHash = gadget_.getSeedHash();
+    return CompactOperations.componentsToCompact(
+        minThetaLong, curCountOut, seedHash, empty, true, dstOrdered, dstOrdered, dstMem,
+        compactCacheOut);
   }
 
   @Override
@@ -374,7 +376,7 @@ final class UnionImpl extends Union {
     final int preLongs = extractPreLongs(skMem);
 
     if (preLongs == 1) {
-      if (isSingleItemSketch(skMem)) {
+      if (otherCheckForSingleItem(skMem)) {
         final long hash = skMem.getLong(8);
         gadget_.hashUpdate(hash);
         return;
diff --git a/src/main/java/org/apache/datasketches/theta/UpdateSketch.java b/src/main/java/org/apache/datasketches/theta/UpdateSketch.java
index 352af73..686789d 100644
--- a/src/main/java/org/apache/datasketches/theta/UpdateSketch.java
+++ b/src/main/java/org/apache/datasketches/theta/UpdateSketch.java
@@ -26,6 +26,7 @@ import static org.apache.datasketches.Util.MIN_LG_NOM_LONGS;
 import static org.apache.datasketches.Util.checkSeedHashes;
 import static org.apache.datasketches.Util.computeSeedHash;
 import static org.apache.datasketches.hash.MurmurHash3.hash;
+import static org.apache.datasketches.theta.CompactOperations.componentsToCompact;
 import static org.apache.datasketches.theta.PreambleUtil.BIG_ENDIAN_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.COMPACT_FLAG_MASK;
 import static org.apache.datasketches.theta.PreambleUtil.FAMILY_BYTE;
@@ -136,7 +137,10 @@ public abstract class UpdateSketch extends Sketch {
 
   @Override
   public CompactSketch compact(final boolean dstOrdered, final WritableMemory dstMem) {
-    return compact(this, dstOrdered, dstMem);
+    return componentsToCompact(getThetaLong(), getRetainedEntries(), getSeedHash(), isEmpty(),
+        false, false, dstOrdered, dstMem, getCache());
+
+    //return compact(this, dstOrdered, dstMem);
   }
 
   static CompactSketch compact(final UpdateSketch usk, final boolean dstOrdered,
diff --git a/src/test/java/org/apache/datasketches/theta/AnotBimplTest.java b/src/test/java/org/apache/datasketches/theta/AnotBimplTest.java
index cc47dd6..3f4ba4d 100644
--- a/src/test/java/org/apache/datasketches/theta/AnotBimplTest.java
+++ b/src/test/java/org/apache/datasketches/theta/AnotBimplTest.java
@@ -33,7 +33,7 @@ import org.testng.annotations.Test;
 /**
  * @author Lee Rhodes
  */
-@SuppressWarnings("javadoc")
+@SuppressWarnings({"javadoc","deprecation"})
 public class AnotBimplTest {
 
   @Test
diff --git a/src/test/java/org/apache/datasketches/theta/CompactSketchTest.java b/src/test/java/org/apache/datasketches/theta/CompactSketchTest.java
index 6170e2f..199043f 100644
--- a/src/test/java/org/apache/datasketches/theta/CompactSketchTest.java
+++ b/src/test/java/org/apache/datasketches/theta/CompactSketchTest.java
@@ -228,16 +228,74 @@ public class CompactSketchTest {
 
   @Test
   public void checkDirectCompactSingleItemSketch() {
+    State state;
     UpdateSketch sk = Sketches.updateSketchBuilder().build();
-    CompactSketch csk = sk.compact(true, WritableMemory.allocate(16));
-    int bytes = csk.getCurrentBytes(true);
-    assertEquals(bytes, 8);
+
+    CompactSketch csko; //ordered
+    CompactSketch csku; //unordered
+
+    WritableMemory wmem = WritableMemory.allocate(16);
+    csko = sk.compact(true, wmem); //empty, direct, ordered
+    //ClassType, Count, Bytes, Compact, Empty, Direct, Memory, Ordered, Estimation
+    state = new State("DirectCompactOrderedSketch", 0, 8, true, true, false, true, true, false);
+    state.check(csko);
+
+    wmem = WritableMemory.allocate(16);
+    csku = sk.compact(false, wmem); //empty, direct, unordered
+    state = new State("DirectCompactOrderedSketch", 0, 8, true, true, false, true, true, false);
+    state.check(csku);
+
     sk.update(1);
-    csk = sk.compact(true, WritableMemory.allocate(16));
-    bytes = csk.getCurrentBytes(true);
-    assertEquals(bytes, 16);
-    assertTrue(csk == csk.compact());
-    assertTrue(csk == csk.compact(true, null));
+    wmem = WritableMemory.allocate(16);
+    csko = sk.compact(true, wmem); //Single, direct, ordered
+    state = new State("DirectCompactOrderedSketch", 1, 16, true, false, false, true, true, false);
+    state.check(csko);
+
+    wmem = WritableMemory.allocate(16);
+    csku = sk.compact(false, wmem); //Single, direct, unordered
+    state = new State("DirectCompactOrderedSketch", 1, 16, true, false, false, true, true, false);
+    state.check(csku);
+
+    CompactSketch csk2o; //ordered
+    CompactSketch csk2u; //unordered
+
+    csk2o = csku.compact(); //single, heap, ordered
+    state = new State("SingleItemSketch", 1, 16, true, false, false, false, true, false);
+    state.check(csk2o);
+
+    csk2o = csku.compact(true, null); //single, heap, ordered
+    state.check(csk2o);
+
+    csk2o = csku.compact(false, null); //single, heap, ordered
+    state.check(csk2o);
+
+    csk2o = csko.compact(true, null); //single, heap, ordered
+    state.check(csk2o);
+
+    csk2o = csko.compact(false, null); //single, heap, ordered
+    state.check(csk2o);
+
+    wmem = WritableMemory.allocate(16);
+    csk2o = csku.compact(true, wmem);
+    state.classType = "DirectCompactOrderedSketch";
+    state.memory = true;
+    state.check(csk2o);
+
+    wmem = WritableMemory.allocate(16);
+    csk2u = csku.compact(false, wmem);
+    state.classType = "DirectCompactOrderedSketch";
+    state.check(csk2u);
+
+    wmem = WritableMemory.allocate(16);
+    csk2o = csko.compact(true, wmem);
+    state.classType = "DirectCompactOrderedSketch";
+    state.memory = true;
+    state.check(csk2o);
+
+    wmem = WritableMemory.allocate(16);
+    csk2u = csko.compact(false, wmem);
+    state.classType = "DirectCompactOrderedSketch";
+    state.check(csk2u);
   }
 
   @Test
@@ -254,9 +312,9 @@ public class CompactSketchTest {
   @Test
   public void checkHeapifyEmptySketch() {
     UpdateSketch sk = Sketches.updateSketchBuilder().build();
-    WritableMemory wmem = WritableMemory.allocate(16); //extra bytes
+    WritableMemory wmem = WritableMemory.allocate(16); //empty, but extra bytes
     CompactSketch csk = sk.compact(false, wmem);
-    assertTrue(csk instanceof EmptyCompactSketch);
+    assertTrue(csk instanceof DirectCompactOrderedSketch);
     Sketch csk2 = Sketch.heapify(wmem);
     assertTrue(csk2 instanceof EmptyCompactSketch);
   }
@@ -271,6 +329,44 @@ public class CompactSketchTest {
     assertTrue(cache.length == 0);
   }
 
+  private static class State {
+    String classType = null;
+    int count = 0;
+    int bytes = 0;
+    boolean compact = false;
+    boolean empty = false;
+    boolean direct = false;
+    boolean memory = false;
+    boolean ordered = false;
+    boolean estimation = false;
+
+
+    State(String classType, int count, int bytes, boolean compact, boolean empty, boolean direct,
+        boolean memory, boolean ordered, boolean estimation) {
+      this.classType = classType;
+      this.count = count;
+      this.bytes = bytes;
+      this.compact = compact;
+      this.empty = empty;
+      this.direct = direct;
+      this.memory = memory;
+      this.ordered = ordered;
+      this.estimation = estimation;
+    }
+
+    void check(CompactSketch csk) {
+      assertEquals(csk.getClass().getSimpleName(), classType, "ClassType");
+      assertEquals(csk.getRetainedEntries(), count, "curCount");
+      assertEquals(csk.getCurrentBytes(true), bytes, "Bytes" );
+      assertEquals(csk.isCompact(), compact, "Compact");
+      assertEquals(csk.isEmpty(), empty, "Empty");
+      assertEquals(csk.isDirect(), direct, "Direct");
+      assertEquals(csk.hasMemory(), memory, "Memory");
+      assertEquals(csk.isOrdered(), ordered, "Ordered");
+      assertEquals(csk.isEstimationMode(), estimation, "Estimation");
+    }
+  }
+
   @Test
   public void printlnTest() {
     println("PRINTING: "+this.getClass().getName());
diff --git a/src/test/java/org/apache/datasketches/theta/ConcurrentDirectQuickSelectSketchTest.java b/src/test/java/org/apache/datasketches/theta/ConcurrentDirectQuickSelectSketchTest.java
index 3fe910e..3802dfd 100644
--- a/src/test/java/org/apache/datasketches/theta/ConcurrentDirectQuickSelectSketchTest.java
+++ b/src/test/java/org/apache/datasketches/theta/ConcurrentDirectQuickSelectSketchTest.java
@@ -27,14 +27,13 @@ import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertFalse;
 import static org.testng.Assert.assertTrue;
 
-import org.testng.annotations.Test;
-
-import org.apache.datasketches.memory.Memory;
-import org.apache.datasketches.memory.WritableDirectHandle;
-import org.apache.datasketches.memory.WritableMemory;
 import org.apache.datasketches.Family;
 import org.apache.datasketches.HashOperations;
 import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableDirectHandle;
+import org.apache.datasketches.memory.WritableMemory;
+import org.testng.annotations.Test;
 
 /**
  * @author eshcar
@@ -439,8 +438,7 @@ public class ConcurrentDirectQuickSelectSketchTest {
       assertEquals(csk2.getUpperBound(2), localUB);
       assertTrue(csk2.isEmpty());
       assertFalse(csk2.isEstimationMode());
-      assertTrue(csk2 instanceof EmptyCompactSketch);
-
+      assertTrue(csk2 instanceof DirectCompactOrderedSketch);
       CompactSketch csk3 = shared.compact(true, mem2);
       csk3.toString(false, true, 0, false);
       csk3.toString();
@@ -449,7 +447,7 @@ public class ConcurrentDirectQuickSelectSketchTest {
       assertEquals(csk3.getUpperBound(2), localUB);
       assertTrue(csk3.isEmpty());
       assertFalse(csk3.isEstimationMode());
-      assertTrue(csk2 instanceof EmptyCompactSketch);
+      assertTrue(csk2 instanceof DirectCompactOrderedSketch);
     }
   }
 
diff --git a/src/test/java/org/apache/datasketches/theta/ConcurrentHeapQuickSelectSketchTest.java b/src/test/java/org/apache/datasketches/theta/ConcurrentHeapQuickSelectSketchTest.java
index dbb6f6a..8ae2d89 100644
--- a/src/test/java/org/apache/datasketches/theta/ConcurrentHeapQuickSelectSketchTest.java
+++ b/src/test/java/org/apache/datasketches/theta/ConcurrentHeapQuickSelectSketchTest.java
@@ -30,14 +30,13 @@ import static org.testng.Assert.fail;
 
 import java.util.Arrays;
 
-import org.testng.annotations.AfterMethod;
-import org.testng.annotations.Test;
-
-import org.apache.datasketches.memory.Memory;
-import org.apache.datasketches.memory.WritableMemory;
 import org.apache.datasketches.Family;
 import org.apache.datasketches.SketchesArgumentException;
 import org.apache.datasketches.SketchesStateException;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+import org.testng.annotations.AfterMethod;
+import org.testng.annotations.Test;
 
 /**
  * @author eshcar
@@ -382,7 +381,7 @@ public class ConcurrentHeapQuickSelectSketchTest {
     assertEquals(csk2.getUpperBound(2), localUB);
     assertEquals(csk2.isEmpty(), true);
     assertEquals(csk2.isEstimationMode(), estimating);
-    assertTrue(csk2 instanceof EmptyCompactSketch);
+    assertTrue(csk2 instanceof DirectCompactOrderedSketch);
 
     CompactSketch csk3 = shared.compact(true, mem2);
     csk3.toString(false, true, 0, false);
@@ -392,7 +391,7 @@ public class ConcurrentHeapQuickSelectSketchTest {
     assertEquals(csk3.getUpperBound(2), localUB);
     assertEquals(csk3.isEmpty(), true);
     assertEquals(csk3.isEstimationMode(), estimating);
-    assertTrue(csk3 instanceof EmptyCompactSketch);
+    assertTrue(csk3 instanceof DirectCompactOrderedSketch);
   }
 
   @Test
@@ -594,7 +593,7 @@ public class ConcurrentHeapQuickSelectSketchTest {
     assertEquals(csk2.getUpperBound(2), uskUB);
     assertTrue(csk2.isEmpty());
     assertFalse(csk2.isEstimationMode());
-    assertTrue(csk2 instanceof EmptyCompactSketch);
+    assertTrue(csk2 instanceof DirectCompactOrderedSketch);
 
     CompactSketch csk3 = shared.compact(true, mem2);
     csk3.toString(false, true, 0, false);
@@ -604,7 +603,7 @@ public class ConcurrentHeapQuickSelectSketchTest {
     assertEquals(csk3.getUpperBound(2), uskUB);
     assertTrue(csk3.isEmpty());
     assertFalse(csk3.isEstimationMode());
-    assertTrue(csk2 instanceof EmptyCompactSketch);
+    assertTrue(csk2 instanceof DirectCompactOrderedSketch);
   }
 
   @Test(expectedExceptions = SketchesArgumentException.class)
diff --git a/src/test/java/org/apache/datasketches/theta/DirectQuickSelectSketchTest.java b/src/test/java/org/apache/datasketches/theta/DirectQuickSelectSketchTest.java
index 2ab13ca..73245e5 100644
--- a/src/test/java/org/apache/datasketches/theta/DirectQuickSelectSketchTest.java
+++ b/src/test/java/org/apache/datasketches/theta/DirectQuickSelectSketchTest.java
@@ -41,16 +41,15 @@ import static org.testng.Assert.fail;
 
 import java.util.Arrays;
 
-import org.testng.annotations.Test;
-
-import org.apache.datasketches.memory.Memory;
-import org.apache.datasketches.memory.WritableDirectHandle;
-import org.apache.datasketches.memory.WritableMemory;
 import org.apache.datasketches.Family;
 import org.apache.datasketches.HashOperations;
 import org.apache.datasketches.ResizeFactor;
 import org.apache.datasketches.SketchesArgumentException;
 import org.apache.datasketches.SketchesReadOnlyException;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableDirectHandle;
+import org.apache.datasketches.memory.WritableMemory;
+import org.testng.annotations.Test;
 
 /**
  * @author Lee Rhodes
@@ -389,7 +388,7 @@ public class DirectQuickSelectSketchTest {
       assertEquals(csk2.getUpperBound(2), uskUB);
       assertEquals(csk2.isEmpty(), true);
       assertEquals(csk2.isEstimationMode(), false);
-      assertEquals(csk2.getClass().getSimpleName(), "EmptyCompactSketch");
+      assertEquals(csk2.getClass().getSimpleName(), "DirectCompactOrderedSketch");
 
       CompactSketch csk3 = usk.compact(true, mem2);
       csk3.toString(false, true, 0, false);
@@ -399,7 +398,7 @@ public class DirectQuickSelectSketchTest {
       assertEquals(csk3.getUpperBound(2), uskUB);
       assertEquals(csk3.isEmpty(), true);
       assertEquals(csk3.isEstimationMode(), false);
-      assertEquals(csk3.getClass().getSimpleName(), "EmptyCompactSketch");
+      assertEquals(csk3.getClass().getSimpleName(), "DirectCompactOrderedSketch");
     }
   }
 
diff --git a/src/test/java/org/apache/datasketches/theta/EmptyTest.java b/src/test/java/org/apache/datasketches/theta/EmptyTest.java
index 7bce4c1..7b93117 100644
--- a/src/test/java/org/apache/datasketches/theta/EmptyTest.java
+++ b/src/test/java/org/apache/datasketches/theta/EmptyTest.java
@@ -36,7 +36,7 @@ import org.testng.annotations.Test;
  *
  * @author Lee Rhodes
  */
-@SuppressWarnings("javadoc")
+@SuppressWarnings({"javadoc","deprecation"})
 public class EmptyTest {
 
   @Test
@@ -136,9 +136,10 @@ public class EmptyTest {
     Sketches.heapifySketch(mem);
   }
 
-  private static Memory badEmptySk() { //missing the empty bit
+  //SerVer 2 had an empty sketch where preLongs = 1, but empty bit was not set.
+  private static Memory badEmptySk() {
     final long preLongs = 1;
-    final long serVer = 3;
+    final long serVer = 2;
     final long family = 3; //compact
     final long flags = (ORDERED_FLAG_MASK | COMPACT_FLAG_MASK | READ_ONLY_FLAG_MASK);
     final long seedHash = 0x93CC;
diff --git a/src/test/java/org/apache/datasketches/theta/ForwardCompatibilityTest.java b/src/test/java/org/apache/datasketches/theta/ForwardCompatibilityTest.java
index 29dfd09..e5d040c 100644
--- a/src/test/java/org/apache/datasketches/theta/ForwardCompatibilityTest.java
+++ b/src/test/java/org/apache/datasketches/theta/ForwardCompatibilityTest.java
@@ -24,12 +24,11 @@ import static org.apache.datasketches.theta.BackwardConversions.convertSerVer3to
 import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertTrue;
 
-import org.testng.annotations.Test;
-
-import org.apache.datasketches.memory.Memory;
-import org.apache.datasketches.memory.WritableMemory;
 import org.apache.datasketches.SketchesArgumentException;
 import org.apache.datasketches.Util;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+import org.testng.annotations.Test;
 
 /**
  * @author Lee Rhodes
@@ -113,16 +112,10 @@ public class ForwardCompatibilityTest {
     WritableMemory srcMemW = WritableMemory.allocate(16);
     srcMem.copyTo(0, srcMemW, 0, 16);
     PreambleUtil.setEmpty(srcMemW); //Force
-    assertTrue(PreambleUtil.isEmptySketch(srcMemW));
+    assertTrue(PreambleUtil.isEmptyFlag(srcMemW));
     srcMemW.putInt(8, 0); //corrupt curCount = 0
 
     Sketch sketch = Sketch.heapify(srcMemW);
-    assertEquals(sketch.isEmpty(), true); //was forced true
-    assertEquals(sketch.isEstimationMode(), false);
-    assertEquals(sketch.isDirect(), false);
-    assertEquals(sketch.hasMemory(), false);
-    assertEquals(sketch.isCompact(), true);
-    assertEquals(sketch.isOrdered(), true);
     assertTrue(sketch instanceof EmptyCompactSketch);
   }
 
@@ -136,17 +129,11 @@ public class ForwardCompatibilityTest {
     WritableMemory srcMemW = WritableMemory.allocate(24);
     srcMem.copyTo(0, srcMemW, 0, 24);
     PreambleUtil.setEmpty(srcMemW); //Force
-    assertTrue(PreambleUtil.isEmptySketch(srcMemW));
+    assertTrue(PreambleUtil.isEmptyFlag(srcMemW));
     srcMemW.putInt(8, 0); //corrupt curCount = 0
     srcMemW.putLong(16, Long.MAX_VALUE); //corrupt to make it look empty
 
-    Sketch sketch = Sketch.heapify(srcMemW);
-    assertEquals(sketch.isEmpty(), true); //was forced true
-    assertEquals(sketch.isEstimationMode(), false);
-    assertEquals(sketch.isDirect(), false);
-    assertEquals(sketch.hasMemory(), false);
-    assertEquals(sketch.isCompact(), true);
-    assertEquals(sketch.isOrdered(), true);
+    Sketch sketch = Sketch.heapify(srcMemW); //now serVer=3, EmptyCompactSketch
     assertTrue(sketch instanceof EmptyCompactSketch);
   }
 
diff --git a/src/test/java/org/apache/datasketches/theta/HeapAlphaSketchTest.java b/src/test/java/org/apache/datasketches/theta/HeapAlphaSketchTest.java
index eb814ab..793eb13 100644
--- a/src/test/java/org/apache/datasketches/theta/HeapAlphaSketchTest.java
+++ b/src/test/java/org/apache/datasketches/theta/HeapAlphaSketchTest.java
@@ -38,14 +38,13 @@ import static org.testng.Assert.assertNull;
 import static org.testng.Assert.assertTrue;
 import static org.testng.Assert.fail;
 
-import org.testng.annotations.Test;
-
-import org.apache.datasketches.memory.Memory;
-import org.apache.datasketches.memory.WritableMemory;
 import org.apache.datasketches.Family;
 import org.apache.datasketches.ResizeFactor;
 import org.apache.datasketches.SketchesArgumentException;
 import org.apache.datasketches.Util;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+import org.testng.annotations.Test;
 
 /**
  * @author Lee Rhodes
@@ -322,7 +321,7 @@ public class HeapAlphaSketchTest {
     assertEquals(csk2.getUpperBound(2), uskUB);
     assertEquals(csk2.isEmpty(), true);
     assertEquals(csk2.isEstimationMode(), estimating);
-    assertTrue(csk2 instanceof EmptyCompactSketch);
+    assertTrue(csk2 instanceof DirectCompactOrderedSketch);
 
     CompactSketch csk3 = usk.compact(true, mem2);
     csk3.toString(false, true, 0, false);
@@ -332,7 +331,7 @@ public class HeapAlphaSketchTest {
     assertEquals(csk3.getUpperBound(2), uskUB);
     assertEquals(csk3.isEmpty(), true);
     assertEquals(csk3.isEstimationMode(), estimating);
-    assertTrue(csk3 instanceof EmptyCompactSketch);
+    assertTrue(csk3 instanceof DirectCompactOrderedSketch);
   }
 
   @Test
diff --git a/src/test/java/org/apache/datasketches/theta/HeapQuickSelectSketchTest.java b/src/test/java/org/apache/datasketches/theta/HeapQuickSelectSketchTest.java
index 2b74ee6..fcec52b 100644
--- a/src/test/java/org/apache/datasketches/theta/HeapQuickSelectSketchTest.java
+++ b/src/test/java/org/apache/datasketches/theta/HeapQuickSelectSketchTest.java
@@ -39,14 +39,13 @@ import static org.testng.Assert.fail;
 
 import java.util.Arrays;
 
-import org.testng.annotations.Test;
-
-import org.apache.datasketches.memory.Memory;
-import org.apache.datasketches.memory.WritableMemory;
 import org.apache.datasketches.Family;
 import org.apache.datasketches.ResizeFactor;
 import org.apache.datasketches.SketchesArgumentException;
 import org.apache.datasketches.Util;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+import org.testng.annotations.Test;
 
 /**
  * @author Lee Rhodes
@@ -319,7 +318,7 @@ public class HeapQuickSelectSketchTest {
     assertEquals(csk2.getUpperBound(2), uskUB);
     assertEquals(csk2.isEmpty(), true);
     assertEquals(csk2.isEstimationMode(), estimating);
-    assertEquals(csk2.getClass().getSimpleName(), "EmptyCompactSketch");
+    assertEquals(csk2.getClass().getSimpleName(), "DirectCompactOrderedSketch");
 
     CompactSketch csk3 = usk.compact(true, mem2);
     println(csk3.toString(false, true, 0, false));
@@ -329,7 +328,7 @@ public class HeapQuickSelectSketchTest {
     assertEquals(csk3.getUpperBound(2), uskUB);
     assertEquals(csk3.isEmpty(), true);
     assertEquals(csk3.isEstimationMode(), estimating);
-    assertEquals(csk3.getClass().getSimpleName(), "EmptyCompactSketch");
+    assertEquals(csk3.getClass().getSimpleName(), "DirectCompactOrderedSketch");
   }
 
   @Test
diff --git a/src/test/java/org/apache/datasketches/theta/PairwiseSetOperationsTest.java b/src/test/java/org/apache/datasketches/theta/PairwiseSetOperationsTest.java
index 5c1b76f..9c61b9a 100644
--- a/src/test/java/org/apache/datasketches/theta/PairwiseSetOperationsTest.java
+++ b/src/test/java/org/apache/datasketches/theta/PairwiseSetOperationsTest.java
@@ -23,7 +23,7 @@ import static org.testng.Assert.assertEquals;
 
 import org.testng.annotations.Test;
 
-@SuppressWarnings("javadoc")
+@SuppressWarnings({"javadoc","deprecation"})
 public class PairwiseSetOperationsTest {
 
   // Intersection
diff --git a/src/test/java/org/apache/datasketches/theta/SetOperationTest.java b/src/test/java/org/apache/datasketches/theta/SetOperationTest.java
index 0f21a2e..cbcb1c3 100644
--- a/src/test/java/org/apache/datasketches/theta/SetOperationTest.java
+++ b/src/test/java/org/apache/datasketches/theta/SetOperationTest.java
@@ -33,20 +33,19 @@ import static org.testng.Assert.assertTrue;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 
-import org.testng.annotations.Test;
-
+import org.apache.datasketches.Family;
+import org.apache.datasketches.ResizeFactor;
+import org.apache.datasketches.SketchesArgumentException;
 import org.apache.datasketches.memory.DefaultMemoryRequestServer;
 import org.apache.datasketches.memory.Memory;
 import org.apache.datasketches.memory.MemoryRequestServer;
 import org.apache.datasketches.memory.WritableMemory;
-import org.apache.datasketches.Family;
-import org.apache.datasketches.ResizeFactor;
-import org.apache.datasketches.SketchesArgumentException;
+import org.testng.annotations.Test;
 
 /**
  * @author Lee Rhodes
  */
-@SuppressWarnings("javadoc")
+@SuppressWarnings({"javadoc","deprecation"})
 public class SetOperationTest {
 
   @Test
diff --git a/src/test/java/org/apache/datasketches/theta/SetOpsCornerCasesTest.java b/src/test/java/org/apache/datasketches/theta/SetOpsCornerCasesTest.java
index 36abe4a..2626760 100644
--- a/src/test/java/org/apache/datasketches/theta/SetOpsCornerCasesTest.java
+++ b/src/test/java/org/apache/datasketches/theta/SetOpsCornerCasesTest.java
@@ -217,6 +217,7 @@ public class SetOpsCornerCasesTest {
     return PairwiseSetOperations.aNotB(tskA, tskB);
   }
 
+
   private static void checkCornerCase(Sketch rskA, Sketch rskB) {
     double estA = rskA.getEstimate();
     double estB = rskB.getEstimate();
@@ -230,8 +231,6 @@ public class SetOpsCornerCasesTest {
     Assert.assertEquals(emptyB, emptyA);
     Assert.assertEquals(thetaLongB, thetaLongA);
     Assert.assertEquals(countB, countA);
-    String A = rskA.getClass().getSimpleName();
-    String B = rskB.getClass().getSimpleName();
     Assert.assertEquals(rskA.getClass().getSimpleName(), rskB.getClass().getSimpleName());
   }
 
diff --git a/src/test/java/org/apache/datasketches/theta/SingleItemSketchTest.java b/src/test/java/org/apache/datasketches/theta/SingleItemSketchTest.java
index 23a0a37..7a50ba8 100644
--- a/src/test/java/org/apache/datasketches/theta/SingleItemSketchTest.java
+++ b/src/test/java/org/apache/datasketches/theta/SingleItemSketchTest.java
@@ -36,7 +36,7 @@ import org.testng.annotations.Test;
 /**
  * @author Lee Rhodes
  */
-@SuppressWarnings("javadoc")
+@SuppressWarnings({"javadoc","deprecation"})
 public class SingleItemSketchTest {
   final static short DEFAULT_SEED_HASH = (short) (computeSeedHash(DEFAULT_UPDATE_SEED) & 0XFFFFL);
 
@@ -202,13 +202,12 @@ public class SingleItemSketchTest {
     csk = sk1.compact(false, null);
     assertTrue(csk instanceof SingleItemSketch);
 
-    //SingleItemSketch has no off-heap form.
     bytes = Sketches.getMaxCompactSketchBytes(1);
     wmem = WritableMemory.wrap(new byte[bytes]);
     csk = sk1.compact(true, wmem);
-    assertTrue(csk instanceof SingleItemSketch);
+    assertTrue(csk instanceof DirectCompactOrderedSketch);
     csk = sk1.compact(false, wmem);
-    assertTrue(csk instanceof SingleItemSketch);
+    assertTrue(csk instanceof DirectCompactOrderedSketch);
   }
 
   @Test
@@ -295,7 +294,7 @@ public class SingleItemSketchTest {
     inter.update(sk2);
     WritableMemory wmem = WritableMemory.wrap(new byte[16]);
     CompactSketch csk = inter.getResult(false, wmem);
-    assertTrue(csk instanceof SingleItemSketch);
+    assertTrue(csk instanceof DirectCompactOrderedSketch);
     Sketch csk2 = Sketches.heapifySketch(wmem);
     assertTrue(csk2 instanceof SingleItemSketch);
     println(csk2.toString(true, true, 1, true));
diff --git a/src/test/java/org/apache/datasketches/theta/SketchTest.java b/src/test/java/org/apache/datasketches/theta/SketchTest.java
index 3cf9783..037dd47 100644
--- a/src/test/java/org/apache/datasketches/theta/SketchTest.java
+++ b/src/test/java/org/apache/datasketches/theta/SketchTest.java
@@ -39,19 +39,18 @@ import static org.testng.Assert.assertFalse;
 import static org.testng.Assert.assertTrue;
 import static org.testng.Assert.fail;
 
-import org.testng.annotations.Test;
-
-import org.apache.datasketches.memory.Memory;
-import org.apache.datasketches.memory.WritableMemory;
 import org.apache.datasketches.Family;
 import org.apache.datasketches.ResizeFactor;
 import org.apache.datasketches.SketchesArgumentException;
 import org.apache.datasketches.Util;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+import org.testng.annotations.Test;
 
 /**
  * @author Lee Rhodes
  */
-@SuppressWarnings("javadoc")
+@SuppressWarnings({"javadoc","deprecation"})
 public class SketchTest {
 
   @Test
diff --git a/src/test/java/org/apache/datasketches/tuple/adouble/AdoubleAnotBTest.java b/src/test/java/org/apache/datasketches/tuple/adouble/AdoubleAnotBTest.java
index d113ed1..229558e 100644
--- a/src/test/java/org/apache/datasketches/tuple/adouble/AdoubleAnotBTest.java
+++ b/src/test/java/org/apache/datasketches/tuple/adouble/AdoubleAnotBTest.java
@@ -44,6 +44,7 @@ public class AdoubleAnotBTest {
   private final DoubleSummary.Mode mode = Mode.Sum;
   private final Results results = new Results();
 
+  @SuppressWarnings("deprecation")
   private static void threeMethodsWithTheta(
       final AnotB<DoubleSummary> aNotB,
       final Sketch<DoubleSummary> skA,
@@ -147,6 +148,7 @@ public class AdoubleAnotBTest {
 
   /*****************************************/
 
+  @SuppressWarnings("deprecation")
   @Test
   public void aNotBNullEmptyCombinations() {
     AnotB<DoubleSummary> aNotB = new AnotB<>();
diff --git a/src/test/java/org/apache/datasketches/tuple/adouble/AdoubleIntersectionTest.java b/src/test/java/org/apache/datasketches/tuple/adouble/AdoubleIntersectionTest.java
index 25359ca..4dc75f7 100644
--- a/src/test/java/org/apache/datasketches/tuple/adouble/AdoubleIntersectionTest.java
+++ b/src/test/java/org/apache/datasketches/tuple/adouble/AdoubleIntersectionTest.java
@@ -82,6 +82,7 @@ public class AdoubleIntersectionTest {
     Assert.assertEquals(result.getUpperBound(1), 0.0);
   }
 
+  @SuppressWarnings("deprecation")
   @Test
   public void intersectionExactMode() {
     UpdatableSketch<Double, DoubleSummary> sketch1 =
diff --git a/src/test/java/org/apache/datasketches/tuple/aninteger/IntegerSketchTest.java b/src/test/java/org/apache/datasketches/tuple/aninteger/IntegerSketchTest.java
index 217b066..5cd7b98 100644
--- a/src/test/java/org/apache/datasketches/tuple/aninteger/IntegerSketchTest.java
+++ b/src/test/java/org/apache/datasketches/tuple/aninteger/IntegerSketchTest.java
@@ -70,6 +70,7 @@ public class IntegerSketchTest {
     assertEquals(csk.getEstimate(), K * 1.0, K * .03);
   }
 
+  @SuppressWarnings("deprecation")
   @Test
   public void aNotBTest() {
     int lgK = 4;
@@ -82,7 +83,7 @@ public class IntegerSketchTest {
       a1Sk1.update(i, 1);
     }
     anotb.update(a1Sk1, a1Sk2);
-    CompactSketch<IntegerSummary> cSk = anotb.getResult();
+    anotb.getResult();
   }
 
   @Test
diff --git a/src/test/java/org/apache/datasketches/tuple/strings/ArrayOfStringsSketchTest.java b/src/test/java/org/apache/datasketches/tuple/strings/ArrayOfStringsSketchTest.java
index 908079f..43b4034 100644
--- a/src/test/java/org/apache/datasketches/tuple/strings/ArrayOfStringsSketchTest.java
+++ b/src/test/java/org/apache/datasketches/tuple/strings/ArrayOfStringsSketchTest.java
@@ -22,14 +22,13 @@ package org.apache.datasketches.tuple.strings;
 import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertTrue;
 
-import org.testng.annotations.Test;
-
 import org.apache.datasketches.memory.WritableMemory;
 import org.apache.datasketches.tuple.AnotB;
 import org.apache.datasketches.tuple.CompactSketch;
 import org.apache.datasketches.tuple.Intersection;
 import org.apache.datasketches.tuple.SketchIterator;
 import org.apache.datasketches.tuple.Union;
+import org.testng.annotations.Test;
 
 /**
  * @author Lee Rhodes
@@ -38,6 +37,7 @@ import org.apache.datasketches.tuple.Union;
 public class ArrayOfStringsSketchTest {
   private static final String LS = System.getProperty("line.separator");
 
+  @SuppressWarnings("deprecation")
   @Test
   public void checkSketch() {
     ArrayOfStringsSketch sketch1 = new ArrayOfStringsSketch();
@@ -57,7 +57,6 @@ public class ArrayOfStringsSketchTest {
     String[] strArr3 = {"g", "h" };
     sketch2.update(strArr3, strArr3);
 
-
     Union<ArrayOfStringsSummary> union = new Union<>(new ArrayOfStringsSummarySetOperations());
     union.update(sketch1);
     union.update(sketch2);
diff --git a/tools/FindBugsExcludeFilter.xml b/tools/FindBugsExcludeFilter.xml
index f090992..ee747b6 100644
--- a/tools/FindBugsExcludeFilter.xml
+++ b/tools/FindBugsExcludeFilter.xml
@@ -1,5 +1,3 @@
-<FindBugsFilter> <!-- sketches-core -->
-
 <!--
 Licensed to the Apache Software Foundation (ASF) under one
 or more contributor license agreements.  See the NOTICE file
@@ -18,6 +16,7 @@ KIND, either express or implied.  See the License for the
 specific language governing permissions and limitations
 under the License.
 -->
+<FindBugsFilter> <!-- sketches-core -->
 
   <!-- Too many false positives to be useful.  I could not make it happy :( -->
   <Match>


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