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 2023/01/13 21:57:15 UTC

[datasketches-memory] 01/03: Got Heap working, but Map, Direct and BB still have errors.

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

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

commit 51a93aff3e657979a159c6edaa26808d5da79ed3
Author: Lee Rhodes <le...@users.noreply.github.com>
AuthorDate: Sun Jan 8 15:56:34 2023 -0800

    Got Heap working, but Map, Direct and BB still have errors.
---
 .../org/apache/datasketches/memory/BaseState.java  | 174 ++++----
 .../apache/datasketches/memory/MurmurHash3v2.java  |   2 +-
 .../apache/datasketches/memory/WritableMemory.java |   6 +-
 .../memory/internal/BaseBufferImpl.java            |   8 +-
 .../memory/internal/BaseStateImpl.java             | 441 ++++++++++-----------
 .../memory/internal/BaseWritableBufferImpl.java    |  15 +-
 .../memory/internal/BaseWritableMemoryImpl.java    |  47 ++-
 .../internal/HeapNonNativeWritableBufferImpl.java  | 108 +++--
 .../internal/HeapNonNativeWritableMemoryImpl.java  |  91 +++--
 .../memory/internal/HeapWritableBufferImpl.java    | 108 +++--
 .../memory/internal/HeapWritableMemoryImpl.java    |  93 +++--
 .../memory/internal/MapWritableMemoryImpl.java     |   6 +-
 .../memory/internal/NativeWritableBufferImpl.java  |   7 +-
 .../memory/internal/NativeWritableMemoryImpl.java  |   7 +-
 .../internal/NonNativeWritableBufferImpl.java      |   7 +-
 .../internal/NonNativeWritableMemoryImpl.java      |   7 +-
 .../internal/AllocateDirectMapMemoryTest.java      |  35 +-
 .../internal/NativeWritableBufferImplTest.java     |   1 -
 .../internal/NativeWritableMemoryImplTest.java     |   4 +-
 .../memory/internal/SpecificLeafTest.java          |  89 +++--
 20 files changed, 711 insertions(+), 545 deletions(-)

diff --git a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/BaseState.java b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/BaseState.java
index 2ec75cb..2da19c4 100644
--- a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/BaseState.java
+++ b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/BaseState.java
@@ -36,23 +36,49 @@ public interface BaseState {
    */
   static final MemoryRequestServer defaultMemReqSvr = null; //new DefaultMemoryRequestServer();
 
-  //Byte Order Related
+  //Monitoring
 
   /**
-   * Gets the current Type ByteOrder.
-   * This may be different from the ByteOrder of the backing resource and of the Native Byte Order.
-   * @return the current Type ByteOrder.
+   * Gets the current size of active direct memory allocated.
+   * @return the current size of active direct memory allocated.
    */
-  ByteOrder getTypeByteOrder();
+  static long getCurrentDirectMemoryAllocated() {
+    return BaseStateImpl.getCurrentDirectMemoryAllocated();
+  }
 
   /**
-   * Returns true if the Native ByteOrder is the same as the ByteOrder of the
-   * current Buffer or Memory and the same ByteOrder as the given byteOrder.
-   * @param byteOrder the given ByteOrder
-   * @return true if the Native ByteOrder is the same as the ByteOrder of the
-   * current Buffer or Memory and the same ByteOrder as the given byteOrder.
+   * Gets the current number of active direct memory allocations.
+   * @return the current number of active direct memory allocations.
    */
-  boolean isByteOrderCompatible(ByteOrder byteOrder);
+  static long getCurrentDirectMemoryAllocations() {
+    return BaseStateImpl.getCurrentDirectMemoryAllocations();
+  }
+
+  /**
+   * Gets the current size of active direct memory map allocated.
+   * @return the current size of active direct memory map allocated.
+   */
+  static long getCurrentDirectMemoryMapAllocated() {
+    return BaseStateImpl.getCurrentDirectMemoryMapAllocated();
+  }
+
+  /**
+   * Gets the current number of active direct memory map allocations.
+   * @return the current number of active direct memory map allocations.
+   */
+  static long getCurrentDirectMemoryMapAllocations() {
+    return BaseStateImpl.getCurrentDirectMemoryMapAllocations();
+  }
+  //End Monitoring
+
+  /**
+   * Checks that the specified range of bytes is within bounds of this object, throws
+   * {@link IllegalArgumentException} if it's not: i. e. if offsetBytes &lt; 0, or length &lt; 0,
+   * or offsetBytes + length &gt; {@link #getCapacity()}.
+   * @param offsetBytes the given offset in bytes of this object
+   * @param lengthBytes the given length in bytes of this object
+   */
+  void checkValidAndBounds(long offsetBytes, long lengthBytes);
 
   /**
    * Returns true if the given object is an instance of this class and has equal data contents.
@@ -99,18 +125,21 @@ public interface BaseState {
 
   /**
    * Gets the cumulative offset in bytes of this object from the backing resource including the given
-   * offsetBytes. This offset may also include other offset components such as the native off-heap
-   * memory address, DirectByteBuffer split offsets, region offsets, and unsafe arrayBaseOffsets.
+   * localOffsetBytes. This offset may also include other offset components such as the native off-heap
+   * memory address, DirectByteBuffer split offsets, region offsets, and object arrayBaseOffsets.
    *
-   * @param offsetBytes offset to be added to the cumulative offset.
+   * @param localOffsetBytes offset to be added to the cumulative offset.
    * @return the cumulative offset in bytes of this object from the backing resource including the
    * given offsetBytes.
    */
-  long getCumulativeOffset(long offsetBytes);
+  default long getCumulativeOffset(long localOffsetBytes) {
+    return getCumulativeOffset() + localOffsetBytes;
+  }
 
   /**
    * Returns the offset of address zero of this object relative to the address zero of the
    * backing resource but not including the size of any Java object header.
+   * This is only used when moving directly across to Memory, Buffer or Buffer duplicate.
    * @return the offset of address zero of this object relative to the address zero of the
    * backing resource but not including the size of any Java object header.
    */
@@ -125,7 +154,16 @@ public interface BaseState {
    * backing resource plus the given offsetBytes but not including the size of any Java object
    * header.
    */
-  long getRegionOffset(long offsetBytes);
+  default long getRegionOffset(long offsetBytes) {
+    return getRegionOffset() + offsetBytes;
+  }
+
+  /**
+   * Gets the current Type ByteOrder.
+   * This may be different from the ByteOrder of the backing resource and of the Native Byte Order.
+   * @return the current Type ByteOrder.
+   */
+  ByteOrder getTypeByteOrder();
 
   /**
    * Returns true if this object is backed by an on-heap primitive array
@@ -133,6 +171,12 @@ public interface BaseState {
    */
   boolean hasArray();
 
+  /**
+   * Returns true if this Memory is backed by a ByteBuffer.
+   * @return true if this Memory is backed by a ByteBuffer.
+   */
+  boolean hasByteBuffer();
+
   /**
    * Returns the hashCode of this object.
    *
@@ -149,32 +193,13 @@ public interface BaseState {
   int hashCode();
 
   /**
-   * Returns the 64-bit hash of the sequence of bytes in this object specified by
-   * <i>offsetBytes</i>, <i>lengthBytes</i> and a <i>seed</i>.  Note that the sequence of bytes is
-   * always processed in the same order independent of endianness.
-   *
-   * @param offsetBytes the given offset in bytes to the first byte of the byte sequence.
-   * @param lengthBytes the given length in bytes of the byte sequence.
-   * @param seed the given long seed.
-   * @return the 64-bit hash of the sequence of bytes in this object specified by
-   * <i>offsetBytes</i> and <i>lengthBytes</i>.
-   */
-  long xxHash64(long offsetBytes, long lengthBytes, long seed);
-
-  /**
-   * Returns a 64-bit hash from a single long. This method has been optimized for speed when only
-   * a single hash of a long is required.
-   * @param in A long.
-   * @param seed A long valued seed.
-   * @return the hash.
-   */
-  long xxHash64(long in, long seed);
-
-  /**
-   * Returns true if this Memory is backed by a ByteBuffer.
-   * @return true if this Memory is backed by a ByteBuffer.
+   * Returns true if the Native ByteOrder is the same as the ByteOrder of the
+   * current Buffer or Memory and the same ByteOrder as the given byteOrder.
+   * @param byteOrder the given ByteOrder
+   * @return true if the Native ByteOrder is the same as the ByteOrder of the
+   * current Buffer or Memory and the same ByteOrder as the given byteOrder.
    */
-  boolean hasByteBuffer();
+  boolean isByteOrderCompatible(ByteOrder byteOrder);
 
   /**
    * Returns true if the backing resource is direct (off-heap) memory.
@@ -206,51 +231,6 @@ public interface BaseState {
    */
   boolean isValid();
 
-  /**
-   * Checks that the specified range of bytes is within bounds of this object, throws
-   * {@link IllegalArgumentException} if it's not: i. e. if offsetBytes &lt; 0, or length &lt; 0,
-   * or offsetBytes + length &gt; {@link #getCapacity()}.
-   * @param offsetBytes the given offset in bytes of this object
-   * @param lengthBytes the given length in bytes of this object
-   */
-  void checkValidAndBounds(long offsetBytes, long lengthBytes);
-
-  //Monitoring
-
-  /**
-   * Gets the current number of active direct memory allocations.
-   * @return the current number of active direct memory allocations.
-   */
-  static long getCurrentDirectMemoryAllocations() {
-    return BaseStateImpl.getCurrentDirectMemoryAllocations();
-  }
-
-  /**
-   * Gets the current size of active direct memory allocated.
-   * @return the current size of active direct memory allocated.
-   */
-  static long getCurrentDirectMemoryAllocated() {
-    return BaseStateImpl.getCurrentDirectMemoryAllocated();
-  }
-
-  /**
-   * Gets the current number of active direct memory map allocations.
-   * @return the current number of active direct memory map allocations.
-   */
-  static long getCurrentDirectMemoryMapAllocations() {
-    return BaseStateImpl.getCurrentDirectMemoryMapAllocations();
-  }
-
-  /**
-   * Gets the current size of active direct memory map allocated.
-   * @return the current size of active direct memory map allocated.
-   */
-  static long getCurrentDirectMemoryMapAllocated() {
-    return BaseStateImpl.getCurrentDirectMemoryMapAllocated();
-  }
-
-  //TO STRING
-
   /**
    * Returns a formatted hex string of a range of this object.
    * Used primarily for testing.
@@ -261,4 +241,26 @@ public interface BaseState {
    */
   String toHexString(String header, long offsetBytes, int lengthBytes);
 
+  /**
+   * Returns the 64-bit hash of the sequence of bytes in this object specified by
+   * <i>offsetBytes</i>, <i>lengthBytes</i> and a <i>seed</i>.  Note that the sequence of bytes is
+   * always processed in the same order independent of endianness.
+   *
+   * @param offsetBytes the given offset in bytes to the first byte of the byte sequence.
+   * @param lengthBytes the given length in bytes of the byte sequence.
+   * @param seed the given long seed.
+   * @return the 64-bit hash of the sequence of bytes in this object specified by
+   * <i>offsetBytes</i> and <i>lengthBytes</i>.
+   */
+  long xxHash64(long offsetBytes, long lengthBytes, long seed);
+
+  /**
+   * Returns a 64-bit hash from a single long. This method has been optimized for speed when only
+   * a single hash of a long is required.
+   * @param in A long.
+   * @param seed A long valued seed.
+   * @return the hash.
+   */
+  long xxHash64(long in, long seed);
+
 }
diff --git a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/MurmurHash3v2.java b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/MurmurHash3v2.java
index 9c5a35b..67c4830 100644
--- a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/MurmurHash3v2.java
+++ b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/MurmurHash3v2.java
@@ -177,7 +177,7 @@ public final class MurmurHash3v2 {
     if (uObj == null) {
       throw new IllegalArgumentException("The backing resource of input mem is not on-heap.");
     }
-    long cumOff = mem.getCumulativeOffset() + offsetBytes;
+    long cumOff = mem.getCumulativeOffset(offsetBytes);
 
     long h1 = seed;
     long h2 = seed;
diff --git a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/WritableMemory.java b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/WritableMemory.java
index cdc1635..feacb1b 100644
--- a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/WritableMemory.java
+++ b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/WritableMemory.java
@@ -162,12 +162,12 @@ public interface WritableMemory extends Memory {
    * <li>Returned object's capacity = <i>capacityBytes</i></li>
    * </ul>
    *
-   * @param offsetBytes the starting offset with respect to this object. It must be &ge; 0.
+   * @param regionOffsetBytes the starting offset with respect to this object. It must be &ge; 0.
    * @param capacityBytes the capacity of the returned object in bytes. It must be &ge; 0.
    * @return a new <i>WritableMemory</i> representing the defined writable region.
    */
-  default WritableMemory writableRegion(long offsetBytes, long capacityBytes) {
-    return writableRegion(offsetBytes, capacityBytes, getTypeByteOrder());
+  default WritableMemory writableRegion(long regionOffsetBytes, long capacityBytes) {
+    return writableRegion(regionOffsetBytes, capacityBytes, getTypeByteOrder());
   }
 
   /**
diff --git a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/BaseBufferImpl.java b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/BaseBufferImpl.java
index 7bc1340..0e815df 100644
--- a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/BaseBufferImpl.java
+++ b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/BaseBufferImpl.java
@@ -45,13 +45,13 @@ public abstract class BaseBufferImpl extends BaseStateImpl implements BaseBuffer
   private long pos = 0;
   private long end;
 
-  //Pass-through ctor
-  BaseBufferImpl(final Object unsafeObj, final long nativeBaseOffset,
-      final long regionOffset, final long capacityBytes) {
-    super(unsafeObj, nativeBaseOffset, regionOffset, capacityBytes);
+  //Pass-through constructor
+  BaseBufferImpl(final long capacityBytes) {
+    super();
     capacity = end = capacityBytes;
   }
 
+
   @Override
   public final BaseBufferImpl incrementPosition(final long increment) {
     incrementAndAssertPositionForRead(pos, increment);
diff --git a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/BaseStateImpl.java b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/BaseStateImpl.java
index 42062c1..8c64e58 100644
--- a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/BaseStateImpl.java
+++ b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/BaseStateImpl.java
@@ -50,6 +50,7 @@ public abstract class BaseStateImpl implements BaseState {
   //class type IDs. Do not change the bit orders
   //The first 3 bits are set dynamically
   // 0000 0XXX
+  static final int WRITABLE = 0;
   static final int READONLY = 1;
   static final int REGION = 2;
   static final int DUPLICATE = 4;
@@ -59,7 +60,6 @@ public abstract class BaseStateImpl implements BaseState {
   static final int HEAP = 0;
   static final int DIRECT = 1 << 3;
   static final int MAP = 2 << 3;
-  static final int BYTEBUF = 3 << 3;
 
   // 00X0 0000
   static final int NATIVE = 0;
@@ -69,57 +69,54 @@ public abstract class BaseStateImpl implements BaseState {
   static final int MEMORY = 0;
   static final int BUFFER = 1 << 6;
 
-  private final long capacityBytes_;
-
+  // X000 0000
+  static final int BYTEBUF = 1 << 7;
   /**
-   * This becomes the base offset used by all Unsafe calls. It is cumulative in that in includes
-   * all offsets from regions, user-defined offsets when creating MemoryImpl, and the array object
-   * header offset when creating MemoryImpl from primitive arrays.
+   * The root of the Memory inheritance hierarchy
    */
-  private final long cumBaseOffset_;
+  BaseStateImpl() { }
 
-  /**
-   *
-   * @param unsafeObj The primitive backing array. It may be null. Used by Unsafe calls.
-   * @param nativeBaseOffset The off-heap memory address including DirectByteBuffer split offsets.
-   * @param regionOffset This offset defines address zero of this object (usually a region)
-   * relative to address zero of the backing resource. It is used to compute cumBaseOffset.
-   * This will be loaded from heap ByteBuffers, which have a similar field used for slices.
-   * It is used by region() and writableRegion().
-   * This offset does not include the size of an object array header, if there is one.
-   * @param capacityBytes the capacity of this object. Used by all methods when checking bounds.
-   */
-  BaseStateImpl(final Object unsafeObj, final long nativeBaseOffset, final long regionOffset,
-      final long capacityBytes) {
-    capacityBytes_ = capacityBytes;
-    cumBaseOffset_ = regionOffset + (unsafeObj == null
-        ? nativeBaseOffset
-        : UnsafeUtil.getArrayBaseOffset(unsafeObj.getClass()));
+  final void assertValid() {
+    assert isValid() : "MemoryImpl not valid.";
   }
 
-  //Byte Order Related
+  final void assertValidAndBoundsForRead(final long offsetBytes, final long lengthBytes) {
+    assertValid();
+    // capacityBytes_ is intentionally read directly instead of calling getCapacity()
+    // because the later can make JVM to not inline the assert code path (and entirely remove it)
+    // even though it does nothing in production code path.
+    assertBounds(offsetBytes, lengthBytes, getCapacity());
+  }
 
-  @Override
-  public final ByteOrder getTypeByteOrder() {
-    return isNonNativeType() ? Util.NON_NATIVE_BYTE_ORDER : ByteOrder.nativeOrder();
+  final void assertValidAndBoundsForWrite(final long offsetBytes, final long lengthBytes) {
+    assertValid();
+    // capacityBytes_ is intentionally read directly instead of calling getCapacity()
+    // because the later can make JVM to not inline the assert code path (and entirely remove it)
+    // even though it does nothing in production code path.
+    assertBounds(offsetBytes, lengthBytes, getCapacity());
+    assert !isReadOnly() : "MemoryImpl is read-only.";
   }
 
-  /**
-   * Returns true if the given byteOrder is the same as the native byte order.
-   * @param byteOrder the given byte order
-   * @return true if the given byteOrder is the same as the native byte order.
-   */
-  public static boolean isNativeByteOrder(final ByteOrder byteOrder) {
-    if (byteOrder == null) {
-      throw new IllegalArgumentException("ByteOrder parameter cannot be null.");
+  void checkValid() {
+    if (!isValid()) {
+      throw new IllegalStateException("MemoryImpl not valid.");
     }
-    return ByteOrder.nativeOrder() == byteOrder;
   }
 
   @Override
-  public final boolean isByteOrderCompatible(final ByteOrder byteOrder) {
-    final ByteOrder typeBO = getTypeByteOrder();
-    return typeBO == ByteOrder.nativeOrder() && typeBO == byteOrder;
+  public final void checkValidAndBounds(final long offsetBytes, final long lengthBytes) {
+    checkValid();
+    //read capacityBytes_ directly to eliminate extra checkValid() call
+    checkBounds(offsetBytes, lengthBytes, getCapacity());
+  }
+
+  final void checkValidAndBoundsForWrite(final long offsetBytes, final long lengthBytes) {
+    checkValid();
+    //read capacityBytes_ directly to eliminate extra checkValid() call
+    checkBounds(offsetBytes, lengthBytes, getCapacity());
+    if (isReadOnly()) {
+      throw new ReadOnlyException("MemoryImpl is read-only.");
+    }
   }
 
   @Override
@@ -144,44 +141,51 @@ public abstract class BaseStateImpl implements BaseState {
     return null;
   }
 
-  @Override
-  public final long getCapacity() {
-    assertValid();
-    return capacityBytes_;
+  //MONITORING
+
+  /**
+   * Gets the current size of active direct memory allocated.
+   * @return the current size of active direct memory allocated.
+   */
+  public static final long getCurrentDirectMemoryAllocated() {
+    return BaseStateImpl.currentDirectMemoryAllocated_.get();
   }
 
-  @Override
-  public final long getCumulativeOffset() {
-    assertValid();
-    return cumBaseOffset_;
+  /**
+   * Gets the current number of active direct memory allocations.
+   * @return the current number of active direct memory allocations.
+   */
+  public static final long getCurrentDirectMemoryAllocations() {
+    return BaseStateImpl.currentDirectMemoryAllocations_.get();
   }
 
-  @Override
-  public final long getCumulativeOffset(final long offsetBytes) {
-    assertValid();
-    return cumBaseOffset_ + offsetBytes;
+  /**
+   * Gets the current size of active direct memory map allocated.
+   * @return the current size of active direct memory map allocated.
+   */
+  public static final long getCurrentDirectMemoryMapAllocated() {
+    return BaseStateImpl.currentDirectMemoryMapAllocated_.get();
+  }
+
+  /**
+   * Gets the current number of active direct memory map allocations.
+   * @return the current number of active direct memory map allocations.
+   */
+  public static final long getCurrentDirectMemoryMapAllocations() {
+    return BaseStateImpl.currentDirectMemoryMapAllocations_.get();
   }
+  //END monitoring
 
   //Documented in WritableMemory and WritableBuffer interfaces.
   //Implemented in the Leaf nodes; Required here by toHex(...).
   abstract MemoryRequestServer getMemoryRequestServer();
 
   //Overridden by ByteBuffer, Direct and Map leafs
-  long getNativeBaseOffset() {
-    return 0;
-  }
-
-  @Override
-  public final long getRegionOffset() {
-    final Object unsafeObj = getUnsafeObject();
-    return unsafeObj == null
-        ? cumBaseOffset_ - getNativeBaseOffset()
-        : cumBaseOffset_ - UnsafeUtil.getArrayBaseOffset(unsafeObj.getClass());
-  }
+  abstract long getNativeBaseOffset();
 
   @Override
-  public final long getRegionOffset(final long offsetBytes) {
-    return getRegionOffset() + offsetBytes;
+  public final ByteOrder getTypeByteOrder() {
+    return isNonNativeType(getTypeId()) ? Util.NON_NATIVE_BYTE_ORDER : ByteOrder.nativeOrder();
   }
 
   //Overridden by all leafs
@@ -193,6 +197,12 @@ public abstract class BaseStateImpl implements BaseState {
     return null;
   }
 
+  @Override
+  public final boolean hasByteBuffer() {
+    assertValid();
+    return getByteBuffer() != null;
+  }
+
   @Override
   public final boolean hasArray() {
     assertValid();
@@ -201,24 +211,21 @@ public abstract class BaseStateImpl implements BaseState {
 
   @Override
   public final int hashCode() {
-    return (int) xxHash64(0, capacityBytes_, 0); //xxHash64() calls checkValid()
+    return (int) xxHash64(0, getCapacity(), 0); //xxHash64() calls checkValid()
   }
 
-  @Override
-  public final long xxHash64(final long offsetBytes, final long lengthBytes, final long seed) {
-    checkValid();
-    return XxHash64.hash(getUnsafeObject(), cumBaseOffset_ + offsetBytes, lengthBytes, seed);
+  final boolean isByteBufferType(final int typeId) {
+    return (typeId & BYTEBUF) > 0;
   }
 
   @Override
-  public final long xxHash64(final long in, final long seed) {
-    return XxHash64.hash(in, seed);
+  public final boolean isByteOrderCompatible(final ByteOrder byteOrder) {
+    final ByteOrder typeBO = getTypeByteOrder();
+    return typeBO == ByteOrder.nativeOrder() && typeBO == byteOrder;
   }
 
-  @Override
-  public final boolean hasByteBuffer() {
-    assertValid();
-    return getByteBuffer() != null;
+  final boolean isBufferType(final int typeId) {
+    return (typeId & BUFFER) > 0;
   }
 
   @Override
@@ -226,174 +233,84 @@ public abstract class BaseStateImpl implements BaseState {
     return getUnsafeObject() == null;
   }
 
-  @Override
-  public final boolean isReadOnly() {
-    assertValid();
-    return isReadOnlyType();
+  final boolean isDirectType(final int typeId) {
+    return (typeId & (MAP | DIRECT)) > 0;
   }
 
-  @Override
-  public final boolean isSameResource(final Object that) {
-    checkValid();
-    if (that == null) { return false; }
-    final BaseStateImpl that1 = (BaseStateImpl) that;
-    that1.checkValid();
-    if (this == that1) { return true; }
-
-    return cumBaseOffset_ == that1.cumBaseOffset_
-            && capacityBytes_ == that1.capacityBytes_
-            && getUnsafeObject() == that1.getUnsafeObject()
-            && getByteBuffer() == that1.getByteBuffer();
+  final boolean isDuplicateType(final int typeId) {
+    return (typeId & DUPLICATE) > 0;
   }
 
-  //Overridden by Direct and Map leafs
-  @Override
-  public boolean isValid() {
-    return true;
+  final boolean isHeapType(final int typeId) {
+    return (typeId & (MAP | DIRECT)) == 0;
   }
 
-  //ASSERTS AND CHECKS
-  final void assertValid() {
-    assert isValid() : "MemoryImpl not valid.";
+  final boolean isMapType(final int typeId) { //not used
+    return (typeId & MAP) > 0;
   }
 
-  void checkValid() {
-    if (!isValid()) {
-      throw new IllegalStateException("MemoryImpl not valid.");
-    }
+  final boolean isMemoryType(final int typeId) { //not used
+    return (typeId & BUFFER) == 0;
   }
 
-  final void assertValidAndBoundsForRead(final long offsetBytes, final long lengthBytes) {
-    assertValid();
-    // capacityBytes_ is intentionally read directly instead of calling getCapacity()
-    // because the later can make JVM to not inline the assert code path (and entirely remove it)
-    // even though it does nothing in production code path.
-    assertBounds(offsetBytes, lengthBytes, capacityBytes_);
+  final boolean isNativeType(final int typeId) { //not used
+    return (typeId & NONNATIVE) == 0;
   }
 
-  final void assertValidAndBoundsForWrite(final long offsetBytes, final long lengthBytes) {
-    assertValid();
-    // capacityBytes_ is intentionally read directly instead of calling getCapacity()
-    // because the later can make JVM to not inline the assert code path (and entirely remove it)
-    // even though it does nothing in production code path.
-    assertBounds(offsetBytes, lengthBytes, capacityBytes_);
-    assert !isReadOnly() : "MemoryImpl is read-only.";
+  final boolean isNonNativeType(final int typeId) {
+    return (typeId & NONNATIVE) > 0;
   }
 
   @Override
-  public final void checkValidAndBounds(final long offsetBytes, final long lengthBytes) {
-    checkValid();
-    //read capacityBytes_ directly to eliminate extra checkValid() call
-    checkBounds(offsetBytes, lengthBytes, capacityBytes_);
-  }
-
-  final void checkValidAndBoundsForWrite(final long offsetBytes, final long lengthBytes) {
-    checkValid();
-    //read capacityBytes_ directly to eliminate extra checkValid() call
-    checkBounds(offsetBytes, lengthBytes, capacityBytes_);
-    if (isReadOnly()) {
-      throw new ReadOnlyException("MemoryImpl is read-only.");
-    }
+  public final boolean isReadOnly() {
+    assertValid();
+    return isReadOnlyType(getTypeId());
   }
 
-  //TYPE ID Management
-  final boolean isReadOnlyType() {
-    return (getTypeId() & READONLY) > 0;
+  final boolean isReadOnlyType(final int typeId) {
+    return (typeId & READONLY) > 0;
   }
 
-  final static byte setReadOnlyType(final byte type, final boolean readOnly) {
-    return (byte)((type & ~1) | (readOnly ? READONLY : 0));
+  final boolean isRegionType(final int typeId) {
+    return (typeId & REGION) > 0;
   }
 
-  final boolean isRegionType() {
-    return (getTypeId() & REGION) > 0;
+  final boolean isWritableType(final int typeId) { //not used
+    return (typeId & READONLY) == 0;
   }
 
-  final boolean isDuplicateType() {
-    return (getTypeId() & DUPLICATE) > 0;
-  }
+  final static int removeNnBuf(final int typeId) { return typeId & ~NONNATIVE & ~BUFFER; }
 
-  //The following are set by the leaf nodes
-  final boolean isBufferType() {
-    return (getTypeId() & BUFFER) > 0;
+  final static int setReadOnlyType(final int typeId, final boolean readOnly) {
+    return readOnly ? typeId | READONLY : typeId & ~READONLY;
   }
 
-  final boolean isNonNativeType() {
-    return (getTypeId() & NONNATIVE) > 0;
-  }
+  final
 
-  final boolean isHeapType() {
-    return (getTypeId() >>> 3 & 3) == 0;
-  }
+  @Override
+  public boolean isSameResource(final Object that) {
+    checkValid();
+    if (that == null) { return false; }
+    final BaseStateImpl that1 = (BaseStateImpl) that;
+    that1.checkValid();
+    if (this == that1) { return true; }
 
-  final boolean isDirectType() {
-    return (getTypeId() >>> 3 & 3) == 1;
+    return getCumulativeOffset(0) == that1.getCumulativeOffset(0)
+            && getCapacity() == that1.getCapacity()
+            && getUnsafeObject() == that1.getUnsafeObject()
+            && getByteBuffer() == that1.getByteBuffer();
   }
 
-  final boolean isMapType() {
-    return (getTypeId() >>> 3 & 3) == 2;
+  //Overridden by Direct and Map leafs
+  @Override
+  public boolean isValid() {
+    return true;
   }
 
-  final boolean isBBType() {
-    return (getTypeId() >>> 3 & 3) == 3;
-  }
+  //REACHABILITY FENCE
+  static void reachabilityFence(final Object obj) { }
 
   //TO STRING
-  /**
-   * Decodes the resource type. This is primarily for debugging.
-   * @param typeId the given typeId
-   * @return a human readable string.
-   */
-  public static final String typeDecode(final int typeId) {
-    final StringBuilder sb = new StringBuilder();
-    final int group1 = typeId & 0x7;
-    switch (group1) {
-      case 1 : sb.append("ReadOnly, "); break;
-      case 2 : sb.append("Region, "); break;
-      case 3 : sb.append("ReadOnly Region, "); break;
-      case 4 : sb.append("Duplicate, "); break;
-      case 5 : sb.append("ReadOnly Duplicate, "); break;
-      case 6 : sb.append("Region Duplicate, "); break;
-      case 7 : sb.append("ReadOnly Region Duplicate, "); break;
-      default: break;
-    }
-    final int group2 = (typeId >>> 3) & 0x3;
-    switch (group2) {
-      case 0 : sb.append("Heap, "); break;
-      case 1 : sb.append("Direct, "); break;
-      case 2 : sb.append("Map, "); break;
-      case 3 : sb.append("ByteBuffer, "); break;
-      default: break;
-    }
-    final int group3 = (typeId >>> 5) & 0x1;
-    switch (group3) {
-      case 0 : sb.append("Native, "); break;
-      case 1 : sb.append("NonNative, "); break;
-      default: break;
-    }
-    final int group4 = (typeId >>> 6) & 0x1;
-    switch (group4) {
-      case 0 : sb.append("Memory"); break;
-      case 1 : sb.append("Buffer"); break;
-      default: break;
-    }
-    return sb.toString();
-  }
-
-  @Override
-  public final String toHexString(final String header, final long offsetBytes,
-      final int lengthBytes) {
-    checkValid();
-    final String klass = this.getClass().getSimpleName();
-    final String s1 = String.format("(..., %d, %d)", offsetBytes, lengthBytes);
-    final long hcode = hashCode() & 0XFFFFFFFFL;
-    final String call = ".toHexString" + s1 + ", hashCode: " + hcode;
-    final StringBuilder sb = new StringBuilder();
-    sb.append("### ").append(klass).append(" SUMMARY ###").append(LS);
-    sb.append("Header Comment      : ").append(header).append(LS);
-    sb.append("Call Parameters     : ").append(call);
-    return toHex(this, sb.toString(), offsetBytes, lengthBytes);
-  }
 
   /**
    * Returns a formatted hex string of an area of this object.
@@ -426,7 +343,7 @@ public abstract class BaseStateImpl implements BaseState {
     final String memReqStr = memReqSvr != null
         ? memReqSvr.getClass().getSimpleName() + ", " + (memReqSvr.hashCode() & 0XFFFFFFFFL)
         : "null";
-    final long cumBaseOffset = state.getCumulativeOffset();
+    final long cumBaseOffset = state.getCumulativeOffset(0);
     sb.append(preamble).append(LS);
     sb.append("UnsafeObj, hashCode : ").append(uObjStr).append(LS);
     sb.append("UnsafeObjHeader     : ").append(uObjHeader).append(LS);
@@ -455,41 +372,91 @@ public abstract class BaseStateImpl implements BaseState {
     return sb.toString();
   }
 
-  //MONITORING
-
-  /**
-   * Gets the current number of active direct memory allocations.
-   * @return the current number of active direct memory allocations.
-   */
-  public static final long getCurrentDirectMemoryAllocations() {
-    return BaseStateImpl.currentDirectMemoryAllocations_.get();
+  @Override
+  public final String toHexString(final String header, final long offsetBytes,
+      final int lengthBytes) {
+    checkValid();
+    final String klass = this.getClass().getSimpleName();
+    final String s1 = String.format("(..., %d, %d)", offsetBytes, lengthBytes);
+    final long hcode = hashCode() & 0XFFFFFFFFL;
+    final String call = ".toHexString" + s1 + ", hashCode: " + hcode;
+    final StringBuilder sb = new StringBuilder();
+    sb.append("### ").append(klass).append(" SUMMARY ###").append(LS);
+    sb.append("Header Comment      : ").append(header).append(LS);
+    sb.append("Call Parameters     : ").append(call);
+    return toHex(this, sb.toString(), offsetBytes, lengthBytes);
   }
 
   /**
-   * Gets the current size of active direct memory allocated.
-   * @return the current size of active direct memory allocated.
+   * Decodes the resource type. This is primarily for debugging.
+   * @param typeId the given typeId
+   * @return a human readable string.
    */
-  public static final long getCurrentDirectMemoryAllocated() {
-    return BaseStateImpl.currentDirectMemoryAllocated_.get();
+  public static final String typeDecode(final int typeId) {
+    final StringBuilder sb = new StringBuilder();
+    final int group1 = typeId & 0x7;
+    switch (group1) {
+      case 1 : sb.append("ReadOnly, "); break;
+      case 2 : sb.append("Region, "); break;
+      case 3 : sb.append("ReadOnly Region, "); break;
+      case 4 : sb.append("Duplicate, "); break;
+      case 5 : sb.append("ReadOnly Duplicate, "); break;
+      case 6 : sb.append("Region Duplicate, "); break;
+      case 7 : sb.append("ReadOnly Region Duplicate, "); break;
+      default: break;
+    }
+    final int group2 = (typeId >>> 3) & 0x3;
+    switch (group2) {
+      case 0 : sb.append("Heap, "); break;
+      case 1 : sb.append("Direct, "); break;
+      case 2 : sb.append("Map, "); break;
+      case 3 : sb.append("ByteBuffer, "); break;
+      default: break;
+    }
+    final int group3 = (typeId >>> 5) & 0x1;
+    switch (group3) {
+      case 0 : sb.append("Native, "); break;
+      case 1 : sb.append("NonNative, "); break;
+      default: break;
+    }
+    final int group4 = (typeId >>> 6) & 0x1;
+    switch (group4) {
+      case 0 : sb.append("Memory"); break;
+      case 1 : sb.append("Buffer"); break;
+      default: break;
+    }
+    return sb.toString();
   }
 
-  /**
-   * Gets the current number of active direct memory map allocations.
-   * @return the current number of active direct memory map allocations.
-   */
-  public static final long getCurrentDirectMemoryMapAllocations() {
-    return BaseStateImpl.currentDirectMemoryMapAllocations_.get();
+  @Override
+  public final long xxHash64(final long offsetBytes, final long lengthBytes, final long seed) {
+    checkValid();
+    return XxHash64.hash(getUnsafeObject(), getCumulativeOffset(0) + offsetBytes, lengthBytes, seed);
   }
 
-  /**
-   * Gets the current size of active direct memory map allocated.
-   * @return the current size of active direct memory map allocated.
-   */
-  public static final long getCurrentDirectMemoryMapAllocated() {
-    return BaseStateImpl.currentDirectMemoryMapAllocated_.get();
+  @Override
+  public final long xxHash64(final long in, final long seed) {
+    return XxHash64.hash(in, seed);
   }
 
-  //REACHABILITY FENCE
-  static void reachabilityFence(final Object obj) { }
+//@Override
+//public final long getCumulativeOffset(final long offsetBytes) {
+//  assertValid();
+//  return cumBaseOffset_ + offsetBytes;
+//}
+
+//@Override
+//public final long getRegionOffset() {
+//  final Object unsafeObj = getUnsafeObject();
+//  final long nativeBaseOff = getNativeBaseOffset();
+//  return unsafeObj == null
+//      ? cumBaseOffset_ - nativeBaseOff
+//      : cumBaseOffset_ - UnsafeUtil.getArrayBaseOffset(unsafeObj.getClass());
+//}
+
+//@Override
+//public final long getRegionOffset(final long offsetBytes) {
+//  return getRegionOffset() + offsetBytes;
+//}
 
 }
diff --git a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/BaseWritableBufferImpl.java b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/BaseWritableBufferImpl.java
index e1dbe93..d52371a 100644
--- a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/BaseWritableBufferImpl.java
+++ b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/BaseWritableBufferImpl.java
@@ -61,11 +61,8 @@ import org.apache.datasketches.memory.WritableMemory;
 @SuppressWarnings("restriction")
 public abstract class BaseWritableBufferImpl extends BaseBufferImpl implements WritableBuffer {
 
-  //Pass-through ctor
-  BaseWritableBufferImpl(final Object unsafeObj, final long nativeBaseOffset,
-      final long regionOffset, final long capacityBytes) {
-    super(unsafeObj, nativeBaseOffset, regionOffset, capacityBytes);
-  }
+  //Pass-through constructor
+  BaseWritableBufferImpl(final long capacityBytes) { super(capacityBytes); }
 
   /**
    * The static constructor that chooses the correct ByteBuffer leaf node based on the byte order.
@@ -154,8 +151,8 @@ public abstract class BaseWritableBufferImpl extends BaseBufferImpl implements W
     if (isReadOnly() && !localReadOnly) {
       throw new ReadOnlyException("Writable duplicate of a read-only Buffer is not allowed.");
     }
-    final boolean readOnly = isReadOnly() || localReadOnly;
-    final WritableBuffer wbuf = toDuplicate(readOnly, byteOrder);
+    final boolean finalReadOnly = isReadOnly() || localReadOnly;
+    final WritableBuffer wbuf = toDuplicate(finalReadOnly, byteOrder);
     wbuf.setStartPositionEnd(getStart(), getPosition(), getEnd());
     return wbuf;
   }
@@ -179,8 +176,8 @@ public abstract class BaseWritableBufferImpl extends BaseBufferImpl implements W
       throw new ReadOnlyException(
           "Converting a read-only Buffer to a writable Memory is not allowed.");
     }
-    final boolean readOnly = isReadOnly() || localReadOnly;
-    final WritableMemory wmem = toWritableMemory(readOnly, byteOrder);
+    final boolean finalReadOnly = isReadOnly() || localReadOnly;
+    final WritableMemory wmem = toWritableMemory(finalReadOnly, byteOrder);
     return wmem;
   }
 
diff --git a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/BaseWritableMemoryImpl.java b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/BaseWritableMemoryImpl.java
index 1e75d7d..8a44c71 100644
--- a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/BaseWritableMemoryImpl.java
+++ b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/BaseWritableMemoryImpl.java
@@ -74,11 +74,8 @@ public abstract class BaseWritableMemoryImpl extends BaseStateImpl implements Wr
     EMPTY_BYTES = new byte[1024];
   }
 
-  //Pass-through ctor
-  BaseWritableMemoryImpl(final Object unsafeObj, final long nativeBaseOffset,
-      final long regionOffset, final long capacityBytes) {
-    super(unsafeObj, nativeBaseOffset, regionOffset, capacityBytes);
-  }
+  //Pass-through constructor
+  BaseWritableMemoryImpl() { }
 
   /**
    * The static constructor that chooses the correct Heap leaf node based on the byte order.
@@ -92,10 +89,11 @@ public abstract class BaseWritableMemoryImpl extends BaseStateImpl implements Wr
    */
   public static BaseWritableMemoryImpl wrapHeapArray(final Object arr, final long offsetBytes, final long lengthBytes,
       final boolean localReadOnly, final ByteOrder byteOrder, final MemoryRequestServer memReqSvr) {
-    final int typeId = localReadOnly ? READONLY : 0;
+    final long cumOffsetBytes = UnsafeUtil.getArrayBaseOffset(arr.getClass()) + offsetBytes;
+    final int typeId = (localReadOnly ? READONLY : 0);
     return Util.isNativeByteOrder(byteOrder)
-        ? new HeapWritableMemoryImpl(arr, offsetBytes, lengthBytes, typeId, memReqSvr)
-        : new HeapNonNativeWritableMemoryImpl(arr, offsetBytes, lengthBytes, typeId, memReqSvr);
+        ? new HeapWritableMemoryImpl(arr, offsetBytes, lengthBytes, typeId, cumOffsetBytes)
+        : new HeapNonNativeWritableMemoryImpl(arr, offsetBytes, lengthBytes, typeId, cumOffsetBytes);
   }
 
   /**
@@ -163,30 +161,31 @@ public abstract class BaseWritableMemoryImpl extends BaseStateImpl implements Wr
 
   //REGIONS
   @Override
-  public Memory region(final long offsetBytes, final long capacityBytes, final ByteOrder byteOrder) {
-    return writableRegionImpl(offsetBytes, capacityBytes, true, byteOrder);
+  public Memory region(final long regionOffsetBytes, final long capacityBytes, final ByteOrder byteOrder) {
+    return writableRegionImpl(regionOffsetBytes, capacityBytes, true, byteOrder);
   }
 
   @Override
-  public WritableMemory writableRegion(final long offsetBytes, final long capacityBytes, final ByteOrder byteOrder) {
-    return writableRegionImpl(offsetBytes, capacityBytes, false, byteOrder);
+  public WritableMemory writableRegion(final long regionOffsetBytes, final long capacityBytes,
+      final ByteOrder byteOrder) {
+    return writableRegionImpl(regionOffsetBytes, capacityBytes, false, byteOrder);
   }
 
-  WritableMemory writableRegionImpl(final long offsetBytes, final long capacityBytes,
+  private WritableMemory writableRegionImpl(final long regionOffsetBytes, final long capacityBytes,
       final boolean localReadOnly, final ByteOrder byteOrder) {
     if (isReadOnly() && !localReadOnly) {
       throw new ReadOnlyException("Writable region of a read-only Memory is not allowed.");
     }
-    negativeCheck(offsetBytes, "offsetBytes must be >= 0");
+    negativeCheck(regionOffsetBytes, "offsetBytes must be >= 0");
     negativeCheck(capacityBytes, "capacityBytes must be >= 0");
     Objects.requireNonNull(byteOrder, "byteOrder must be non-null.");
-    checkValidAndBounds(offsetBytes, capacityBytes);
-    final boolean readOnly = isReadOnly() || localReadOnly;
-    return toWritableRegion(offsetBytes, capacityBytes, readOnly, byteOrder);
+    checkValidAndBounds(regionOffsetBytes, capacityBytes);
+    final boolean finalReadOnly = isReadOnly() || localReadOnly;
+    return toWritableRegion(regionOffsetBytes, capacityBytes, finalReadOnly, byteOrder);
   }
 
   abstract BaseWritableMemoryImpl toWritableRegion(
-      long offsetBytes, long capcityBytes, boolean readOnly, ByteOrder byteOrder);
+      long regionOffsetBytes, long capacityBytes, boolean finalReadOnly, ByteOrder byteOrder);
 
   //AS BUFFER
   @Override
@@ -199,19 +198,19 @@ public abstract class BaseWritableMemoryImpl extends BaseStateImpl implements Wr
     return asWritableBuffer(false, byteOrder);
   }
 
-  WritableBuffer asWritableBuffer(final boolean localReadOnly, final ByteOrder byteOrder) {
+  private WritableBuffer asWritableBuffer(final boolean localReadOnly, final ByteOrder byteOrder) {
     Objects.requireNonNull(byteOrder, "byteOrder must be non-null");
     if (isReadOnly() && !localReadOnly) {
       throw new ReadOnlyException(
           "Converting a read-only Memory to a writable Buffer is not allowed.");
     }
-    final boolean readOnly = isReadOnly() || localReadOnly;
-    final WritableBuffer wbuf = toWritableBuffer(readOnly, byteOrder);
+    final boolean finalReadOnly = isReadOnly() || localReadOnly;
+    final WritableBuffer wbuf = toWritableBuffer(finalReadOnly, byteOrder);
     wbuf.setStartPositionEnd(0, 0, getCapacity());
     return wbuf;
   }
 
-  abstract BaseWritableBufferImpl toWritableBuffer(boolean readOnly, ByteOrder byteOrder);
+  abstract BaseWritableBufferImpl toWritableBuffer(boolean finalReadOnly, ByteOrder byteOrder);
 
   //PRIMITIVE getX() and getXArray()
   @Override
@@ -258,7 +257,7 @@ public abstract class BaseWritableMemoryImpl extends BaseStateImpl implements Wr
   public final int getCharsFromUtf8(final long offsetBytes, final int utf8LengthBytes,
       final Appendable dst) throws IOException, Utf8CodingException {
     checkValidAndBounds(offsetBytes, utf8LengthBytes);
-    return Utf8.getCharsFromUtf8(offsetBytes, utf8LengthBytes, dst, getCumulativeOffset(),
+    return Utf8.getCharsFromUtf8(offsetBytes, utf8LengthBytes, dst, getCumulativeOffset(0),
         getUnsafeObject());
   }
 
@@ -371,7 +370,7 @@ public abstract class BaseWritableMemoryImpl extends BaseStateImpl implements Wr
   @Override
   public final long putCharsToUtf8(final long offsetBytes, final CharSequence src) {
     checkValid();
-    return Utf8.putCharsToUtf8(offsetBytes, src, getCapacity(), getCumulativeOffset(),
+    return Utf8.putCharsToUtf8(offsetBytes, src, getCapacity(), getCumulativeOffset(0),
         getUnsafeObject());
   }
 
diff --git a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/HeapNonNativeWritableBufferImpl.java b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/HeapNonNativeWritableBufferImpl.java
index 1a324f7..b3e9492 100644
--- a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/HeapNonNativeWritableBufferImpl.java
+++ b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/HeapNonNativeWritableBufferImpl.java
@@ -31,66 +31,114 @@ import org.apache.datasketches.memory.WritableBuffer;
  * @author Lee Rhodes
  */
 final class HeapNonNativeWritableBufferImpl extends NonNativeWritableBufferImpl {
-  private static final int id = BUFFER | NONNATIVE | HEAP;
   private final Object unsafeObj;
-  private final MemoryRequestServer memReqSvr;
-  private final byte typeId;
+  private final long offsetBytes;
+  private final long capacityBytes;
+  private final int typeId;
+  private long cumOffsetBytes;
+  private long regionOffsetBytes;
 
   HeapNonNativeWritableBufferImpl(
       final Object unsafeObj,
-      final long regionOffset,
+      final long offsetBytes,
       final long capacityBytes,
       final int typeId,
-      final MemoryRequestServer memReqSvr) {
-    super(unsafeObj, 0L, regionOffset, capacityBytes);
+      final long cumOffsetBytes) {
+    super(capacityBytes);
     this.unsafeObj = unsafeObj;
-    this.memReqSvr = memReqSvr;
-    this.typeId = (byte) (id | (typeId & 0x7));
+    this.offsetBytes = offsetBytes;
+    this.capacityBytes = capacityBytes;
+    this.typeId = removeNnBuf(typeId) | HEAP | BUFFER | NONNATIVE;
+    this.cumOffsetBytes = cumOffsetBytes;
+    this.regionOffsetBytes = 0;
   }
 
   @Override
-  BaseWritableBufferImpl toWritableRegion(final long offsetBytes, final long capacityBytes,
-      final boolean readOnly, final ByteOrder byteOrder) {
-    final int type = setReadOnlyType(typeId, readOnly) | REGION;
-    return Util.isNativeByteOrder(byteOrder)
-        ? new HeapWritableBufferImpl(
-            unsafeObj, getRegionOffset(offsetBytes), capacityBytes, type, memReqSvr)
-        : new HeapNonNativeWritableBufferImpl(
-            unsafeObj, getRegionOffset(offsetBytes), capacityBytes, type, memReqSvr);
+  BaseWritableBufferImpl toWritableRegion(
+      final long regionOffsetBytes,
+      final long capacityBytes,
+      final boolean readOnly,
+      final ByteOrder byteOrder) {
+    final Object unsafeObj = this.unsafeObj;
+    final long newOffsetBytes = this.offsetBytes + this.regionOffsetBytes;
+    this.cumOffsetBytes += this.regionOffsetBytes;
+    int typeIdOut = removeNnBuf(typeId) | BUFFER | REGION | (readOnly ? READONLY : 0);
+    if (Util.isNativeByteOrder(byteOrder)) {
+      typeIdOut |= NATIVE;
+      return new HeapWritableBufferImpl(unsafeObj, newOffsetBytes, capacityBytes, typeIdOut, cumOffsetBytes);
+    } else {
+      typeIdOut |= NONNATIVE;
+      return new HeapNonNativeWritableBufferImpl(unsafeObj, newOffsetBytes, capacityBytes, typeIdOut, cumOffsetBytes);
+    }
   }
 
   @Override
   BaseWritableBufferImpl toDuplicate(final boolean readOnly, final ByteOrder byteOrder) {
-    final int type = setReadOnlyType(typeId, readOnly) | DUPLICATE;
-    return Util.isNativeByteOrder(byteOrder)
-        ? new HeapWritableBufferImpl(
-            unsafeObj, getRegionOffset(), getCapacity(), type, memReqSvr)
-        : new HeapNonNativeWritableBufferImpl(
-            unsafeObj, getRegionOffset(), getCapacity(), type, memReqSvr);
+    int typeIdOut = removeNnBuf(typeId) | BUFFER | DUPLICATE | (readOnly ? READONLY : 0);
+
+    if (byteOrder == ByteOrder.nativeOrder()) {
+      typeIdOut |= NATIVE;
+      return new HeapWritableBufferImpl(
+          unsafeObj, offsetBytes, capacityBytes, typeIdOut, cumOffsetBytes);
+    } else {
+      typeIdOut |= NONNATIVE;
+      return new HeapNonNativeWritableBufferImpl(
+          unsafeObj, regionOffsetBytes, capacityBytes, typeIdOut, cumOffsetBytes);
+    }
   }
 
   @Override
   BaseWritableMemoryImpl toWritableMemory(final boolean readOnly, final ByteOrder byteOrder) {
-    final int type = setReadOnlyType(typeId, readOnly);
-    return Util.isNativeByteOrder(byteOrder)
-        ? new HeapWritableMemoryImpl(
-            unsafeObj, getRegionOffset(), getCapacity(), type, memReqSvr)
-        : new HeapNonNativeWritableMemoryImpl(
-            unsafeObj, getRegionOffset(), getCapacity(), type, memReqSvr);
+    int typeIdOut = removeNnBuf(typeId) | MEMORY | (readOnly ? READONLY : 0);
+
+    if (byteOrder == ByteOrder.nativeOrder()) {
+      typeIdOut |= NATIVE;
+      return new HeapWritableMemoryImpl(
+          unsafeObj, offsetBytes, capacityBytes, typeIdOut, cumOffsetBytes);
+    } else {
+      typeIdOut |= NONNATIVE;
+      return new HeapNonNativeWritableMemoryImpl(
+          unsafeObj, regionOffsetBytes, capacityBytes, typeIdOut, cumOffsetBytes);
+    }
+  }
+
+  @Override
+  public long getCapacity() {
+    assertValid();
+    return capacityBytes;
+  }
+
+  @Override
+  public long getCumulativeOffset() {
+    assertValid();
+    return cumOffsetBytes;
   }
 
   @Override
   public MemoryRequestServer getMemoryRequestServer() {
-    return memReqSvr;
+    return null;
+  }
+
+  @Override
+  public long getNativeBaseOffset() {
+    return 0;
+  }
+
+  @Override
+  public long getRegionOffset() {
+    assertValid();
+    return regionOffsetBytes;
   }
 
   @Override
   int getTypeId() {
-    return typeId & 0xff;
+    assertValid();
+    return typeId;
   }
 
   @Override
   Object getUnsafeObject() {
+    assertValid();
     return unsafeObj;
   }
 
diff --git a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/HeapNonNativeWritableMemoryImpl.java b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/HeapNonNativeWritableMemoryImpl.java
index 91702c5..7ca8627 100644
--- a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/HeapNonNativeWritableMemoryImpl.java
+++ b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/HeapNonNativeWritableMemoryImpl.java
@@ -31,56 +31,99 @@ import org.apache.datasketches.memory.WritableMemory;
  * @author Lee Rhodes
  */
 final class HeapNonNativeWritableMemoryImpl extends NonNativeWritableMemoryImpl {
-  private static final int id = MEMORY | NONNATIVE | HEAP;
   private final Object unsafeObj;
-  private final MemoryRequestServer memReqSvr;
-  private final byte typeId;
+  private final long offsetBytes;
+  private final long capacityBytes;
+  private final int typeId;
+  private long cumOffsetBytes;
+  private long regionOffsetBytes;
 
   HeapNonNativeWritableMemoryImpl(
       final Object unsafeObj,
-      final long regionOffset,
+      final long offsetBytes,
       final long capacityBytes,
       final int typeId,
-      final MemoryRequestServer memReqSvr) {
-    super(unsafeObj, 0L, regionOffset, capacityBytes);
+      final long cumOffsetBytes) {
+    super();
     this.unsafeObj = unsafeObj;
-    this.memReqSvr = memReqSvr;
-    this.typeId = (byte) (id | (typeId & 0x7));
+    this.offsetBytes = offsetBytes;
+    this.capacityBytes = capacityBytes;
+    this.typeId = removeNnBuf(typeId) | HEAP | MEMORY | NONNATIVE;
+    this.cumOffsetBytes = cumOffsetBytes;
+    this.regionOffsetBytes = 0;
   }
 
   @Override
-  BaseWritableMemoryImpl toWritableRegion(final long offsetBytes, final long capacityBytes,
-      final boolean readOnly, final ByteOrder byteOrder) {
-    final int type = setReadOnlyType(typeId, readOnly) | REGION;
-    return Util.isNativeByteOrder(byteOrder)
-        ? new HeapWritableMemoryImpl(
-            unsafeObj, getRegionOffset(offsetBytes), capacityBytes, type, memReqSvr)
-        : new HeapNonNativeWritableMemoryImpl(
-            unsafeObj, getRegionOffset(offsetBytes), capacityBytes, type, memReqSvr);
+  BaseWritableMemoryImpl toWritableRegion(
+      final long regionOffsetBytes,
+      final long capacityBytes,
+      final boolean readOnly,
+      final ByteOrder byteOrder) {
+    final Object unsafeObj = this.unsafeObj;
+    final long newOffsetBytes = this.offsetBytes + this.regionOffsetBytes;
+    this.cumOffsetBytes += this.regionOffsetBytes;
+    int typeIdOut = removeNnBuf(typeId) | MEMORY | REGION | (readOnly ? READONLY : 0);
+    if (Util.isNativeByteOrder(byteOrder)) {
+      typeIdOut |= NATIVE;
+      return new HeapWritableMemoryImpl(unsafeObj, newOffsetBytes, capacityBytes, typeIdOut, cumOffsetBytes);
+    } else {
+      typeIdOut |= NONNATIVE;
+      return new HeapNonNativeWritableMemoryImpl(unsafeObj, newOffsetBytes, capacityBytes, typeIdOut, cumOffsetBytes);
+    }
   }
 
   @Override
   BaseWritableBufferImpl toWritableBuffer(final boolean readOnly, final ByteOrder byteOrder) {
-    final int type = setReadOnlyType(typeId, readOnly);
-    return Util.isNativeByteOrder(byteOrder)
-        ? new HeapWritableBufferImpl(
-            unsafeObj, getRegionOffset(), getCapacity(), type, memReqSvr)
-        : new HeapNonNativeWritableBufferImpl(
-            unsafeObj, getRegionOffset(), getCapacity(), type, memReqSvr);
+    int typeIdOut = removeNnBuf(typeId) | BUFFER | (readOnly ? READONLY : 0);
+
+    if (byteOrder == ByteOrder.nativeOrder()) {
+      typeIdOut |= NATIVE;
+      return new HeapWritableBufferImpl(
+          unsafeObj, offsetBytes, capacityBytes, typeIdOut, cumOffsetBytes);
+    } else {
+      typeIdOut |= NONNATIVE;
+      return new HeapNonNativeWritableBufferImpl(
+          unsafeObj, regionOffsetBytes, capacityBytes, typeIdOut, cumOffsetBytes);
+    }
+  }
+
+  @Override
+  public long getCapacity() {
+    assertValid();
+    return capacityBytes;
+  }
+
+  @Override
+  public long getCumulativeOffset() {
+    assertValid();
+    return cumOffsetBytes;
   }
 
   @Override
   public MemoryRequestServer getMemoryRequestServer() {
-    return memReqSvr;
+    return null;
+  }
+
+  @Override
+  public long getNativeBaseOffset() {
+    return 0;
+  }
+
+  @Override
+  public long getRegionOffset() {
+    assertValid();
+    return regionOffsetBytes;
   }
 
   @Override
   int getTypeId() {
-    return typeId & 0xff;
+    assertValid();
+    return typeId;
   }
 
   @Override
   Object getUnsafeObject() {
+    assertValid();
     return unsafeObj;
   }
 
diff --git a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/HeapWritableBufferImpl.java b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/HeapWritableBufferImpl.java
index 033ad87..d57ae2d 100644
--- a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/HeapWritableBufferImpl.java
+++ b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/HeapWritableBufferImpl.java
@@ -31,66 +31,114 @@ import org.apache.datasketches.memory.WritableBuffer;
  * @author Lee Rhodes
  */
 final class HeapWritableBufferImpl extends NativeWritableBufferImpl {
-  private static final int id = BUFFER | NATIVE | HEAP;
   private final Object unsafeObj;
-  private final MemoryRequestServer memReqSvr;
-  private final byte typeId;
+  private final long offsetBytes;
+  private final long capacityBytes;
+  private final int typeId;
+  private long cumOffsetBytes;
+  private long regionOffsetBytes;
 
   HeapWritableBufferImpl(
       final Object unsafeObj,
-      final long regionOffset,
+      final long offsetBytes,
       final long capacityBytes,
       final int typeId,
-      final MemoryRequestServer memReqSvr) {
-    super(unsafeObj, 0L, regionOffset, capacityBytes);
+      final long cumOffsetBytes) {
+    super(capacityBytes);
     this.unsafeObj = unsafeObj;
-    this.memReqSvr = memReqSvr;
-    this.typeId = (byte) (id | (typeId & 0x7));
+    this.offsetBytes = offsetBytes;
+    this.capacityBytes = capacityBytes;
+    this.typeId = removeNnBuf(typeId) | HEAP | BUFFER | NATIVE;
+    this.cumOffsetBytes = cumOffsetBytes;
+    this.regionOffsetBytes = 0;
   }
 
   @Override
-  BaseWritableBufferImpl toWritableRegion(final long offsetBytes, final long capacityBytes,
-      final boolean readOnly, final ByteOrder byteOrder) {
-    final int type = setReadOnlyType(typeId, readOnly) | REGION;
-    return Util.isNativeByteOrder(byteOrder)
-        ? new HeapWritableBufferImpl(
-            unsafeObj, getRegionOffset(offsetBytes), capacityBytes, type, memReqSvr)
-        : new HeapNonNativeWritableBufferImpl(
-            unsafeObj, getRegionOffset(offsetBytes), capacityBytes, type, memReqSvr);
+  BaseWritableBufferImpl toWritableRegion(
+      final long regionOffsetBytes,
+      final long capacityBytes,
+      final boolean readOnly,
+      final ByteOrder byteOrder) {
+    final Object unsafeObj = this.unsafeObj;
+    final long newOffsetBytes = this.offsetBytes + regionOffsetBytes;
+    this.cumOffsetBytes += regionOffsetBytes;
+    int typeIdOut = removeNnBuf(typeId) | BUFFER | REGION | (readOnly ? READONLY : 0);
+    if (Util.isNativeByteOrder(byteOrder)) {
+      typeIdOut |= NATIVE;
+      return new HeapWritableBufferImpl(unsafeObj, newOffsetBytes, capacityBytes, typeIdOut, cumOffsetBytes);
+    } else {
+      typeIdOut |= NONNATIVE;
+      return new HeapNonNativeWritableBufferImpl(unsafeObj, newOffsetBytes, capacityBytes, typeIdOut, cumOffsetBytes);
+    }
   }
 
   @Override
   BaseWritableBufferImpl toDuplicate(final boolean readOnly, final ByteOrder byteOrder) {
-    final int type = setReadOnlyType(typeId, readOnly) | DUPLICATE;
-    return Util.isNativeByteOrder(byteOrder)
-        ? new HeapWritableBufferImpl(
-            unsafeObj, getRegionOffset(), getCapacity(), type, memReqSvr)
-        : new HeapNonNativeWritableBufferImpl(
-            unsafeObj, getRegionOffset(), getCapacity(), type, memReqSvr);
+    int typeIdOut = removeNnBuf(typeId) | BUFFER | DUPLICATE | (readOnly ? READONLY : 0);
+
+    if (byteOrder == ByteOrder.nativeOrder()) {
+      typeIdOut |= NATIVE;
+      return new HeapWritableBufferImpl(
+          unsafeObj, offsetBytes, capacityBytes, typeIdOut, cumOffsetBytes);
+    } else {
+      typeIdOut |= NONNATIVE;
+      return new HeapNonNativeWritableBufferImpl(
+          unsafeObj, regionOffsetBytes, capacityBytes, typeIdOut, cumOffsetBytes);
+    }
   }
 
   @Override
   BaseWritableMemoryImpl toWritableMemory(final boolean readOnly, final ByteOrder byteOrder) {
-    final int type = setReadOnlyType(typeId, readOnly);
-    return Util.isNativeByteOrder(byteOrder)
-        ? new HeapWritableMemoryImpl(
-            unsafeObj, getRegionOffset(), getCapacity(), type, memReqSvr)
-        : new HeapNonNativeWritableMemoryImpl(
-            unsafeObj, getRegionOffset(), getCapacity(), type, memReqSvr);
+    int typeIdOut = removeNnBuf(typeId) | MEMORY | (readOnly ? READONLY : 0);
+
+    if (byteOrder == ByteOrder.nativeOrder()) {
+      typeIdOut |= NATIVE;
+      return new HeapWritableMemoryImpl(
+          unsafeObj, offsetBytes, capacityBytes, typeIdOut, cumOffsetBytes);
+    } else {
+      typeIdOut |= NONNATIVE;
+      return new HeapNonNativeWritableMemoryImpl(
+          unsafeObj, regionOffsetBytes, capacityBytes, typeIdOut, cumOffsetBytes);
+    }
+  }
+
+  @Override
+  public long getCapacity() {
+    assertValid();
+    return capacityBytes;
+  }
+
+  @Override
+  public long getCumulativeOffset() {
+    assertValid();
+    return cumOffsetBytes;
   }
 
   @Override
   public MemoryRequestServer getMemoryRequestServer() {
-    return memReqSvr;
+    return null;
+  }
+
+  @Override
+  public long getNativeBaseOffset() {
+    return 0;
+  }
+
+  @Override
+  public long getRegionOffset() {
+    assertValid();
+    return regionOffsetBytes;
   }
 
   @Override
   int getTypeId() {
-    return typeId & 0xff;
+    assertValid();
+    return typeId;
   }
 
   @Override
   Object getUnsafeObject() {
+    assertValid();
     return unsafeObj;
   }
 
diff --git a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/HeapWritableMemoryImpl.java b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/HeapWritableMemoryImpl.java
index d2f05d9..85c75e7 100644
--- a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/HeapWritableMemoryImpl.java
+++ b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/HeapWritableMemoryImpl.java
@@ -31,56 +31,99 @@ import org.apache.datasketches.memory.WritableMemory;
  * @author Lee Rhodes
  */
 final class HeapWritableMemoryImpl extends NativeWritableMemoryImpl {
-  private static final int id = MEMORY | NATIVE | HEAP;
   private final Object unsafeObj;
-  private final MemoryRequestServer memReqSvr;
-  private final byte typeId;
+  private final long offsetBytes;
+  private final long capacityBytes;
+  private final int typeId;
+  private long cumOffsetBytes;
+  private long regionOffsetBytes;
 
   HeapWritableMemoryImpl(
       final Object unsafeObj,
-      final long regionOffset,
+      final long offsetBytes,
       final long capacityBytes,
-      final int typeId,
-      final MemoryRequestServer memReqSvr) {
-    super(unsafeObj, 0L, regionOffset, capacityBytes);
+      final int typeId, //if this is RO it stays RO
+      final long cumOffsetBytes) {
+    super();
     this.unsafeObj = unsafeObj;
-    this.memReqSvr = memReqSvr;
-    this.typeId = (byte) (id | (typeId & 0x7));
+    this.offsetBytes = offsetBytes;
+    this.capacityBytes = capacityBytes;
+    this.typeId = removeNnBuf(typeId) | HEAP | MEMORY | NATIVE;
+    this.cumOffsetBytes = cumOffsetBytes;
+    this.regionOffsetBytes = 0;
   }
 
   @Override
-  BaseWritableMemoryImpl toWritableRegion(final long offsetBytes, final long capacityBytes,
-      final boolean readOnly, final ByteOrder byteOrder) {
-    final int type = setReadOnlyType(typeId, readOnly) | REGION;
-    return Util.isNativeByteOrder(byteOrder)
-        ? new HeapWritableMemoryImpl(
-            unsafeObj, getRegionOffset(offsetBytes), capacityBytes, type, memReqSvr)
-        : new HeapNonNativeWritableMemoryImpl(
-            unsafeObj, getRegionOffset(offsetBytes), capacityBytes, type, memReqSvr);
+  BaseWritableMemoryImpl toWritableRegion(
+      final long regionOffsetBytes,
+      final long capacityBytes,
+      final boolean readOnly,
+      final ByteOrder byteOrder) {
+    this.regionOffsetBytes = regionOffsetBytes;
+    final long newOffsetBytes = offsetBytes + regionOffsetBytes;
+    cumOffsetBytes += regionOffsetBytes;
+    int typeIdOut = removeNnBuf(typeId) | MEMORY | REGION | (readOnly ? READONLY : 0);
+    if (Util.isNativeByteOrder(byteOrder)) {
+      typeIdOut |= NATIVE;
+      return new HeapWritableMemoryImpl(unsafeObj, newOffsetBytes, capacityBytes, typeIdOut, cumOffsetBytes);
+    } else {
+      typeIdOut |= NONNATIVE;
+      return new HeapNonNativeWritableMemoryImpl(unsafeObj, newOffsetBytes, capacityBytes, typeIdOut, cumOffsetBytes);
+    }
   }
 
   @Override
   BaseWritableBufferImpl toWritableBuffer(final boolean readOnly, final ByteOrder byteOrder) {
-    final int type = setReadOnlyType(typeId, readOnly);
-    return Util.isNativeByteOrder(byteOrder)
-        ? new HeapWritableBufferImpl(
-            unsafeObj, getRegionOffset(), getCapacity(), type, memReqSvr)
-        : new HeapNonNativeWritableBufferImpl(
-            unsafeObj, getRegionOffset(), getCapacity(), type, memReqSvr);
+    int typeIdOut = removeNnBuf(typeId) | BUFFER | (readOnly ? READONLY : 0);
+
+    if (byteOrder == ByteOrder.nativeOrder()) {
+      typeIdOut |= NATIVE;
+      return new HeapWritableBufferImpl(
+          unsafeObj, offsetBytes, capacityBytes, typeIdOut, cumOffsetBytes);
+    } else {
+      typeIdOut |= NONNATIVE;
+      return new HeapNonNativeWritableBufferImpl(
+          unsafeObj, regionOffsetBytes, capacityBytes, typeIdOut, cumOffsetBytes);
+    }
+  }
+
+  @Override
+  public long getCapacity() {
+    assertValid();
+    return capacityBytes;
+  }
+
+  @Override
+  public long getCumulativeOffset() {
+    assertValid();
+    return cumOffsetBytes;
   }
 
   @Override
   public MemoryRequestServer getMemoryRequestServer() {
-    return memReqSvr;
+    return null;
+  }
+
+  @Override
+  public long getNativeBaseOffset() {
+    return 0;
+  }
+
+  @Override
+  public long getRegionOffset() {
+    assertValid();
+    return regionOffsetBytes;
   }
 
   @Override
   int getTypeId() {
-    return typeId & 0xff;
+    assertValid();
+    return typeId;
   }
 
   @Override
   Object getUnsafeObject() {
+    assertValid();
     return unsafeObj;
   }
 
diff --git a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/MapWritableMemoryImpl.java b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/MapWritableMemoryImpl.java
index 184c1a6..4dbbc42 100644
--- a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/MapWritableMemoryImpl.java
+++ b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/MapWritableMemoryImpl.java
@@ -49,14 +49,14 @@ final class MapWritableMemoryImpl extends NativeWritableMemoryImpl {
   }
 
   @Override
-  BaseWritableMemoryImpl toWritableRegion(final long offsetBytes, final long capacityBytes,
+  BaseWritableMemoryImpl toWritableRegion(final long regionOffsetBytes, final long capacityBytes,
       final boolean readOnly, final ByteOrder byteOrder) {
     final int type = setReadOnlyType(typeId, readOnly) | REGION;
     return Util.isNativeByteOrder(byteOrder)
         ? new MapWritableMemoryImpl(
-            nativeBaseOffset, getRegionOffset(offsetBytes), capacityBytes, type, valid)
+            nativeBaseOffset, getRegionOffset(regionOffsetBytes), capacityBytes, type, valid)
         : new MapNonNativeWritableMemoryImpl(
-            nativeBaseOffset, getRegionOffset(offsetBytes), capacityBytes, type, valid);
+            nativeBaseOffset, getRegionOffset(regionOffsetBytes), capacityBytes, type, valid);
   }
 
   @Override
diff --git a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/NativeWritableBufferImpl.java b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/NativeWritableBufferImpl.java
index 2dd1ce2..3b14993 100644
--- a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/NativeWritableBufferImpl.java
+++ b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/NativeWritableBufferImpl.java
@@ -59,11 +59,8 @@ import org.apache.datasketches.memory.WritableBuffer;
 @SuppressWarnings("restriction")
 abstract class NativeWritableBufferImpl extends BaseWritableBufferImpl {
 
-  //Pass-through ctor
-  NativeWritableBufferImpl(final Object unsafeObj, final long nativeBaseOffset, final long regionOffset,
-      final long capacityBytes) {
-    super(unsafeObj, nativeBaseOffset, regionOffset, capacityBytes);
-  }
+  //Pass-through constructor
+  NativeWritableBufferImpl(final long capacityBytes) { super(capacityBytes); }
 
   //PRIMITIVE getX() and getXArray()
   @Override
diff --git a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/NativeWritableMemoryImpl.java b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/NativeWritableMemoryImpl.java
index c16423e..1a7b375 100644
--- a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/NativeWritableMemoryImpl.java
+++ b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/NativeWritableMemoryImpl.java
@@ -59,11 +59,8 @@ import org.apache.datasketches.memory.WritableMemory;
 @SuppressWarnings("restriction")
 abstract class NativeWritableMemoryImpl extends BaseWritableMemoryImpl {
 
-  //Pass-through ctor
-  NativeWritableMemoryImpl(final Object unsafeObj, final long nativeBaseOffset,
-      final long regionOffset, final long capacityBytes) {
-    super(unsafeObj, nativeBaseOffset, regionOffset, capacityBytes);
-  }
+  //Pass-through constructor
+  NativeWritableMemoryImpl() { }
 
   ///PRIMITIVE getX() and getXArray()
   @Override
diff --git a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/NonNativeWritableBufferImpl.java b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/NonNativeWritableBufferImpl.java
index a8203ba..8df6111 100644
--- a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/NonNativeWritableBufferImpl.java
+++ b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/NonNativeWritableBufferImpl.java
@@ -52,11 +52,8 @@ import org.apache.datasketches.memory.WritableBuffer;
 @SuppressWarnings("restriction")
 abstract class NonNativeWritableBufferImpl extends BaseWritableBufferImpl {
 
-  //Pass-through ctor
-  NonNativeWritableBufferImpl(final Object unsafeObj, final long nativeBaseOffset, final long regionOffset,
-      final long capacityBytes) {
-    super(unsafeObj, nativeBaseOffset, regionOffset, capacityBytes);
-  }
+  //Pass-through constructor
+  NonNativeWritableBufferImpl(final long capacityBytes) { super(capacityBytes); }
 
   //PRIMITIVE getX() and getXArray()
   @Override
diff --git a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/NonNativeWritableMemoryImpl.java b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/NonNativeWritableMemoryImpl.java
index de74333..6d1e460 100644
--- a/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/NonNativeWritableMemoryImpl.java
+++ b/datasketches-memory-java8/src/main/java/org/apache/datasketches/memory/internal/NonNativeWritableMemoryImpl.java
@@ -52,11 +52,8 @@ import org.apache.datasketches.memory.WritableMemory;
 @SuppressWarnings("restriction")
 abstract class NonNativeWritableMemoryImpl extends BaseWritableMemoryImpl {
 
-  //Pass-through ctor
-  NonNativeWritableMemoryImpl(final Object unsafeObj, final long nativeBaseOffset,
-      final long regionOffset, final long capacityBytes) {
-    super(unsafeObj, nativeBaseOffset, regionOffset, capacityBytes);
-  }
+  //Pass-through constructor
+  NonNativeWritableMemoryImpl() { }
 
   ///PRIMITIVE getX() and getXArray()
   @Override
diff --git a/datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/AllocateDirectMapMemoryTest.java b/datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/AllocateDirectMapMemoryTest.java
index 4911e7d..1331781 100644
--- a/datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/AllocateDirectMapMemoryTest.java
+++ b/datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/AllocateDirectMapMemoryTest.java
@@ -23,7 +23,7 @@
 
 package org.apache.datasketches.memory.internal;
 
-import static org.apache.datasketches.memory.internal.Util.*;
+import static org.apache.datasketches.memory.internal.Util.getResourceFile;
 import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertFalse;
 import static org.testng.Assert.assertTrue;
@@ -56,6 +56,37 @@ public class AllocateDirectMapMemoryTest {
     }
   }
 
+  @Test
+  public void simpleMap2() throws Exception {
+    File file = getResourceFile("GettysburgAddress.txt");
+    try (
+        MapHandle rh =
+        Memory.map(file)
+        )
+    {
+      Memory mem = rh.get();
+      println("Mem Cap: " + mem.getCapacity());
+      println("Cum Offset: " + mem.getCumulativeOffset(0));
+      println("Region Offset: " + mem.getRegionOffset());
+      StringBuilder sb = new StringBuilder();
+      mem.getCharsFromUtf8(43, 176, sb);
+      println(sb.toString());
+
+      Memory mem2 = mem.region(38, 12);
+      println("Mem Cap: " + mem2.getCapacity());
+      println("Cum Offset: " + mem2.getCumulativeOffset(0));
+      println("Region Offset: " + mem2.getRegionOffset());
+      StringBuilder sb2 = new StringBuilder();
+      mem2.getCharsFromUtf8(0, 12, sb2);
+      println(sb2.toString());
+
+
+
+      rh.close();
+    }
+  }
+
+
   @Test
   public void testIllegalArguments() throws Exception {
     File file = getResourceFile("GettysburgAddress.txt");
@@ -163,7 +194,7 @@ public class AllocateDirectMapMemoryTest {
    */
   static void print(final Object o) {
     if (o != null) {
-      //System.out.print(o.toString()); //disable here
+      System.out.print(o.toString()); //disable here
     }
   }
 
diff --git a/datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/NativeWritableBufferImplTest.java b/datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/NativeWritableBufferImplTest.java
index 0461104..b1fccf2 100644
--- a/datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/NativeWritableBufferImplTest.java
+++ b/datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/NativeWritableBufferImplTest.java
@@ -478,7 +478,6 @@ public class NativeWritableBufferImplTest {
     WritableBuffer buf = reg.asWritableBuffer();
     assertEquals(buf.getRegionOffset(), 32);
     assertEquals(buf.getRegionOffset(0), 32);
-    assertEquals(buf.getCumulativeOffset(), 32 + 16);
     assertEquals(buf.getCumulativeOffset(0), 32 + 16);
   }
 
diff --git a/datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/NativeWritableMemoryImplTest.java b/datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/NativeWritableMemoryImplTest.java
index 8d52250..e6d13cc 100644
--- a/datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/NativeWritableMemoryImplTest.java
+++ b/datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/NativeWritableMemoryImplTest.java
@@ -27,11 +27,11 @@ import static org.testng.Assert.fail;
 import java.nio.ByteBuffer;
 import java.nio.ByteOrder;
 
-import org.apache.datasketches.memory.WritableHandle;
 import org.apache.datasketches.memory.Buffer;
 import org.apache.datasketches.memory.Memory;
 import org.apache.datasketches.memory.ReadOnlyException;
 import org.apache.datasketches.memory.WritableBuffer;
+import org.apache.datasketches.memory.WritableHandle;
 import org.apache.datasketches.memory.WritableMemory;
 import org.testng.annotations.Test;
 
@@ -656,8 +656,8 @@ public class NativeWritableMemoryImplTest {
     WritableMemory reg = wmem.writableRegion(32, 32);
     assertEquals(reg.getRegionOffset(), 32);
     assertEquals(reg.getRegionOffset(0), 32);
-    assertEquals(reg.getCumulativeOffset(), 32 + 16);
     assertEquals(reg.getCumulativeOffset(0), 32 + 16);
+
   }
 
   @Test
diff --git a/datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/SpecificLeafTest.java b/datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/SpecificLeafTest.java
index 87f7e35..2b96543 100644
--- a/datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/SpecificLeafTest.java
+++ b/datasketches-memory-java8/src/test/java/org/apache/datasketches/memory/internal/SpecificLeafTest.java
@@ -46,7 +46,8 @@ public class SpecificLeafTest {
     bb.order(ByteOrder.nativeOrder());
 
     Memory mem = Memory.wrap(bb).region(0, bytes, ByteOrder.nativeOrder());
-    assertTrue(((BaseStateImpl)mem).isBBType());
+
+    assertTrue(((BaseStateImpl)mem).isByteBufferType(((BaseStateImpl)mem).getTypeId()));
     assertTrue(mem.isReadOnly());
     checkCrossLeafTypeIds(mem);
     Buffer buf = mem.asBuffer().region(0, bytes, ByteOrder.nativeOrder());
@@ -56,11 +57,11 @@ public class SpecificLeafTest {
     Buffer buf2 = mem2.asBuffer().region(0, bytes, Util.NON_NATIVE_BYTE_ORDER);
     Buffer buf3 = buf2.duplicate();
 
-    assertTrue(((BaseStateImpl)mem).isRegionType());
-    assertTrue(((BaseStateImpl)mem2).isRegionType());
-    assertTrue(((BaseStateImpl)buf).isRegionType());
-    assertTrue(((BaseStateImpl)buf2).isRegionType());
-    assertTrue(((BaseStateImpl)buf3).isDuplicateType());
+    assertTrue(((BaseStateImpl)mem).isRegionType(((BaseStateImpl)mem).getTypeId()));
+    assertTrue(((BaseStateImpl)mem2).isRegionType(((BaseStateImpl)mem2).getTypeId()));
+    assertTrue(((BaseStateImpl)buf).isRegionType(((BaseStateImpl)buf).getTypeId()));
+    assertTrue(((BaseStateImpl)buf2).isRegionType(((BaseStateImpl)buf2).getTypeId()));
+    assertTrue(((BaseStateImpl)buf3).isDuplicateType(((BaseStateImpl)buf3).getTypeId()));
   }
 
   @Test
@@ -68,7 +69,7 @@ public class SpecificLeafTest {
     int bytes = 128;
     try (WritableHandle h = WritableMemory.allocateDirect(bytes)) {
       WritableMemory wmem = h.getWritable(); //native mem
-      assertTrue(((BaseStateImpl)wmem).isDirectType());
+      assertTrue(((BaseStateImpl)wmem).isDirectType(((BaseStateImpl)wmem).getTypeId()));
       assertFalse(wmem.isReadOnly());
       checkCrossLeafTypeIds(wmem);
       WritableMemory nnwmem = wmem.writableRegion(0, bytes, Util.NON_NATIVE_BYTE_ORDER);
@@ -81,11 +82,11 @@ public class SpecificLeafTest {
       Buffer buf2 = mem2.asBuffer().region(0, bytes, Util.NON_NATIVE_BYTE_ORDER);
       Buffer buf3 = buf2.duplicate();
 
-      assertTrue(((BaseStateImpl)mem).isRegionType());
-      assertTrue(((BaseStateImpl)mem2).isRegionType());
-      assertTrue(((BaseStateImpl)buf).isRegionType());
-      assertTrue(((BaseStateImpl)buf2).isRegionType());
-      assertTrue(((BaseStateImpl)buf3).isDuplicateType());
+      assertTrue(((BaseStateImpl)mem).isRegionType(((BaseStateImpl)mem).getTypeId()));
+      assertTrue(((BaseStateImpl)mem2).isRegionType(((BaseStateImpl)mem2).getTypeId()));
+      assertTrue(((BaseStateImpl)buf).isRegionType(((BaseStateImpl)buf).getTypeId()));
+      assertTrue(((BaseStateImpl)buf2).isRegionType(((BaseStateImpl)buf2).getTypeId()));
+      assertTrue(((BaseStateImpl)buf3).isDuplicateType(((BaseStateImpl)buf3).getTypeId()));
     }
   }
 
@@ -108,7 +109,7 @@ public class SpecificLeafTest {
 
     try (WritableMapHandle h = WritableMemory.writableMap(file, 0L, bytes, ByteOrder.nativeOrder())) {
       WritableMemory mem = h.getWritable(); //native mem
-      assertTrue(((BaseStateImpl)mem).isMapType());
+      assertTrue(((BaseStateImpl)mem).isMapType(((BaseStateImpl)mem).getTypeId()));
       assertFalse(mem.isReadOnly());
       checkCrossLeafTypeIds(mem);
       Memory nnreg = mem.region(0, bytes, Util.NON_NATIVE_BYTE_ORDER);
@@ -121,12 +122,12 @@ public class SpecificLeafTest {
       Buffer buf2 = reg2.asBuffer().region(0, bytes, Util.NON_NATIVE_BYTE_ORDER);
       Buffer buf3 = buf2.duplicate();
 
-      assertTrue(((BaseStateImpl)reg).isRegionType());
-      assertTrue(((BaseStateImpl)reg2).isRegionType());
-      assertTrue(((BaseStateImpl)buf).isRegionType());
-      assertTrue(((BaseStateImpl)buf2).isRegionType());
-      assertTrue(((BaseStateImpl)buf3).isDuplicateType());
-      assertTrue(((BaseStateImpl)buf4).isDuplicateType());
+      assertTrue(((BaseStateImpl)reg).isRegionType(((BaseStateImpl)reg).getTypeId()));
+      assertTrue(((BaseStateImpl)reg2).isRegionType(((BaseStateImpl)reg2).getTypeId()));
+      assertTrue(((BaseStateImpl)buf).isRegionType(((BaseStateImpl)buf).getTypeId()));
+      assertTrue(((BaseStateImpl)buf2).isRegionType(((BaseStateImpl)buf2).getTypeId()));
+      assertTrue(((BaseStateImpl)buf3).isDuplicateType(((BaseStateImpl)buf3).getTypeId()));
+      assertTrue(((BaseStateImpl)buf4).isDuplicateType(((BaseStateImpl)buf4).getTypeId()));
     }
   }
 
@@ -134,8 +135,8 @@ public class SpecificLeafTest {
   public void checkHeapLeafs() {
     int bytes = 128;
     Memory mem = Memory.wrap(new byte[bytes]);
-    assertTrue(((BaseStateImpl)mem).isHeapType());
-    assertTrue(((BaseStateImpl)mem).isReadOnlyType());
+    assertTrue(((BaseStateImpl)mem).isHeapType(((BaseStateImpl)mem).getTypeId()));
+    assertTrue(((BaseStateImpl)mem).isReadOnlyType(((BaseStateImpl)mem).getTypeId()));
     checkCrossLeafTypeIds(mem);
     Memory nnreg = mem.region(0, bytes, Util.NON_NATIVE_BYTE_ORDER);
 
@@ -147,47 +148,47 @@ public class SpecificLeafTest {
     Buffer buf2 = reg2.asBuffer().region(0, bytes, Util.NON_NATIVE_BYTE_ORDER);
     Buffer buf3 = buf2.duplicate();
 
-    assertFalse(((BaseStateImpl)mem).isRegionType());
-    assertTrue(((BaseStateImpl)reg2).isRegionType());
-    assertTrue(((BaseStateImpl)buf).isRegionType());
-    assertTrue(((BaseStateImpl)buf2).isRegionType());
-    assertTrue(((BaseStateImpl)buf3).isDuplicateType());
-    assertTrue(((BaseStateImpl)buf4).isDuplicateType());
+    assertFalse(((BaseStateImpl)mem).isRegionType(((BaseStateImpl)mem).getTypeId()));
+    assertTrue(((BaseStateImpl)reg2).isRegionType(((BaseStateImpl)reg2).getTypeId()));
+    assertTrue(((BaseStateImpl)buf).isRegionType(((BaseStateImpl)buf).getTypeId()));
+    assertTrue(((BaseStateImpl)buf2).isRegionType(((BaseStateImpl)buf2).getTypeId()));
+    assertTrue(((BaseStateImpl)buf3).isDuplicateType(((BaseStateImpl)buf3).getTypeId()));
+    assertTrue(((BaseStateImpl)buf4).isDuplicateType(((BaseStateImpl)buf4).getTypeId()));
   }
 
   private static void checkCrossLeafTypeIds(Memory mem) {
     Memory reg1 = mem.region(0, mem.getCapacity());
-    assertTrue(((BaseStateImpl)reg1).isRegionType());
+    assertTrue(((BaseStateImpl)reg1).isRegionType(((BaseStateImpl)reg1).getTypeId()));
 
     Buffer buf1 = reg1.asBuffer();
-    assertTrue(((BaseStateImpl)buf1).isRegionType());
-    assertTrue(((BaseStateImpl)buf1).isBufferType());
+    assertTrue(((BaseStateImpl)buf1).isRegionType(((BaseStateImpl)buf1).getTypeId()));
+    assertTrue(((BaseStateImpl)buf1).isBufferType(((BaseStateImpl)buf1).getTypeId()));
     assertTrue(buf1.isReadOnly());
 
     Buffer buf2 = buf1.duplicate();
-    assertTrue(((BaseStateImpl)buf2).isRegionType());
-    assertTrue(((BaseStateImpl)buf2).isBufferType());
-    assertTrue(((BaseStateImpl)buf2).isDuplicateType());
+    assertTrue(((BaseStateImpl)buf2).isRegionType(((BaseStateImpl)buf2).getTypeId()));
+    assertTrue(((BaseStateImpl)buf2).isBufferType(((BaseStateImpl)buf2).getTypeId()));
+    assertTrue(((BaseStateImpl)buf2).isDuplicateType(((BaseStateImpl)buf2).getTypeId()));
     assertTrue(buf2.isReadOnly());
 
     Memory mem2 = buf1.asMemory(); //
-    assertTrue(((BaseStateImpl)mem2).isRegionType());
-    assertFalse(((BaseStateImpl)mem2).isBufferType());
-    assertFalse(((BaseStateImpl)mem2).isDuplicateType());
+    assertTrue(((BaseStateImpl)mem2).isRegionType(((BaseStateImpl)mem2).getTypeId()));
+    assertFalse(((BaseStateImpl)mem2).isBufferType(((BaseStateImpl)mem2).getTypeId()));
+    assertFalse(((BaseStateImpl)mem2).isDuplicateType(((BaseStateImpl)mem2).getTypeId()));
     assertTrue(mem2.isReadOnly());
 
     Buffer buf3 = buf1.duplicate(Util.NON_NATIVE_BYTE_ORDER);
-    assertTrue(((BaseStateImpl)buf3).isRegionType());
-    assertTrue(((BaseStateImpl)buf3).isBufferType());
-    assertTrue(((BaseStateImpl)buf3).isDuplicateType());
-    assertTrue(((BaseStateImpl)buf3).isNonNativeType());
+    assertTrue(((BaseStateImpl)buf3).isRegionType(((BaseStateImpl)buf3).getTypeId()));
+    assertTrue(((BaseStateImpl)buf3).isBufferType(((BaseStateImpl)buf3).getTypeId()));
+    assertTrue(((BaseStateImpl)buf3).isDuplicateType(((BaseStateImpl)buf3).getTypeId()));
+    assertTrue(((BaseStateImpl)buf3).isNonNativeType(((BaseStateImpl)buf3).getTypeId()));
     assertTrue(buf3.isReadOnly());
 
     Memory mem3 = buf3.asMemory();
-    assertTrue(((BaseStateImpl)mem3).isRegionType());
-    assertFalse(((BaseStateImpl)mem3).isBufferType());
-    assertTrue(((BaseStateImpl)mem3).isDuplicateType());
-    assertTrue(((BaseStateImpl)mem3).isNonNativeType());
+    assertTrue(((BaseStateImpl)mem3).isRegionType(((BaseStateImpl)mem3).getTypeId()));
+    assertFalse(((BaseStateImpl)mem3).isBufferType(((BaseStateImpl)mem3).getTypeId()));
+    assertTrue(((BaseStateImpl)mem3).isDuplicateType(((BaseStateImpl)mem3).getTypeId()));
+    assertTrue(((BaseStateImpl)mem3).isNonNativeType(((BaseStateImpl)mem3).getTypeId()));
     assertTrue(mem3.isReadOnly());
   }
 


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