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:12:01 UTC

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

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