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/07/25 01:22:11 UTC

[incubator-datasketches-java] 02/02: Refactoring of Theta Intersection.

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

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

commit 410a1c53cb500b7fd3c651f52cdaba60f4bb493c
Author: Lee Rhodes <le...@users.noreply.github.com>
AuthorDate: Fri Jul 24 18:21:45 2020 -0700

    Refactoring of Theta Intersection.
    
    Eliminated the ItersectionImplR class
    
    Implemented the new standard way of handling nulls.
    
    Changed the operation from "update" to "intersect"
    
    We now have both stateful and stateless options.
    
    Lots of code cleanup.
---
 .../apache/datasketches/kll/KllFloatsSketch.java   |  43 +-
 .../apache/datasketches/theta/Intersection.java    |  94 ++++
 .../datasketches/theta/IntersectionImpl.java       | 498 +++++++++++++--------
 .../datasketches/theta/IntersectionImplR.java      | 340 --------------
 .../apache/datasketches/theta/SetOperation.java    |   4 +-
 .../apache/datasketches/tuple/Intersection.java    |   3 -
 .../datasketches/theta/CompactSketchTest.java      |   2 +-
 .../datasketches/theta/DirectIntersectionTest.java |   6 +-
 8 files changed, 447 insertions(+), 543 deletions(-)

diff --git a/src/main/java/org/apache/datasketches/kll/KllFloatsSketch.java b/src/main/java/org/apache/datasketches/kll/KllFloatsSketch.java
index c571f31..a287f46 100644
--- a/src/main/java/org/apache/datasketches/kll/KllFloatsSketch.java
+++ b/src/main/java/org/apache/datasketches/kll/KllFloatsSketch.java
@@ -107,28 +107,51 @@ import org.apache.datasketches.memory.Memory;
  * <li>Then <i>r - eps &le; trueRank &le; r + eps</i> with a confidence of 99%.</li>
  * </ul>
  *
- * <p>A <i>getPMF()</i> query has the following guarantees:
+ * <p>A <i>getPMF(...)</i> query has the following guarantees:
  * <ul>
- * <li>Let <i>{r1, r2, ..., r(m+1)} = getPMF(v1, v2, ..., vm)</i> where <i>v1, v2</i> are values
- * between the min and max values of the input stream.
- * <li>Let <i>mass<sub>i</sub> = estimated mass between v<sub>i</sub> and v<sub>i+1</sub></i>.</li>
+ * <li>Let <i>{r<sub>1</sub>, r<sub>2</sub>, ..., r<sub>m+1</sub>}
+ * = getPMF(v<sub>1</sub>, v<sub>2</sub>, ..., v<sub>m</sub>)</i> where
+ * <i>v<sub>1</sub>, v<sub>2</sub>, ..., v<sub>m</sub></i> are monotonically increasing values
+ * supplied by the user that are part of the monotonic sequence
+ * <i>v<sub>0</sub> = min, v<sub>1</sub>, v<sub>2</sub>, ..., v<sub>m</sub>, v<sub>m+1</sub> = max</i>,
+ * and where <i>min</i> and <i>max</i> are the actual minimum and maximum values of the input
+ * stream automatically included in the sequence by the <i>getPMF(...)</i> function.
+ *
+ * <li>Let <i>r<sub>i</sub> = mass<sub>i</sub></i> = estimated mass between
+ * <i>v<sub>i-1</sub></i> and <i>v<sub>i</sub></i> where <i>v<sub>0</sub> = min</i>
+ * and <i>v<sub>m+1</sub> = max</i>.</li>
+ *
  * <li>Let <i>trueMass</i> be the true mass between the values of <i>v<sub>i</sub>,
  * v<sub>i+1</sub></i> derived from the hypothetical sorted stream of all <i>N</i> values.</li>
  * <li>Let <i>eps = getNormalizedRankError(true)</i>.</li>
- * <li>then <i>mass - eps &le; trueMass &le; mass + eps</i> with a confidence of 99%.</li>
- * <li>r(m+1) includes the mass of all points larger than vm.</li>
+ * <li>Then <i>mass - eps &le; trueMass &le; mass + eps</i> with a confidence of 99%.</li>
+ * <li><i>r<sub>1</sub></i> includes the mass of all points between <i>min = v<sub>0</sub></i> and
+ * <i>v<sub>1</sub></i>.</li>
+ * <li><i>r<sub>m+1</sub></i> includes the mass of all points between <i>v<sub>m</sub></i> and
+ * <i>max = v<sub>m+1</sub></i>.</li>
  * </ul>
  *
  * <p>A <i>getCDF(...)</i> query has the following guarantees;
  * <ul>
- * <li>Let <i>{r1, r2, ..., r(m+1)} = getCDF(v1, v2, ..., vm)</i> where <i>v1, v2</i> are values
- * between the min and max values of the input stream.
- * <li>Let <i>mass<sub>i</sub> = r<sub>i+1</sub> - r<sub>i</sub></i>.</li>
+ * <li>Let <i>{r<sub>1</sub>, r<sub>2</sub>, ..., r<sub>m+1</sub>}
+ * = getCDF(v<sub>1</sub>, v<sub>2</sub>, ..., v<sub>m</sub>)</i> where
+ * <i>v<sub>1</sub>, v<sub>2</sub>, ..., v<sub>m</sub>)</i> are monotonically increasing values
+ * supplied by the user that are part of the monotonic sequence
+ * <i>{v<sub>0</sub> = min, v<sub>1</sub>, v<sub>2</sub>, ..., v<sub>m</sub>, v<sub>m+1</sub> = max}</i>,
+ * and where <i>min</i> and <i>max</i> are the actual minimum and maximum values of the input
+ * stream automatically included in the sequence by the <i>getCDF(...)</i> function.
+ *
+ * <li>Let <i>r<sub>i</sub> = mass<sub>i</sub></i> = estimated mass between
+ * <i>v<sub>0</sub> = min</i> and <i>v<sub>i</sub></i>.</li>
+ *
  * <li>Let <i>trueMass</i> be the true mass between the true ranks of <i>v<sub>i</sub>,
  * v<sub>i+1</sub></i> derived from the hypothetical sorted stream of all <i>N</i> values.</li>
  * <li>Let <i>eps = getNormalizedRankError(true)</i>.</li>
  * <li>then <i>mass - eps &le; trueMass &le; mass + eps</i> with a confidence of 99%.</li>
- * <li>1 - r(m+1) includes the mass of all points larger than vm.</li>
+ * <li><i>r<sub>1</sub></i> includes the mass of all points between <i>min = v<sub>0</sub></i> and
+ * <i>v<sub>1</sub></i>.</li>
+ * <li><i>r<sub>m+1</sub></i> includes the mass of all points between <i>min = v<sub>0</sub></i> and
+ * <i>max = v<sub>m+1</sub></i>.</li>
  * </ul>
  *
  * <p>From the above, it might seem like we could make some estimates to bound the
diff --git a/src/main/java/org/apache/datasketches/theta/Intersection.java b/src/main/java/org/apache/datasketches/theta/Intersection.java
index 3a726a6..c7394ed 100644
--- a/src/main/java/org/apache/datasketches/theta/Intersection.java
+++ b/src/main/java/org/apache/datasketches/theta/Intersection.java
@@ -19,7 +19,21 @@
 
 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.PreambleUtil.EMPTY_FLAG_MASK;
+import static org.apache.datasketches.theta.PreambleUtil.SER_VER;
+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.extractPreLongs;
+import static org.apache.datasketches.theta.PreambleUtil.extractSerVer;
+
+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;
 
 /**
@@ -133,4 +147,84 @@ public abstract class Intersection extends SetOperation {
   public abstract CompactSketch intersect(Sketch a, Sketch b, boolean dstOrdered,
       WritableMemory dstMem);
 
+  // Restricted
+
+  /**
+   * Returns the maximum lgArrLongs given the capacity of the Memory.
+   * @param dstMem the given Memory
+   * @return the maximum lgArrLongs given the capacity of the Memory
+   */
+  protected static int getMaxLgArrLongs(final Memory dstMem) {
+    final int preBytes = CONST_PREAMBLE_LONGS << 3;
+    final long cap = dstMem.getCapacity();
+    return Integer.numberOfTrailingZeros(floorPowerOf2((int)(cap - preBytes)) >>> 3);
+  }
+
+  protected static void checkMinSizeMemory(final Memory mem) {
+    final int minBytes = (CONST_PREAMBLE_LONGS << 3) + (8 << MIN_LG_ARR_LONGS);//280
+    final long cap = mem.getCapacity();
+    if (cap < minBytes) {
+      throw new SketchesArgumentException(
+          "Memory must be at least " + minBytes + " bytes. Actual capacity: " + cap);
+    }
+  }
+
+  /**
+   * 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 IntersectionImpl & Test
+  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;
+  }
+
+  protected static void memChecks(final Memory srcMem) {
+    //Get Preamble
+    //Note: Intersection does not use lgNomLongs (or k), per se.
+    //seedHash loaded and checked in private constructor
+    final int preLongs = extractPreLongs(srcMem);
+    final int serVer = extractSerVer(srcMem);
+    final int famID = extractFamilyID(srcMem);
+    final boolean empty = (extractFlags(srcMem) & EMPTY_FLAG_MASK) > 0;
+    final int curCount = extractCurCount(srcMem);
+    //Checks
+    if (preLongs != CONST_PREAMBLE_LONGS) {
+      throw new SketchesArgumentException(
+          "Memory PreambleLongs must equal " + CONST_PREAMBLE_LONGS + ": " + preLongs);
+    }
+    if (serVer != SER_VER) {
+      throw new SketchesArgumentException("Serialization Version must equal " + SER_VER);
+    }
+    Family.INTERSECTION.checkFamilyID(famID);
+    if (empty) {
+      if (curCount != 0) {
+        throw new SketchesArgumentException(
+            "srcMem empty state inconsistent with curCount: " + empty + "," + curCount);
+      }
+      //empty = true AND curCount_ = 0: OK
+    } //else empty = false, curCount could be anything
+  }
+
 }
diff --git a/src/main/java/org/apache/datasketches/theta/IntersectionImpl.java b/src/main/java/org/apache/datasketches/theta/IntersectionImpl.java
index aa49451..63c3668 100644
--- a/src/main/java/org/apache/datasketches/theta/IntersectionImpl.java
+++ b/src/main/java/org/apache/datasketches/theta/IntersectionImpl.java
@@ -27,21 +27,27 @@ import static org.apache.datasketches.HashOperations.hashSearch;
 import static org.apache.datasketches.HashOperations.minLgHashTableSize;
 import static org.apache.datasketches.Util.MIN_LG_ARR_LONGS;
 import static org.apache.datasketches.Util.REBUILD_THRESHOLD;
+import static org.apache.datasketches.Util.computeSeedHash;
 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.LG_ARR_LONGS_BYTE;
+import static org.apache.datasketches.theta.PreambleUtil.LG_NOM_LONGS_BYTE;
 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;
+import static org.apache.datasketches.theta.PreambleUtil.SEED_HASH_SHORT;
 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 static org.apache.datasketches.theta.PreambleUtil.clearEmpty;
+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.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.insertLgArrLongs;
-import static org.apache.datasketches.theta.PreambleUtil.insertLgNomLongs;
 import static org.apache.datasketches.theta.PreambleUtil.insertP;
 import static org.apache.datasketches.theta.PreambleUtil.insertPreLongs;
 import static org.apache.datasketches.theta.PreambleUtil.insertSerVer;
@@ -52,146 +58,174 @@ import java.util.Arrays;
 
 import org.apache.datasketches.Family;
 import org.apache.datasketches.SketchesArgumentException;
+import org.apache.datasketches.SketchesReadOnlyException;
+import org.apache.datasketches.SketchesStateException;
 import org.apache.datasketches.Util;
 import org.apache.datasketches.memory.Memory;
 import org.apache.datasketches.memory.WritableMemory;
 
-final class IntersectionImpl extends IntersectionImplR {
+/**
+ * Intersection operation for Theta Sketches.
+ *
+ * <p>This implementation uses data either on-heap or off-heap in a given Memory
+ * that is owned and managed by the caller.
+ * The off-heap Memory, which if managed properly, will greatly reduce the need for
+ * the JVM to perform garbage collection.</p>
+ *
+ * @author Lee Rhodes
+ * @author Kevin Lang
+ */
+class IntersectionImpl extends Intersection {
+  protected final short seedHash_;
+  protected final boolean readOnly_; //True if this sketch is to be treated as read only
+  protected final WritableMemory wmem_;
+  protected final int maxLgArrLongs_; //only used with WritableMemory, not serialized
+
+  //Note: Intersection does not use lgNomLongs or k, per se.
+  protected int lgArrLongs_; //current size of hash table
+  protected int curCount_; //curCount of HT, if < 0 means Universal Set (US) is true
+  protected long thetaLong_;
+  protected boolean empty_; //A virgin intersection represents the Universal Set, so empty is FALSE!
+  protected long[] hashTable_; //retained entries of the intersection, on-heap only.
 
-  private IntersectionImpl(final WritableMemory wmem, final long seed, final boolean newMem) {
-    super(wmem, seed, newMem);
+  /**
+   * Constructor: Sets the class finals and computes, sets and checks the seedHash.
+   * @param wmem Can be either a Source(e.g. wrap) or Destination (new Direct) WritableMemory.
+   * @param seed Used to validate incoming sketch arguments.
+   * @param dstMemFlag The given memory is a Destination (new Direct) WritableMemory.
+   * @param readOnly True if memory is to be treated as read only.
+   */
+  protected IntersectionImpl(final WritableMemory wmem, final long seed, final boolean dstMemFlag,
+      final boolean readOnly) {
+    readOnly_ = readOnly;
+    if (wmem != null) {
+      wmem_ = wmem;
+      if (dstMemFlag) { //DstMem: compute & store seedHash, no seedhash checking
+        checkMinSizeMemory(wmem);
+        maxLgArrLongs_ = !readOnly ? getMaxLgArrLongs(wmem) : 0; //Only Off Heap
+        seedHash_ = computeSeedHash(seed);
+        wmem_.putShort(SEED_HASH_SHORT, seedHash_);
+      } else { //SrcMem:gets and stores the seedHash, checks mem_seedHash against the seed
+        seedHash_ = wmem_.getShort(SEED_HASH_SHORT);
+        Util.checkSeedHashes(seedHash_, computeSeedHash(seed)); //check for seed hash conflict
+        maxLgArrLongs_ = 0;
+      }
+    } else { //compute & store seedHash
+      wmem_ = null;
+      maxLgArrLongs_ = 0;
+      seedHash_ = computeSeedHash(seed);
+    }
   }
 
   /**
-   * Construct a new Intersection target on the java heap.
+   * Factory: Construct a new Intersection target on the java heap.
+   * Called by SetOperationBuilder, test.
    *
    * @param seed <a href="{@docRoot}/resources/dictionary.html#seed">See Seed</a>
    * @return a new IntersectionImpl on the Java heap
    */
   static IntersectionImpl initNewHeapInstance(final long seed) {
-    //compute & store seedHash,
-    final IntersectionImpl impl = new IntersectionImpl(null, seed, false);
-    impl.lgArrLongs_ = 0;
-    impl.curCount_ = -1;  //Universal Set is true
-    impl.thetaLong_ = Long.MAX_VALUE;
-    impl.empty_ = false;  //A virgin intersection represents the Universal Set so empty is FALSE!
-    impl.hashTable_ = null; //retained entries of the intersection as a hash table, on-heap only.
+    final boolean dstMemFlag = false;
+    final boolean readOnly = false;
+    final IntersectionImpl impl = new IntersectionImpl(null, seed, dstMemFlag, readOnly);
+    impl.hardReset();
     return impl;
   }
 
   /**
-   * Construct a new Intersection target direct to the given destination Memory.
-   * Called by SetOperation.Builder.
+   * Factory: Construct a new Intersection target direct to the given destination Memory.
+   * Called by SetOperationBuilder, test.
    *
    * @param seed <a href="{@docRoot}/resources/dictionary.html#seed">See Seed</a>
-   * @param dstMem destination Memory.
+   * @param dstMem destination Memory
    * <a href="{@docRoot}/resources/dictionary.html#mem">See Memory</a>
    * @return a new IntersectionImpl that may be off-heap
    */
   static IntersectionImpl initNewDirectInstance(final long seed, final WritableMemory dstMem) {
-    //DstMem: compute & store seedHash, true means no seedhash checking
-    final IntersectionImpl impl = new IntersectionImpl(dstMem, seed, true); //compute & store seedHash
-
     //Load Preamble
-
+    //Pre0
+    dstMem.clear(0, CONST_PREAMBLE_LONGS << 3);
     insertPreLongs(dstMem, CONST_PREAMBLE_LONGS); //RF not used = 0
     insertSerVer(dstMem, SER_VER);
     insertFamilyID(dstMem, Family.INTERSECTION.getID());
-    insertLgNomLongs(dstMem, 0); //Note: Intersection does not use lgNomLongs or k, per se.
-    //set lgArrLongs initially to minimum.  Don't clear cache in mem
-    insertLgArrLongs(dstMem, MIN_LG_ARR_LONGS);
-    insertFlags(dstMem, 0); //bigEndian = readOnly = compact = ordered = empty = false;
-    //seedHash loaded and checked in private constructor
-    insertCurCount(dstMem, -1);
-    insertP(dstMem, (float) 1.0);
-    //pre2
-    insertThetaLong(dstMem, Long.MAX_VALUE);
-
-    //internal initialize
-    impl.lgArrLongs_ = MIN_LG_ARR_LONGS;
-    impl.curCount_ = -1; //set in mem below
-    impl.thetaLong_ = Long.MAX_VALUE;
-    impl.empty_ = false;
-    impl.maxLgArrLongs_ = getMaxLgArrLongs(dstMem); //Only Off Heap
+    //lgNomLongs not used by Intersection
+    //lgArrLongs set by hardReset
+    //flags are already 0: bigEndian = readOnly = compact = ordered = empty = false;
+    //seedHash loaded and checked in IntersectionImpl constructor
+    //Pre1
+    //CurCount set by hardReset
+    insertP(dstMem, (float) 1.0); //not used by intersection
+    //Pre2
+    //thetaLong set by hardReset
 
+    //Initialize
+    final boolean dstMemFlag = true;
+    final boolean readOnly = false;
+    final IntersectionImpl impl = new IntersectionImpl(dstMem, seed, dstMemFlag, readOnly);
+    impl.hardReset();
     return impl;
   }
 
   /**
-   * Heapify an intersection target from a Memory image containing data.
+   * Factory: Heapify an intersection target from a Memory image containing data.
    * @param srcMem The source Memory object.
    * <a href="{@docRoot}/resources/dictionary.html#mem">See Memory</a>
    * @param seed <a href="{@docRoot}/resources/dictionary.html#seed">See seed</a>
-   * @return a IntersectionImplR instance on the Java heap
+   * @return a IntersectionImpl instance on the Java heap
    */
-  static IntersectionImplR heapifyInstance(final Memory srcMem, final long seed) {
-    //compute & store seedHash,
-    final IntersectionImpl impl = new IntersectionImpl(null, seed, false);
-
-    //Get Preamble
-    //Note: Intersection does not use lgNomLongs (or k), per se.
-    //seedHash loaded and checked in private constructor
-    final int preLongsMem = srcMem.getByte(PREAMBLE_LONGS_BYTE) & 0X3F;
-    final int serVer = srcMem.getByte(SER_VER_BYTE) & 0XFF;
-    final int famID = srcMem.getByte(FAMILY_BYTE) & 0XFF;
-    final int lgArrLongs = srcMem.getByte(LG_ARR_LONGS_BYTE) & 0XFF;
-    final int flags = srcMem.getByte(FLAGS_BYTE) & 0XFF;
-    final int curCount = srcMem.getInt(RETAINED_ENTRIES_INT);
-    final long thetaLong = srcMem.getLong(THETA_LONG);
-    final boolean empty = (flags & EMPTY_FLAG_MASK) > 0;
-
-    //Checks
-    if (preLongsMem != CONST_PREAMBLE_LONGS) {
-      throw new SketchesArgumentException(
-          "Memory PreambleLongs must equal " + CONST_PREAMBLE_LONGS + ": " + preLongsMem);
-    }
-
-    if (serVer != SER_VER) {
-      throw new SketchesArgumentException("Serialization Version must equal " + SER_VER);
-    }
-
-    Family.INTERSECTION.checkFamilyID(famID);
-
-    if (empty) {
-      if (curCount != 0) {
-        throw new SketchesArgumentException(
-            "srcMem empty state inconsistent with curCount: " + empty + "," + curCount);
-      }
-      //empty = true AND curCount_ = 0: OK
-    }
+  static IntersectionImpl heapifyInstance(final Memory srcMem, final long seed) {
+    final boolean dstMemFlag = false;
+    final boolean readOnly = false;
+    final IntersectionImpl impl = new IntersectionImpl(null, seed, dstMemFlag, readOnly);
+    memChecks(srcMem);
 
     //Initialize
-    impl.lgArrLongs_ = lgArrLongs;
-    impl.curCount_ = curCount;
-    impl.thetaLong_ = thetaLong;
-    impl.empty_ = empty;
-
-    if (!empty) {
-      if (curCount > 0) { //can't be virgin, empty, or curCount == 0
-        impl.hashTable_ = new long[1 << lgArrLongs];
-        srcMem.getLongArray(CONST_PREAMBLE_LONGS << 3, impl.hashTable_, 0, 1 << lgArrLongs);
+    impl.lgArrLongs_ = extractLgArrLongs(srcMem);
+    impl.curCount_ = extractCurCount(srcMem);
+    impl.thetaLong_ = extractThetaLong(srcMem);
+    impl.empty_ = (extractFlags(srcMem) & EMPTY_FLAG_MASK) > 0;
+    if (!impl.empty_) {
+      if (impl.curCount_ > 0) {
+        impl.hashTable_ = new long[1 << impl.lgArrLongs_];
+        srcMem.getLongArray(CONST_PREAMBLE_LONGS << 3, impl.hashTable_, 0, 1 << impl.lgArrLongs_);
       }
     }
     return impl;
   }
 
   /**
-   * Wrap an Intersection target around the given source WritableMemory containing intersection data.
+   * Factory: Wrap an Intersection target around the given source WritableMemory containing
+   * intersection data.
    * @param srcMem The source WritableMemory image.
    * <a href="{@docRoot}/resources/dictionary.html#mem">See Memory</a>
    * @param seed <a href="{@docRoot}/resources/dictionary.html#seed">See seed</a>
+   * @param readOnly True if memory is to be treated as read only
    * @return a IntersectionImpl that wraps a source WritableMemory that contains an Intersection image
    */
-  static IntersectionImpl wrapInstance(final WritableMemory srcMem, final long seed) {
-    //SrcMem:gets and stores the seedHash, checks mem_seedHash against the seed
-    final IntersectionImpl impl = new IntersectionImpl(srcMem, seed, false);
-    return (IntersectionImpl) internalWrapInstance(srcMem, impl);
+  static IntersectionImpl wrapInstance(
+      final WritableMemory srcMem,
+      final long seed,
+      final boolean readOnly) {
+    final boolean dstMemFlag = false;
+    final IntersectionImpl impl = new IntersectionImpl(srcMem, seed, dstMemFlag, readOnly);
+    memChecks(srcMem);
+    impl.lgArrLongs_ = extractLgArrLongs(srcMem);
+    impl.curCount_ = extractCurCount(srcMem);
+    impl.thetaLong_ = extractThetaLong(srcMem);
+    impl.empty_ = (extractFlags(srcMem) & EMPTY_FLAG_MASK) > 0;
+    return impl;
   }
 
-  @Deprecated
   @Override
-  public void update(final Sketch sketchIn) {
-    intersect(sketchIn);
+  public CompactSketch intersect(final Sketch a, final Sketch b, final boolean dstOrdered,
+     final WritableMemory dstMem) {
+    if ((wmem_ != null) && readOnly_) { throw new SketchesReadOnlyException(); }
+    hardReset();
+    intersect(a);
+    intersect(b);
+    final CompactSketch csk = getResult(dstOrdered, dstMem);
+    hardReset();
+    return csk;
   }
 
   @Override
@@ -199,23 +233,15 @@ final class IntersectionImpl extends IntersectionImplR {
     if (sketchIn == null) {
       throw new SketchesArgumentException("Intersection argument must not be null.");
     }
+    if ((wmem_ != null) && readOnly_) { throw new SketchesReadOnlyException(); }
     if (empty_ || sketchIn.isEmpty()) { //empty rule
       //Because of the def of null above and the Empty Rule (which is OR), empty_ must be true.
-      //Whatever the current internal state, we make it empty.
-      lgArrLongs_ = 0;
-      curCount_ = 0;
-      thetaLong_ = Long.MAX_VALUE;
-      empty_ = true;
-      hashTable_ = null;
-      if (wmem_ != null) {
-        setEmpty(wmem_); //true
-        insertThetaLong(wmem_, thetaLong_);
-        insertCurCount(wmem_, 0);
-        insertLgArrLongs(wmem_, lgArrLongs_);
-      }
+      //Whatever the current internal state, we make our local empty.
+      resetToEmpty();
       return;
     }
     Util.checkSeedHashes(seedHash_, sketchIn.getSeedHash());
+    //Set minTheta
     thetaLong_ = min(thetaLong_, sketchIn.getThetaLong()); //Theta rule
     empty_ = false;
     if (wmem_ != null) {
@@ -223,114 +249,187 @@ final class IntersectionImpl extends IntersectionImplR {
       clearEmpty(wmem_); //false
     }
 
-    final int sketchInEntries = sketchIn.getRetainedEntries(true);
-
-    // The truth table for the following state machine
+    // The truth table for the following state machine. MinTheta is set above.
+    // Incoming sketch is not null and not empty, but could have 0 count and Theta < 1.0
     //   Case  curCount  sketchInEntries | Actions
-    //     1      <0            0        | First intersect, curCount < 0; HT = null; exit
-    //     2       0            0        | CurCount = 0; HT = null; exit
-    //     3      >0            0        | CurCount = 0; HT = null; exit
+    //     1      <0            0        | First intersect, set curCount = 0; HT = null; minTh; exit
+    //     2       0            0        | set curCount = 0; HT = null; minTh; exit
+    //     3      >0            0        | set curCount = 0; HT = null; minTh; exit
     //     4                             | Not used
     //     5      <0           >0        | First intersect, clone SketchIn; exit
-    //     6       0           >0        | CurCount = 0; HT = null; exit
+    //     6       0           >0        | set curCount = 0; HT = null; minTh; exit
     //     7      >0           >0        | Perform full intersect
-    final int sw = ((curCount_ < 0) ? 1 : (curCount_ == 0) ? 2 : 3)
-        | (((sketchInEntries > 0) ? 1 : 0) << 2) ;
-    switch (sw) {
-      case 1:
-      case 2:
-      case 3:
-      case 6: { //(curCount_ == 0) || (sketchInEntries == 0)
-        //All future intersections result in zero data, but theta can still be reduced.
-        curCount_ = 0;
-        if (wmem_ != null) { insertCurCount(wmem_, 0); }
-        hashTable_ = null; //No need for a HT. Don't bother clearing mem if valid
-        break;
-      }
-      case 5: { // curCount_ < 0; This is the 1st intersect, clone the incoming sketch
-        curCount_ = sketchIn.getRetainedEntries(true);
-        final int requiredLgArrLongs = minLgHashTableSize(curCount_, REBUILD_THRESHOLD);
-        final int priorLgArrLongs = lgArrLongs_; //prior only used in error message
-        lgArrLongs_ = requiredLgArrLongs;
-
-        if (wmem_ != null) { //Off heap, check if current dstMem is large enough
-          insertCurCount(wmem_, curCount_);
-          insertLgArrLongs(wmem_, lgArrLongs_);
-          if (requiredLgArrLongs <= maxLgArrLongs_) { //OK
-            wmem_.clear(CONST_PREAMBLE_LONGS << 3, 8 << lgArrLongs_); //clear only what required
-          }
-          else { //not enough space in dstMem
-            final int requiredBytes = (8 << requiredLgArrLongs) + 24;
-            final int givenBytes = (8 << priorLgArrLongs) + 24;
-            throw new SketchesArgumentException(
-                "Insufficient internal Memory space: " + requiredBytes + " > " + givenBytes);
-          }
+    final int sketchInEntries = sketchIn.getRetainedEntries(true);
+
+    //states 1,2,3,6
+    if ((curCount_ == 0) || (sketchInEntries == 0)) {
+      curCount_ = 0;
+      if (wmem_ != null) { insertCurCount(wmem_, 0); }
+      hashTable_ = null; //No need for a HT. Don't bother clearing mem if valid
+    } //end of states 1,2,3,6
+
+    // state 5
+    else if ((curCount_ < 0) && (sketchInEntries > 0)) {
+      curCount_ = sketchIn.getRetainedEntries(true);
+      final int requiredLgArrLongs = minLgHashTableSize(curCount_, REBUILD_THRESHOLD);
+      final int priorLgArrLongs = lgArrLongs_; //prior only used in error message
+      lgArrLongs_ = requiredLgArrLongs;
+
+      if (wmem_ != null) { //Off heap, check if current dstMem is large enough
+        insertCurCount(wmem_, curCount_);
+        insertLgArrLongs(wmem_, lgArrLongs_);
+        if (requiredLgArrLongs <= maxLgArrLongs_) {
+          wmem_.clear(CONST_PREAMBLE_LONGS << 3, 8 << lgArrLongs_); //clear only what required
         }
-        else { //On the heap, allocate a HT
-          hashTable_ = new long[1 << lgArrLongs_];
+        else { //not enough space in dstMem
+          final int requiredBytes = (8 << requiredLgArrLongs) + 24;
+          final int givenBytes = (8 << priorLgArrLongs) + 24;
+          throw new SketchesArgumentException(
+              "Insufficient internal Memory space: " + requiredBytes + " > " + givenBytes);
         }
-        moveDataToTgt(sketchIn.getCache(), curCount_);
-        break;
       }
-      case 7: { // (curCount > 0) && (sketchInEntries > 0); Perform full intersect
-        //Sets resulting hashTable, curCount and adjusts lgArrLongs
-        performIntersect(sketchIn);
-        break;
+      else { //On the heap, allocate a HT
+        hashTable_ = new long[1 << lgArrLongs_];
       }
-      //default: assert false : "Should not happen";
+      moveDataToTgt(sketchIn.getCache(), curCount_);
+    } //end of state 5
+
+    //state 7
+    else if ((curCount_ > 0) && (sketchInEntries > 0)) {
+      //Sets resulting hashTable, curCount and adjusts lgArrLongs
+      performIntersect(sketchIn);
+    } //end of state 7
+
+    else {
+      assert false : "Should not happen";
     }
   }
 
   @Override
-  public CompactSketch intersect(final Sketch a, final Sketch b, final boolean dstOrdered,
-     final WritableMemory dstMem) {
-    reset();
-    intersect(a);
-    intersect(b);
-    return getResult(dstOrdered, dstMem);
+  public CompactSketch getResult(final boolean dstOrdered, final WritableMemory dstMem) {
+    if (curCount_ < 0) {
+      throw new SketchesStateException(
+          "Calling getResult() with no intervening intersections would represent the infinite set, "
+          + "which is not a legal result.");
+    }
+    long[] compactCache;
+    final boolean srcOrdered, srcCompact;
+    if (curCount_ == 0) {
+      compactCache = new long[0];
+      srcCompact = true;
+      srcOrdered = false; //hashTable, even tho empty
+      return CompactOperations.componentsToCompact(
+          thetaLong_, curCount_, seedHash_, empty_, srcCompact, srcOrdered, dstOrdered,
+          dstMem, compactCache);
+    }
+    //else curCount > 0
+    final long[] hashTable;
+    if (wmem_ != null) {
+      final int htLen = 1 << lgArrLongs_;
+      hashTable = new long[htLen];
+      wmem_.getLongArray(CONST_PREAMBLE_LONGS << 3, hashTable, 0, htLen);
+    } else {
+      hashTable = hashTable_;
+    }
+    compactCache = compactCachePart(hashTable, lgArrLongs_, curCount_, thetaLong_, dstOrdered);
+    srcCompact = true;
+    srcOrdered = dstOrdered;
+    return CompactOperations.componentsToCompact(
+        thetaLong_, curCount_, seedHash_, empty_, srcCompact, srcOrdered, dstOrdered,
+        dstMem, compactCache);
   }
 
   @Override
   public void reset() {
-    lgArrLongs_ = 0;
-    curCount_ = -1;
-    thetaLong_ = Long.MAX_VALUE;
-    empty_ = false;
-    hashTable_ = null;
-    if (wmem_ != null) {
-      insertLgArrLongs(wmem_, 0);
-      insertCurCount(wmem_, -1);
-      insertThetaLong(wmem_, Long.MAX_VALUE);
-      clearEmpty(wmem_);
-    }
+    hardReset();
   }
 
-  private void hardReset() { //Universal Set
-    resetToEmpty();
-    curCount_ = -1;
-    empty_ = false;
+  @Override
+  public byte[] toByteArray() {
+    final int preBytes = CONST_PREAMBLE_LONGS << 3;
+    final int dataBytes = (curCount_ > 0) ? 8 << lgArrLongs_ : 0;
+    final byte[] byteArrOut = new byte[preBytes + dataBytes];
     if (wmem_ != null) {
-      clearEmpty(wmem_);
-      insertCurCount(wmem_, -1);
+      wmem_.getByteArray(0, byteArrOut, 0, preBytes + dataBytes);
     }
+    else {
+      final WritableMemory memOut = WritableMemory.wrap(byteArrOut);
+
+      //preamble
+      memOut.putByte(PREAMBLE_LONGS_BYTE, (byte) CONST_PREAMBLE_LONGS); //RF not used = 0
+      memOut.putByte(SER_VER_BYTE, (byte) SER_VER);
+      memOut.putByte(FAMILY_BYTE, (byte) Family.INTERSECTION.getID());
+      memOut.putByte(LG_NOM_LONGS_BYTE, (byte) 0); //not used
+      memOut.putByte(LG_ARR_LONGS_BYTE, (byte) lgArrLongs_);
+      if (empty_) { memOut.setBits(FLAGS_BYTE, (byte) EMPTY_FLAG_MASK); }
+      else { memOut.clearBits(FLAGS_BYTE, (byte) EMPTY_FLAG_MASK); }
+      memOut.putShort(SEED_HASH_SHORT, seedHash_);
+      memOut.putInt(RETAINED_ENTRIES_INT, curCount_);
+      memOut.putFloat(P_FLOAT, (float) 1.0);
+      memOut.putLong(THETA_LONG, thetaLong_);
+
+      //data
+      if (curCount_ > 0) {
+        memOut.putLongArray(preBytes, hashTable_, 0, 1 << lgArrLongs_);
+      }
+    }
+    return byteArrOut;
   }
 
-  private void resetToEmpty() { //empty state
-    lgArrLongs_ = 0;
-    curCount_ = 0;
-    thetaLong_ = Long.MAX_VALUE;
-    empty_ = true;
-    hashTable_ = null;
-    if (wmem_ != null) {
-      insertLgArrLongs(wmem_, 0);
-      insertCurCount(wmem_, 0);
-      insertThetaLong(wmem_, Long.MAX_VALUE);
-      setEmpty(wmem_); //true
-    }
+  @Override
+  public boolean hasResult() {
+    return (wmem_ != null) ? wmem_.getInt(RETAINED_ENTRIES_INT) >= 0 : curCount_ >= 0;
+  }
+
+  @Override
+  public boolean isSameResource(final Memory that) {
+    return (wmem_ != null) ? wmem_.isSameResource(that) : false;
+  }
+
+  @Deprecated
+  @Override
+  public void update(final Sketch sketchIn) {
+    intersect(sketchIn);
   }
 
   //restricted
 
+  /**
+   * Gets the number of retained entries from this operation. If negative, it is interpreted
+   * as the infinite <i>Universal Set</i>.
+   */
+  @Override
+  int getRetainedEntries() {
+    return curCount_;
+  }
+
+  @Override
+  boolean isEmpty() {
+    return empty_;
+  }
+
+  @Override
+  long[] getCache() {
+    if (wmem_ == null) {
+      return (hashTable_ != null) ? hashTable_ : new long[0];
+    }
+    //Direct
+    final int arrLongs = 1 << lgArrLongs_;
+    final long[] outArr = new long[arrLongs];
+    wmem_.getLongArray(CONST_PREAMBLE_LONGS << 3, outArr, 0, arrLongs);
+    return outArr;
+  }
+
+  @Override
+  short getSeedHash() {
+    return seedHash_;
+  }
+
+  @Override
+  long getThetaLong() {
+    return thetaLong_;
+  }
+
   private void performIntersect(final Sketch sketchIn) {
     // curCount and input data are nonzero, match against HT
     assert ((curCount_ > 0) && (!empty_));
@@ -415,4 +514,35 @@ final class IntersectionImpl extends IntersectionImplR {
     assert (tmpCnt == count) : "Intersection Count Check: got: " + tmpCnt + ", expected: " + count;
   }
 
+  private void hardReset() {
+    resetCommon();
+    if (wmem_ != null) {
+      insertCurCount(wmem_, -1); //Universal Set
+      clearEmpty(wmem_); //false
+    }
+    curCount_ = -1; //Universal Set
+    empty_ = false;
+  }
+
+  private void resetToEmpty() {
+    resetCommon();
+    if (wmem_ != null) {
+      insertCurCount(wmem_, 0);
+      setEmpty(wmem_); //true
+    }
+    curCount_ = 0;
+    empty_ = true;
+  }
+
+  private void resetCommon() {
+    if (wmem_ != null) {
+      if (readOnly_) { throw new SketchesReadOnlyException(); }
+      wmem_.clear(CONST_PREAMBLE_LONGS << 3, 8 << MIN_LG_ARR_LONGS);
+      insertLgArrLongs(wmem_, MIN_LG_ARR_LONGS);
+      insertThetaLong(wmem_, Long.MAX_VALUE);
+    }
+    lgArrLongs_ = MIN_LG_ARR_LONGS;
+    thetaLong_ = Long.MAX_VALUE;
+    hashTable_ = null;
+  }
 }
diff --git a/src/main/java/org/apache/datasketches/theta/IntersectionImplR.java b/src/main/java/org/apache/datasketches/theta/IntersectionImplR.java
deleted file mode 100644
index 658d293..0000000
--- a/src/main/java/org/apache/datasketches/theta/IntersectionImplR.java
+++ /dev/null
@@ -1,340 +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 org.apache.datasketches.Util.MIN_LG_ARR_LONGS;
-import static org.apache.datasketches.Util.computeSeedHash;
-import static org.apache.datasketches.Util.floorPowerOf2;
-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.LG_ARR_LONGS_BYTE;
-import static org.apache.datasketches.theta.PreambleUtil.LG_NOM_LONGS_BYTE;
-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;
-import static org.apache.datasketches.theta.PreambleUtil.SEED_HASH_SHORT;
-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;
-import org.apache.datasketches.SketchesStateException;
-import org.apache.datasketches.Util;
-import org.apache.datasketches.memory.Memory;
-import org.apache.datasketches.memory.WritableMemory;
-
-/**
- * Intersection operation for Theta Sketches.
- *
- * <p>This implementation uses data either on-heap or off-heap in a given Memory
- * that is owned and managed by the caller.
- * The off-heap Memory, which if managed properly, will greatly reduce the need for
- * the JVM to perform garbage collection.</p>
- *
- * @author Lee Rhodes
- * @author Kevin Lang
- */
-class IntersectionImplR extends Intersection {
-  protected final short seedHash_;
-  protected final boolean readOnly_;
-  protected final WritableMemory wmem_;
-
-  //Note: Intersection does not use lgNomLongs or k, per se.
-  protected int lgArrLongs_; //current size of hash table
-  protected int curCount_; //curCount of HT, if < 0 means Universal Set (US) is true
-  protected long thetaLong_;
-  protected boolean empty_;
-
-  protected long[] hashTable_ = null;  //HT => Data.  Only used On Heap
-  protected int maxLgArrLongs_ = 0; //max size of wmem_ hash table. Only used Off Heap, never reset.
-
-  /**
-   * Constructor.
-   * @param wmem Can be either a Source(e.g. wrap) or Destination (new Direct) WritableMemory
-   * @param seed Used to validate incoming sketch arguments
-   * @param dstMemFlag The given memory is a Destination (new Direct) WritableMemory
-   */
-  protected IntersectionImplR(final WritableMemory wmem, final long seed, final boolean dstMemFlag) {
-    readOnly_ = !dstMemFlag;
-    if (wmem != null) {
-      wmem_ = wmem;
-      if (dstMemFlag) { //DstMem: compute & store seedHash, newMem means no seedhash checking
-        checkMinSizeMemory(wmem);
-        seedHash_ = computeSeedHash(seed);
-        wmem_.putShort(SEED_HASH_SHORT, seedHash_);
-      } else { //SrcMem:gets and stores the seedHash, checks mem_seedHash against the seed
-        seedHash_ = wmem_.getShort(SEED_HASH_SHORT);
-        Util.checkSeedHashes(seedHash_, computeSeedHash(seed)); //check for seed hash conflict
-      }
-    } else { //compute & store seedHash
-      wmem_ = null;
-      seedHash_ = computeSeedHash(seed);
-    }
-  }
-
-  /**
-   * Wrap an Intersection target around the given source Memory containing intersection data.
-   * @See SetOperation#wrap(Memory, long)
-   * @param srcMem The source Memory image.
-   * <a href="{@docRoot}/resources/dictionary.html#mem">See Memory</a>
-   * @param seed <a href="{@docRoot}/resources/dictionary.html#seed">See seed</a>
-   * @return an IntersectionImplR that wraps a read-only Intersection image
-   */
-  static IntersectionImplR wrapInstance(final Memory srcMem, final long seed) {
-    //SrcMem:gets and stores the seedHash, checks mem_seedHash against the seed
-    final IntersectionImplR impl = new IntersectionImplR((WritableMemory) srcMem, seed, false);
-    return internalWrapInstance(srcMem, impl);
-  }
-
-  static IntersectionImplR internalWrapInstance(final Memory srcMem, final IntersectionImplR impl) {
-    //Get Preamble
-    //Note: Intersection does not use lgNomLongs (or k), per se.
-    //seedHash loaded and checked in constructor
-    final int preLongsMem = srcMem.getByte(PREAMBLE_LONGS_BYTE) & 0X3F;
-    final int serVer = srcMem.getByte(SER_VER_BYTE) & 0XFF;
-    final int famID = srcMem.getByte(FAMILY_BYTE) & 0XFF;
-    final int lgArrLongs = srcMem.getByte(LG_ARR_LONGS_BYTE) & 0XFF;
-    final int flags = srcMem.getByte(FLAGS_BYTE) & 0XFF;
-    final boolean empty = (flags & EMPTY_FLAG_MASK) > 0;
-
-    //Checks
-    if (preLongsMem != CONST_PREAMBLE_LONGS) {
-      throw new SketchesArgumentException(
-          "Memory PreambleLongs must equal " + CONST_PREAMBLE_LONGS + ": " + preLongsMem);
-    }
-
-    if (serVer != SER_VER) {
-      throw new SketchesArgumentException("Serialization Version must equal " + SER_VER);
-    }
-
-    Family.INTERSECTION.checkFamilyID(famID);
-
-    final int curCount = srcMem.getInt(RETAINED_ENTRIES_INT);
-    final long thetaLong = srcMem.getLong(THETA_LONG);
-
-    if (empty) {
-      if (curCount != 0) {
-        throw new SketchesArgumentException(
-            "srcMem empty state inconsistent with curCount: " + empty + "," + curCount);
-      }
-      //empty = true AND curCount_ = 0: OK
-    } //else empty = false, curCount could be anything
-
-    //Initialize
-    impl.lgArrLongs_ = lgArrLongs;
-    impl.curCount_ = curCount;
-    impl.thetaLong_ = thetaLong;
-    impl.empty_ = empty;
-    impl.maxLgArrLongs_ = getMaxLgArrLongs(srcMem); //Only Off Heap, check for min size
-    return impl;
-  }
-
-  @Override
-  public CompactSketch getResult(final boolean dstOrdered, final WritableMemory dstMem) {
-    if (curCount_ < 0) {
-      throw new SketchesStateException(
-          "Calling getResult() with no intervening intersections would represent the infinite set, "
-          + "which is not a legal result.");
-    }
-    long[] compactCacheR;
-
-    if (curCount_ == 0) {
-      compactCacheR = new long[0];
-      return CompactOperations.componentsToCompact(
-          thetaLong_, curCount_, seedHash_, empty_, true, false, dstOrdered, dstMem, compactCacheR);
-    }
-    //else curCount > 0
-    final long[] hashTable;
-    if (wmem_ != null) {
-      final int htLen = 1 << lgArrLongs_;
-      hashTable = new long[htLen];
-      wmem_.getLongArray(CONST_PREAMBLE_LONGS << 3, hashTable, 0, htLen);
-    } else {
-      hashTable = hashTable_;
-    }
-    compactCacheR = compactCachePart(hashTable, lgArrLongs_, curCount_, thetaLong_, dstOrdered);
-
-    return CompactOperations.componentsToCompact(
-        thetaLong_, curCount_, seedHash_, empty_, true, dstOrdered, dstOrdered, dstMem, compactCacheR);
-  }
-
-  /**
-   * Gets the number of retained entries from this operation. If negative, it is interpreted
-   * as the infinite <i>Universal Set</i>.
-   */
-  @Override
-  int getRetainedEntries() {
-    return curCount_;
-  }
-
-  @Override
-  public boolean hasResult() {
-    return (wmem_ != null) ? wmem_.getInt(RETAINED_ENTRIES_INT) >= 0 : curCount_ >= 0;
-  }
-
-  @Override
-  boolean isEmpty() {
-    return empty_;
-  }
-
-  @Override
-  public boolean isSameResource(final Memory that) {
-    return (wmem_ != null) ? wmem_.isSameResource(that) : false;
-  }
-
-  @Override
-  public void reset() {
-    throw new SketchesReadOnlyException();
-  }
-
-  @Override
-  public byte[] toByteArray() {
-    final int preBytes = CONST_PREAMBLE_LONGS << 3;
-    final int dataBytes = (curCount_ > 0) ? 8 << lgArrLongs_ : 0;
-    final byte[] byteArrOut = new byte[preBytes + dataBytes];
-    if (wmem_ != null) {
-      wmem_.getByteArray(0, byteArrOut, 0, preBytes + dataBytes);
-    }
-    else {
-      final WritableMemory memOut = WritableMemory.wrap(byteArrOut);
-
-      //preamble
-      memOut.putByte(PREAMBLE_LONGS_BYTE, (byte) CONST_PREAMBLE_LONGS); //RF not used = 0
-      memOut.putByte(SER_VER_BYTE, (byte) SER_VER);
-      memOut.putByte(FAMILY_BYTE, (byte) Family.INTERSECTION.getID());
-      memOut.putByte(LG_NOM_LONGS_BYTE, (byte) 0); //not used
-      memOut.putByte(LG_ARR_LONGS_BYTE, (byte) lgArrLongs_);
-      if (empty_) {
-        memOut.setBits(FLAGS_BYTE, (byte) EMPTY_FLAG_MASK);
-      }
-      else {
-        memOut.clearBits(FLAGS_BYTE, (byte) EMPTY_FLAG_MASK);
-      }
-      memOut.putShort(SEED_HASH_SHORT, seedHash_);
-      memOut.putInt(RETAINED_ENTRIES_INT, curCount_);
-      memOut.putFloat(P_FLOAT, (float) 1.0);
-      memOut.putLong(THETA_LONG, thetaLong_);
-
-      //data
-      if (curCount_ > 0) {
-        memOut.putLongArray(preBytes, hashTable_, 0, 1 << lgArrLongs_);
-      }
-    }
-    return byteArrOut;
-  }
-
-  @Override
-  public void intersect(final Sketch sketchIn) {
-    throw new SketchesReadOnlyException();
-  }
-
-  @Deprecated
-  @Override
-  public void update(final Sketch sketchIn) {
-    throw new SketchesReadOnlyException();
-  }
-
-  @Override
-  public CompactSketch intersect(final Sketch a, final Sketch b, final boolean dstOrdered,
-     final WritableMemory dstMem) {
-    throw new SketchesReadOnlyException();
-  }
-
-  //restricted
-
-  @Override
-  long[] getCache() {
-    if (wmem_ == null) {
-      return (hashTable_ != null) ? hashTable_ : new long[0];
-    }
-    //Direct
-    final int arrLongs = 1 << lgArrLongs_;
-    final long[] outArr = new long[arrLongs];
-    wmem_.getLongArray(CONST_PREAMBLE_LONGS << 3, outArr, 0, arrLongs);
-    return outArr;
-  }
-
-  @Override
-  short getSeedHash() {
-    return seedHash_;
-  }
-
-  @Override
-  long getThetaLong() {
-    return thetaLong_;
-  }
-
-  /**
-   * Returns the maximum lgArrLongs given the capacity of the Memory.
-   * @param dstMem the given Memory
-   * @return the maximum lgArrLongs given the capacity of the Memory
-   */
-  static final int getMaxLgArrLongs(final Memory dstMem) {
-    final int preBytes = CONST_PREAMBLE_LONGS << 3;
-    final long cap = dstMem.getCapacity();
-    return Integer.numberOfTrailingZeros(floorPowerOf2((int)(cap - preBytes)) >>> 3);
-  }
-
-  static final void checkMinSizeMemory(final Memory mem) {
-    final int minBytes = (CONST_PREAMBLE_LONGS << 3) + (8 << MIN_LG_ARR_LONGS);//280
-    final long cap = mem.getCapacity();
-    if (cap < minBytes) {
-      throw new SketchesArgumentException(
-          "Memory must be at least " + minBytes + " bytes. Actual capacity: " + cap);
-    }
-  }
-
-  /**
-   * 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/SetOperation.java b/src/main/java/org/apache/datasketches/theta/SetOperation.java
index 837d1a9..cba59ad 100644
--- a/src/main/java/org/apache/datasketches/theta/SetOperation.java
+++ b/src/main/java/org/apache/datasketches/theta/SetOperation.java
@@ -137,7 +137,7 @@ public abstract class SetOperation {
         return UnionImpl.wrapInstance(srcMem, seed);
       }
       case INTERSECTION : {
-        return IntersectionImplR.wrapInstance(srcMem, seed);
+        return IntersectionImpl.wrapInstance((WritableMemory)srcMem, seed, true);
       }
       default:
         throw new SketchesArgumentException("SetOperation cannot wrap family: " + family.toString());
@@ -185,7 +185,7 @@ public abstract class SetOperation {
         return UnionImpl.wrapInstance(srcMem, seed);
       }
       case INTERSECTION : {
-        return IntersectionImpl.wrapInstance(srcMem, seed);
+        return IntersectionImpl.wrapInstance(srcMem, seed, false);
       }
       default:
         throw new SketchesArgumentException("SetOperation cannot wrap family: "
diff --git a/src/main/java/org/apache/datasketches/tuple/Intersection.java b/src/main/java/org/apache/datasketches/tuple/Intersection.java
index 5dd5ea2..c10cad7 100644
--- a/src/main/java/org/apache/datasketches/tuple/Intersection.java
+++ b/src/main/java/org/apache/datasketches/tuple/Intersection.java
@@ -257,7 +257,6 @@ public class Intersection<S extends Summary> {
         if (summaryTable_ == null) {
           summaryTable_ = (S[]) Array.newInstance(mySummary.getClass(), 1 << lgTableSize_);
         }
-        hashTable_[index] = hash;
         summaryTable_[index] = mySummary;
       }
     }
@@ -276,7 +275,6 @@ public class Intersection<S extends Summary> {
         if (summaryTable_ == null) {
           summaryTable_ = (S[]) Array.newInstance(mySummary.getClass(), 1 << lgTableSize_);
         }
-        hashTable_[index] = hash;
         summaryTable_[index] = mySummary;
       }
     }
@@ -296,7 +294,6 @@ public class Intersection<S extends Summary> {
         if (summaryTable_ == null) {
           summaryTable_ = (S[]) Array.newInstance(mySummary.getClass(), 1 << lgTableSize_);
         }
-        hashTable_[index] = hash;
         summaryTable_[index] = summaryArr[i];
       }
     }
diff --git a/src/test/java/org/apache/datasketches/theta/CompactSketchTest.java b/src/test/java/org/apache/datasketches/theta/CompactSketchTest.java
index 5e0d26f..3c4b49f 100644
--- a/src/test/java/org/apache/datasketches/theta/CompactSketchTest.java
+++ b/src/test/java/org/apache/datasketches/theta/CompactSketchTest.java
@@ -216,7 +216,7 @@ public class CompactSketchTest {
   @Test
   public void checkCompactCachePart() {
     //phony values except for curCount = 0.
-    long[] result = IntersectionImplR.compactCachePart(null, 4, 0, 0L, false);
+    long[] result = Intersection.compactCachePart(null, 4, 0, 0L, false);
     assertEquals(result.length, 0);
   }
 
diff --git a/src/test/java/org/apache/datasketches/theta/DirectIntersectionTest.java b/src/test/java/org/apache/datasketches/theta/DirectIntersectionTest.java
index 062c190..3a7683c 100644
--- a/src/test/java/org/apache/datasketches/theta/DirectIntersectionTest.java
+++ b/src/test/java/org/apache/datasketches/theta/DirectIntersectionTest.java
@@ -691,7 +691,7 @@ public class DirectIntersectionTest {
     //cheap trick
     int k = 16;
     WritableMemory mem = WritableMemory.wrap(new byte[(k*16) + PREBYTES]);
-    IntersectionImplR impl = IntersectionImpl.initNewDirectInstance(DEFAULT_UPDATE_SEED, mem);
+    IntersectionImpl impl = IntersectionImpl.initNewDirectInstance(DEFAULT_UPDATE_SEED, mem);
     assertEquals(impl.getFamily(), Family.INTERSECTION);
   }
 
@@ -702,7 +702,7 @@ public class DirectIntersectionTest {
     IntersectionImpl.initNewDirectInstance(DEFAULT_UPDATE_SEED, mem);
     //corrupt SerVer
     mem.putByte(PreambleUtil.SER_VER_BYTE, (byte) 2);
-    IntersectionImplR.wrapInstance(mem, DEFAULT_UPDATE_SEED);
+    IntersectionImpl.wrapInstance(mem, DEFAULT_UPDATE_SEED, false);
   }
 
   @Test(expectedExceptions = SketchesArgumentException.class)
@@ -714,7 +714,7 @@ public class DirectIntersectionTest {
     //corrupt empty and CurCount
     mem.setBits(PreambleUtil.FLAGS_BYTE, (byte) PreambleUtil.EMPTY_FLAG_MASK);
     mem.putInt(PreambleUtil.RETAINED_ENTRIES_INT, 2);
-    IntersectionImplR.wrapInstance(mem, DEFAULT_UPDATE_SEED);
+    IntersectionImpl.wrapInstance(mem, DEFAULT_UPDATE_SEED, false);
   }
 
   //Check Alex's bug intersecting 2 direct full sketches with only overlap of 2


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