You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@datasketches.apache.org by jm...@apache.org on 2024/03/29 21:23:42 UTC

(datasketches-java) branch bloom-direct created (now 90623439)

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

jmalkin pushed a change to branch bloom-direct
in repository https://gitbox.apache.org/repos/asf/datasketches-java.git


      at 90623439 still WIP: add untested methods for direct operation

This branch includes the following new commits:

     new d913fe7e WIP: start adding direct memory bloom filter
     new 312f3d49 WIP: fix wrap methods to take input isEmpty flag
     new a6f8dc60 Add rename BitArrayTest to HeapBitArrayTest, add tests for DirectBitArrayR
     new 99bcbec9 WIP: possibly finish testing direct BitArray variants
     new 895c42fe fix that empty filter is only 3 longs, add (untested) wrap methods to base filter class
     new 90623439 still WIP: add untested methods for direct operation

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



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


(datasketches-java) 04/06: WIP: possibly finish testing direct BitArray variants

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

jmalkin pushed a commit to branch bloom-direct
in repository https://gitbox.apache.org/repos/asf/datasketches-java.git

commit 99bcbec94d7e6054f5ca740b75d536bf28428739
Author: Jon <jm...@apache.org>
AuthorDate: Fri Mar 29 00:40:22 2024 -0700

    WIP: possibly finish testing direct BitArray variants
---
 .../datasketches/filters/bloomfilter/BitArray.java |  11 +
 .../filters/bloomfilter/DirectBitArray.java        |   8 +-
 .../filters/bloomfilter/DirectBitArrayR.java       |   3 +-
 .../filters/bloomfilter/DirectBitArrayRTest.java   |  50 ++++-
 .../filters/bloomfilter/DirectBitArrayTest.java    | 228 +++++++++++++++++++++
 .../filters/bloomfilter/HeapBitArrayTest.java      |  12 ++
 6 files changed, 298 insertions(+), 14 deletions(-)

diff --git a/src/main/java/org/apache/datasketches/filters/bloomfilter/BitArray.java b/src/main/java/org/apache/datasketches/filters/bloomfilter/BitArray.java
index 5c196ce4..951d87fc 100644
--- a/src/main/java/org/apache/datasketches/filters/bloomfilter/BitArray.java
+++ b/src/main/java/org/apache/datasketches/filters/bloomfilter/BitArray.java
@@ -19,6 +19,7 @@
 
  package org.apache.datasketches.filters.bloomfilter;
 
+import org.apache.datasketches.common.SketchesArgumentException;
 import org.apache.datasketches.memory.Buffer;
 import org.apache.datasketches.memory.Memory;
 import org.apache.datasketches.memory.WritableMemory;
@@ -98,6 +99,16 @@ abstract class BitArray {
     return Long.BYTES * (isEmpty() ? 1L : (2L + getArrayLength()));
   }
 
+  // returns the number of bytes needed for a non-empty BitArray of the requested size
+  static long getSerializedSizeBytes(final long numBits) {
+    if (numBits > MAX_BITS) {
+      throw new SketchesArgumentException("Requested number of bits exceeds maximum allowed. "
+        + "Requested: " + numBits + ", maximum: " + MAX_BITS);
+    }
+    final int numLongs = (int) Math.ceil(numBits / 64.0);
+    return Long.BYTES * (numLongs + 2);
+  }
+
   abstract protected boolean isDirty();
 
   // used to get a long from the array regardless of underlying storage
diff --git a/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArray.java b/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArray.java
index cf564f82..09be8481 100644
--- a/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArray.java
+++ b/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArray.java
@@ -44,7 +44,7 @@ final class DirectBitArray extends DirectBitArrayR {
       throw new SketchesArgumentException("Number of bits must be strictly positive. Found: " + numBits);
     }
 
-    final int arrayLength = (int) numBits >>> 3; // we know it'll fit in an int based on above checks
+    final int arrayLength = (int) numBits >>> 6; // we know it'll fit in an int based on above checks
     final long requiredBytes = (2L + arrayLength) * Long.BYTES;
     if (wmem.getCapacity() < requiredBytes) {
       throw new SketchesArgumentException("Provided WritableMemory too small for requested array length. "
@@ -97,12 +97,12 @@ final class DirectBitArray extends DirectBitArrayR {
 
   @Override
   boolean getBit(final long index) {
-    return (wmem_.getByte(DATA_OFFSET + ((int) index >>> 6)) & (1L << index)) != 0 ? true : false;
+    return (wmem_.getByte(DATA_OFFSET + ((int) index >>> 3)) & (1 << (index & 0x7))) != 0;
   }
 
   @Override
   protected long getLong(final int arrayIndex) {
-    return wmem_.getLong(DATA_OFFSET + arrayIndex);
+    return wmem_.getLong(DATA_OFFSET + (arrayIndex << 3));
   }
 
   @Override
@@ -192,7 +192,7 @@ final class DirectBitArray extends DirectBitArrayR {
   }
 
   private final void setNumBitsSet(final long numBitsSet) {
-    numBitsSet_ = 0;
+    numBitsSet_ = numBitsSet;
     wmem_.putLong(NUM_BITS_OFFSET, numBitsSet);
   }
 }
diff --git a/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayR.java b/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayR.java
index 6e29099e..9f7218ec 100644
--- a/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayR.java
+++ b/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayR.java
@@ -91,8 +91,7 @@ public class DirectBitArrayR extends BitArray {
   @Override
   boolean getBit(final long index) {
     if (isEmpty()) { return false; }
-    // offset is bytes, not longs, so (index >>> 6) << 3 = index >>> 3
-    return (wmem_.getLong(DATA_OFFSET + ((int) index >>> 3)) & (1L << index)) != 0;
+    return (wmem_.getByte(DATA_OFFSET + ((int) index >>> 3)) & (1 << (index & 0x7))) != 0;
   }
 
   @Override
diff --git a/src/test/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayRTest.java b/src/test/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayRTest.java
index 403cd1cf..26b9fe04 100644
--- a/src/test/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayRTest.java
+++ b/src/test/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayRTest.java
@@ -64,24 +64,33 @@ public class DirectBitArrayRTest {
     DirectBitArrayR.wrap(wmem, true);
   }
 
+  @Test(expectedExceptions = SketchesArgumentException.class)
+  public void tooSmallCapacityTest() {
+    final byte[] bytes = new byte[32];
+    final WritableMemory wmem = WritableMemory.writableWrap(bytes);
+    wmem.putInt(0, 1024); // array length in longs
+    wmem.putLong(8, 201); // number of bits set (non-empty)
+    DirectBitArrayR.wrap(wmem, false);
+  }
+
   // no text of max size because the BitArray allows up to Integer.MAX_VALUE
 
   @Test
   public void basicOperationTest() {
     final HeapBitArray hba = new HeapBitArray(128);
-    assertEquals(hba.getAndSetBit(1), false);
-    assertEquals(hba.getAndSetBit(2), false);
+    assertFalse(hba.getAndSetBit(1));
+    assertFalse(hba.getAndSetBit(2));
     for (int i = 4; i < 64; i <<= 1) {
       hba.getAndSetBit(64 + i);
     }
     assertEquals(hba.getNumBitsSet(), 6);
-    assertEquals(hba.getBit(68), true);
+    assertTrue(hba.getBit(68));
     assertFalse(hba.isEmpty());
 
     final Memory mem = bitArrayToMemory(hba);
     DirectBitArrayR dba = DirectBitArrayR.wrap(mem, hba.isEmpty());
     assertEquals(dba.getNumBitsSet(), 6);
-    assertEquals(dba.getBit(68), true);
+    assertTrue(dba.getBit(68));
     assertFalse(dba.isEmpty());
     assertFalse(dba.isDirty());
 
@@ -95,23 +104,48 @@ public class DirectBitArrayRTest {
     // like basicOperationTest but with setBit which does
     // not neecssarily track numBitsSet_
     final HeapBitArray hba = new HeapBitArray(128);
-    assertEquals(hba.getAndSetBit(1), false);
-    assertEquals(hba.getAndSetBit(2), false);
+    assertFalse(hba.getAndSetBit(1));
+    assertFalse(hba.getAndSetBit(2));
     for (int i = 4; i < 64; i <<= 1) {
       hba.setBit(64 + i);
     }
     assertEquals(hba.getNumBitsSet(), 6);
-    assertEquals(hba.getBit(68), true);
+    assertTrue(hba.getBit(68));
     assertFalse(hba.isEmpty());
 
     final Memory mem = bitArrayToMemory(hba);
     DirectBitArrayR dba = DirectBitArrayR.wrap(mem, hba.isEmpty());
     assertEquals(dba.getNumBitsSet(), 6);
-    assertEquals(dba.getBit(68), true);
+    assertTrue(dba.getBit(68));
     assertFalse(dba.isEmpty());
     assertFalse(dba.isDirty());
   }
 
+  @Test
+  public void bitAddresOutOfBoundsEmptyTest() {
+    final int numBits = 256;
+    final HeapBitArray hba = new HeapBitArray(numBits);
+    final Memory mem = bitArrayToMemory(hba);
+    DirectBitArrayR dba = DirectBitArrayR.wrap(mem, hba.isEmpty());
+    assertFalse(dba.getBit(19));   // in range
+    assertFalse(dba.getBit(-10));        // out of bounds
+    assertFalse(dba.getBit(2048)); // out of bounds
+  }
+
+  @Test
+  public void bitAddresOutOfBoundsNonEmptyTest() {
+    final int numBits = 1024;
+    final HeapBitArray hba = new HeapBitArray(numBits);
+    for (int i = 0; i < numBits; i += numBits / 8) {
+      hba.getAndSetBit(i);
+    }
+
+    final Memory mem = bitArrayToMemory(hba);
+    DirectBitArrayR dba = DirectBitArrayR.wrap(mem, hba.isEmpty());
+    assertThrows(AssertionError.class, () -> dba.getBit(-10));
+    assertThrows(AssertionError.class, () -> dba.getBit(2048));
+  }
+
   @Test
   public void checkInvalidMethods() {
     final int numBits = 1024;
diff --git a/src/test/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayTest.java b/src/test/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayTest.java
new file mode 100644
index 00000000..55ac733c
--- /dev/null
+++ b/src/test/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayTest.java
@@ -0,0 +1,228 @@
+/*
+ * 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.filters.bloomfilter;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertThrows;
+import static org.testng.Assert.assertTrue;
+
+import org.apache.datasketches.common.SketchesArgumentException;
+import org.apache.datasketches.memory.WritableMemory;
+import org.testng.annotations.Test;
+
+public class DirectBitArrayTest {
+
+  private WritableMemory bitArrayToWritableMemory(HeapBitArray ba) {
+    // assumes we're using small enough an array to test that
+    // size can be measured with an int
+    final int numBytes = (int) ba.getSerializedSizeBytes();
+    final byte[] bytes = new byte[numBytes];
+    final WritableMemory wmem = WritableMemory.writableWrap(bytes);
+    ba.writeToBuffer(wmem.asWritableBuffer());
+
+    return wmem;
+  }
+
+  @Test(expectedExceptions = SketchesArgumentException.class)
+  public void writableWrapEmptyBitArrayTest() {
+    final HeapBitArray hba = new HeapBitArray(119);
+    assertTrue(hba.isEmpty());
+
+    final WritableMemory mem = bitArrayToWritableMemory(hba);
+    DirectBitArray.writableWrap(mem, hba.isEmpty());
+  }
+
+  @Test(expectedExceptions = SketchesArgumentException.class)
+  public void createNegativeSizeBitArrayTest() {
+    final byte[] bytes = new byte[32];
+    final WritableMemory wmem = WritableMemory.writableWrap(bytes);
+    wmem.putInt(0, -1); // negative length
+    DirectBitArray.writableWrap(wmem, true);
+  }
+
+  @Test(expectedExceptions = SketchesArgumentException.class)
+  public void tooSmallCapacityTest() {
+    final byte[] bytes = new byte[32];
+    final WritableMemory wmem = WritableMemory.writableWrap(bytes);
+    wmem.putInt(0, 1024); // array length in longs
+    wmem.putLong(8, 201); // number of bits set (non-empty)
+    DirectBitArray.writableWrap(wmem, false);
+  }
+
+  // no text of max size because the BitArray allows up to Integer.MAX_VALUE
+
+  @Test
+  public void basicInitializeOperationsTest() {
+    final byte[] bytes = new byte[56];
+    final WritableMemory wmem = WritableMemory.writableWrap(bytes);
+
+    DirectBitArray dba = DirectBitArray.initialize(192, wmem);
+    assertTrue(dba.isEmpty());
+    assertTrue(dba.hasMemory());
+    assertFalse(dba.isReadOnly());
+    assertEquals(dba.getNumBitsSet(), 0);
+
+    assertFalse(dba.getAndSetBit(13));
+    assertTrue(dba.getBit(13));
+    dba.setBit(17);
+    assertTrue(dba.getAndSetBit(17));
+    assertEquals(dba.getArrayLength(), 3);
+    assertFalse(dba.isEmpty());
+    assertFalse(dba.getBit(183));
+
+    assertTrue(dba.isDirty());
+    assertEquals(dba.getNumBitsSet(), 2);
+    assertFalse(dba.isDirty());
+
+    dba.reset();
+    assertTrue(dba.isEmpty());
+    assertTrue(dba.hasMemory());
+    assertFalse(dba.isReadOnly());
+    assertEquals(dba.getNumBitsSet(), 0);
+  }
+
+  @Test
+  public void basicWritableWrapTest() {
+    final HeapBitArray hba = new HeapBitArray(128);
+    assertFalse(hba.getAndSetBit(1));
+    assertFalse(hba.getAndSetBit(2));
+    for (int i = 4; i < 64; i <<= 1) {
+      hba.getAndSetBit(64 + i);
+    }
+    assertEquals(hba.getNumBitsSet(), 6);
+    assertTrue(hba.getBit(68));
+    assertFalse(hba.isEmpty());
+
+    final WritableMemory mem = bitArrayToWritableMemory(hba);
+    DirectBitArray dba = DirectBitArray.writableWrap(mem, hba.isEmpty());
+    assertEquals(dba.getNumBitsSet(), 6);
+    assertTrue(dba.getBit(68));
+    assertFalse(dba.isEmpty());
+    assertFalse(dba.isDirty());
+
+    assertTrue(dba.hasMemory());
+    assertFalse(dba.isDirect());
+    assertFalse(dba.isReadOnly());
+
+    assertFalse(dba.getAndSetBit(75));
+    dba.setBit(100);
+    assertTrue(dba.getAndSetBit(100));
+    assertEquals(dba.getNumBitsSet(), 8);
+  }
+
+  @Test
+  public void countWritableWrappedBitsWhenDirty() {
+    // like basicOperationTest but with setBit which does
+    // not neecssarily track numBitsSet_
+    final HeapBitArray hba = new HeapBitArray(128);
+    assertFalse(hba.getAndSetBit(1));
+    assertFalse(hba.getAndSetBit(2));
+    for (int i = 4; i < 64; i <<= 1) {
+      hba.setBit(64 + i);
+    }
+    assertEquals(hba.getNumBitsSet(), 6);
+    assertTrue(hba.getBit(68));
+    assertFalse(hba.isEmpty());
+
+    final WritableMemory mem = bitArrayToWritableMemory(hba);
+    DirectBitArray dba = DirectBitArray.writableWrap(mem, hba.isEmpty());
+    assertEquals(dba.getNumBitsSet(), 6);
+    assertTrue(dba.getBit(68));
+    assertFalse(dba.isEmpty());
+    assertFalse(dba.isDirty());
+
+    dba.setBit(102);
+    assertTrue(dba.isDirty());
+  }
+
+  @Test
+  public void bitAddresOutOfBoundsNonEmptyTest() {
+    final int numBits = 1024;
+    final byte[] bytes = new byte[((numBits / 64) + 2) * Long.BYTES];
+    final WritableMemory wmem = WritableMemory.writableWrap(bytes);
+    final DirectBitArray dba = DirectBitArray.initialize(numBits, wmem);
+
+    for (int i = 0; i < numBits; i += numBits / 8) {
+      dba.getAndSetBit(i);
+    }
+
+    assertThrows(AssertionError.class, () -> dba.getBit(-10));
+    assertThrows(AssertionError.class, () -> dba.getBit(2048));
+    assertThrows(AssertionError.class, () -> dba.setBit(-20));
+    assertThrows(AssertionError.class, () -> dba.setBit(4096));
+    assertThrows(AssertionError.class, () -> dba.getAndSetBit(-30));
+    assertThrows(AssertionError.class, () -> dba.getAndSetBit(8192));
+  }
+
+  @Test
+  public void inversionTest() {
+    final int numBits = 1024;
+    final byte[] bytes = new byte[((numBits / 64) + 2) * Long.SIZE];
+    final WritableMemory wmem = WritableMemory.writableWrap(bytes);
+    final DirectBitArray dba = DirectBitArray.initialize(numBits, wmem);
+
+    for (int i = 0; i < numBits; i += numBits / 8) {
+      dba.getAndSetBit(i);
+    }
+    assertTrue(dba.getBit(0));
+
+    final long numSet = dba.getNumBitsSet();
+    dba.invert();
+
+    assertEquals(dba.getNumBitsSet(), numBits - numSet);
+    assertFalse(dba.getBit(0));
+  }
+
+  @Test
+  public void invalidUnionIntersectionTest() {
+    final HeapBitArray hba = new HeapBitArray(128);
+    hba.setBit(0);
+    final WritableMemory wmem = bitArrayToWritableMemory(hba);
+    final DirectBitArray dba = DirectBitArray.writableWrap(wmem, false);
+    assertThrows(SketchesArgumentException.class, () -> dba.union(new HeapBitArray(64)));
+    assertThrows(SketchesArgumentException.class, () -> dba.intersect(new HeapBitArray(512)));
+  }
+
+  @Test
+  public void validUnionAndIntersectionTest() {
+    final long numBits = 64;
+    final int sizeBytes = (int) BitArray.getSerializedSizeBytes(64);
+    final DirectBitArray ba1 = DirectBitArray.initialize(numBits, WritableMemory.allocate(sizeBytes));
+    final DirectBitArray ba2 = DirectBitArray.initialize(numBits, WritableMemory.allocate(sizeBytes));
+    final DirectBitArray ba3 = DirectBitArray.initialize(numBits, WritableMemory.allocate(sizeBytes));
+
+    final int n = 10;
+    for (int i = 0; i < n; ++i) {
+      ba1.getAndSetBit(i);
+      ba2.getAndSetBit(i + (n / 2));
+      ba3.getAndSetBit(2 * i);
+    }
+    assertEquals(ba1.getNumBitsSet(), n);
+    assertEquals(ba2.getNumBitsSet(), n);
+    assertEquals(ba3.getNumBitsSet(), n);
+
+    ba1.intersect(ba2);
+    assertEquals(ba1.getNumBitsSet(), n / 2);
+
+    ba3.union(ba2);
+    assertEquals(ba3.getNumBitsSet(), 3 * n / 2);
+  }
+}
diff --git a/src/test/java/org/apache/datasketches/filters/bloomfilter/HeapBitArrayTest.java b/src/test/java/org/apache/datasketches/filters/bloomfilter/HeapBitArrayTest.java
index e741b1b6..41682442 100644
--- a/src/test/java/org/apache/datasketches/filters/bloomfilter/HeapBitArrayTest.java
+++ b/src/test/java/org/apache/datasketches/filters/bloomfilter/HeapBitArrayTest.java
@@ -21,6 +21,7 @@ package org.apache.datasketches.filters.bloomfilter;
 
 import static org.testng.Assert.assertEquals;
 import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertThrows;
 import static org.testng.Assert.assertTrue;
 
 import org.apache.datasketches.common.SketchesArgumentException;
@@ -71,6 +72,17 @@ public class HeapBitArrayTest {
     assertEquals(ba.getNumBitsSet(), 0);
   }
 
+  @Test
+  public void bitAddresOutOfBoundsTest() {
+    final HeapBitArray ba = new HeapBitArray(1024);
+    assertThrows(ArrayIndexOutOfBoundsException.class, () -> ba.getBit(-10));
+    assertThrows(ArrayIndexOutOfBoundsException.class, () -> ba.getBit(2048));
+    assertThrows(ArrayIndexOutOfBoundsException.class, () -> ba.setBit(-20));
+    assertThrows(ArrayIndexOutOfBoundsException.class, () -> ba.setBit(4096));
+    assertThrows(ArrayIndexOutOfBoundsException.class, () -> ba.getAndSetBit(-30));
+    assertThrows(ArrayIndexOutOfBoundsException.class, () -> ba.getAndSetBit(8192));
+  }
+
   @Test
   public void inversionTest() {
     final int numBits = 1024;


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


(datasketches-java) 03/06: Add rename BitArrayTest to HeapBitArrayTest, add tests for DirectBitArrayR

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

jmalkin pushed a commit to branch bloom-direct
in repository https://gitbox.apache.org/repos/asf/datasketches-java.git

commit a6f8dc602af4700cdf9aedf039ff1d45712488fd
Author: Jon <jm...@apache.org>
AuthorDate: Thu Mar 28 17:08:33 2024 -0700

    Add rename BitArrayTest to HeapBitArrayTest, add tests for DirectBitArrayR
---
 .../datasketches/filters/bloomfilter/BitArray.java |  13 +-
 .../filters/bloomfilter/BloomFilter.java           |  49 ++++----
 .../filters/bloomfilter/BloomFilterBuilder.java    |  16 +--
 .../filters/bloomfilter/DirectBitArray.java        |  31 ++++-
 .../filters/bloomfilter/DirectBitArrayR.java       |  13 +-
 .../bloomfilter/BloomFilterBuilderTest.java        |   6 +-
 .../filters/bloomfilter/BloomFilterTest.java       |  48 ++++----
 .../filters/bloomfilter/DirectBitArrayRTest.java   | 134 +++++++++++++++++++++
 .../{BitArrayTest.java => HeapBitArrayTest.java}   |   2 +-
 9 files changed, 239 insertions(+), 73 deletions(-)

diff --git a/src/main/java/org/apache/datasketches/filters/bloomfilter/BitArray.java b/src/main/java/org/apache/datasketches/filters/bloomfilter/BitArray.java
index 22715126..5c196ce4 100644
--- a/src/main/java/org/apache/datasketches/filters/bloomfilter/BitArray.java
+++ b/src/main/java/org/apache/datasketches/filters/bloomfilter/BitArray.java
@@ -19,6 +19,7 @@
 
  package org.apache.datasketches.filters.bloomfilter;
 
+import org.apache.datasketches.memory.Buffer;
 import org.apache.datasketches.memory.Memory;
 import org.apache.datasketches.memory.WritableMemory;
 
@@ -34,16 +35,16 @@ abstract class BitArray {
 
   protected BitArray() {}
 
-  static BitArray heapify(final Memory mem, final boolean isEmpty) {
-    return HeapBitArray.heapify(mem);
+  static BitArray heapify(final Buffer mem, final boolean isEmpty) {
+    return HeapBitArray.heapify(mem, isEmpty);
   }
 
-  static BitArray wrap(final Memory mem) {
-    return DirectBitArrayR.wrap(mem);
+  static BitArray wrap(final Memory mem, final boolean isEmpty) {
+    return DirectBitArrayR.wrap(mem, isEmpty);
   }
 
-  static BitArray writableWrap(final WritableMemory wmem) {
-    return DirectBitArray.writableWrap(wmem);
+  static BitArray writableWrap(final WritableMemory wmem, final boolean isEmpty) {
+    return DirectBitArray.writableWrap(wmem, isEmpty);
   }
 
   boolean isEmpty() {
diff --git a/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilter.java b/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilter.java
index 0c45f780..7cf051e1 100644
--- a/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilter.java
+++ b/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilter.java
@@ -675,18 +675,22 @@ public final class BloomFilter {
     }
 
     final byte[] bytes = new byte[(int) sizeBytes];
-    final WritableBuffer wbuf = WritableMemory.writableWrap(bytes).asWritableBuffer();
 
-    wbuf.putByte((byte) Family.BLOOMFILTER.getMinPreLongs());
-    wbuf.putByte((byte) SER_VER); // to do: add constant
-    wbuf.putByte((byte) Family.BLOOMFILTER.getID());
-    wbuf.putByte((byte) (bitArray_.isEmpty() ? EMPTY_FLAG_MASK : 0));
-    wbuf.putShort(numHashes_);
-    wbuf.putShort((short) 0); // unused
-    wbuf.putLong(seed_);
+    if (wmem_ == null) {
+      final WritableBuffer wbuf = WritableMemory.writableWrap(bytes).asWritableBuffer();
 
-    bitArray_.writeToBuffer(wbuf);
+      wbuf.putByte((byte) Family.BLOOMFILTER.getMinPreLongs());
+      wbuf.putByte((byte) SER_VER); // to do: add constant
+      wbuf.putByte((byte) Family.BLOOMFILTER.getID());
+      wbuf.putByte((byte) (bitArray_.isEmpty() ? EMPTY_FLAG_MASK : 0));
+      wbuf.putShort(numHashes_);
+      wbuf.putShort((short) 0); // unused
+      wbuf.putLong(seed_);
 
+      ((HeapBitArray) bitArray_).writeToBuffer(wbuf);
+    } else {
+      wmem_.getByteArray(0, bytes, 0, (int) sizeBytes);
+    }
     return bytes;
   }
 
@@ -700,18 +704,21 @@ public final class BloomFilter {
     final long sizeBytes = getSerializedSizeBytes();
 
     final long[] longs = new long[(int) (sizeBytes >> 3)];
-    final WritableBuffer wbuf = WritableMemory.writableWrap(longs).asWritableBuffer();
-
-    wbuf.putByte((byte) Family.BLOOMFILTER.getMinPreLongs());
-    wbuf.putByte((byte) SER_VER); // to do: add constant
-    wbuf.putByte((byte) Family.BLOOMFILTER.getID());
-    wbuf.putByte((byte) (bitArray_.isEmpty() ? EMPTY_FLAG_MASK : 0));
-    wbuf.putShort(numHashes_);
-    wbuf.putShort((short) 0); // unused
-    wbuf.putLong(seed_);
-
-    bitArray_.writeToBuffer(wbuf);
-
+    if (wmem_ == null) {
+      final WritableBuffer wbuf = WritableMemory.writableWrap(longs).asWritableBuffer();
+
+      wbuf.putByte((byte) Family.BLOOMFILTER.getMinPreLongs());
+      wbuf.putByte((byte) SER_VER); // to do: add constant
+      wbuf.putByte((byte) Family.BLOOMFILTER.getID());
+      wbuf.putByte((byte) (bitArray_.isEmpty() ? EMPTY_FLAG_MASK : 0));
+      wbuf.putShort(numHashes_);
+      wbuf.putShort((short) 0); // unused
+      wbuf.putLong(seed_);
+
+      ((HeapBitArray) bitArray_).writeToBuffer(wbuf);
+    } else {
+      wmem_.getLongArray(0, longs, 0, (int) (sizeBytes >>> 3));
+    }
     return longs;
   }
 
diff --git a/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilterBuilder.java b/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilterBuilder.java
index 6d4c86d6..3e3659bd 100644
--- a/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilterBuilder.java
+++ b/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilterBuilder.java
@@ -84,7 +84,7 @@ public final class BloomFilterBuilder {
    * @param targetFalsePositiveProb A desired false positive probability per item
    * @return A new BloomFilter configured for the given input parameters
    */
-  public static HeapBloomFilter createByAccuracy(final long maxDistinctItems, final double targetFalsePositiveProb) {
+  public static BloomFilter createByAccuracy(final long maxDistinctItems, final double targetFalsePositiveProb) {
     return createByAccuracy(maxDistinctItems, targetFalsePositiveProb, ThreadLocalRandom.current().nextLong());
   }
 
@@ -96,7 +96,7 @@ public final class BloomFilterBuilder {
    * @param seed A base hash seed
    * @return A new BloomFilter configured for the given input parameters
    */
-  public static HeapBloomFilter createByAccuracy(final long maxDistinctItems, final double targetFalsePositiveProb, final long seed) {
+  public static BloomFilter createByAccuracy(final long maxDistinctItems, final double targetFalsePositiveProb, final long seed) {
     if (maxDistinctItems <= 0) {
       throw new SketchesArgumentException("maxDistinctItems must be strictly positive");
     }
@@ -105,7 +105,7 @@ public final class BloomFilterBuilder {
     }
     final long numBits = suggestNumFilterBits(maxDistinctItems, targetFalsePositiveProb);
     final short numHashes = suggestNumHashes(maxDistinctItems, numBits);
-    return new HeapBloomFilter(numBits, numHashes, seed);
+    return new BloomFilter(numBits, numHashes, seed);
   }
 
   /**
@@ -116,7 +116,7 @@ public final class BloomFilterBuilder {
    * @param numHashes The number of hash functions to apply to items
    * @return A new BloomFilter configured for the given input parameters
    */
-  public static HeapBloomFilter createBySize(final long numBits, final int numHashes) {
+  public static BloomFilter createBySize(final long numBits, final int numHashes) {
     return createBySize(numBits, numHashes, ThreadLocalRandom.current().nextLong());
   }
 
@@ -129,10 +129,10 @@ public final class BloomFilterBuilder {
    * @param seed A base hash seed
    * @return A new BloomFilter configured for the given input parameters
    */
-  public static HeapBloomFilter createBySize(final long numBits, final int numHashes, final long seed) {
-    if (numBits > HeapBloomFilter.MAX_SIZE) {
+  public static BloomFilter createBySize(final long numBits, final int numHashes, final long seed) {
+    if (numBits > BloomFilter.MAX_SIZE) {
       throw new SketchesArgumentException("Size of BloomFilter must be <= "
-      + HeapBloomFilter.MAX_SIZE + ". Requested: " + numBits);
+      + BloomFilter.MAX_SIZE + ". Requested: " + numBits);
     }
     if (numHashes < 1) {
       throw new SketchesArgumentException("Must specify a strictly positive number of hash functions. "
@@ -143,6 +143,6 @@ public final class BloomFilterBuilder {
       + ". Requested: " + numHashes);
     }
 
-    return new HeapBloomFilter(numBits, numHashes, seed);
+    return new BloomFilter(numBits, numHashes, seed);
   }
 }
diff --git a/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArray.java b/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArray.java
index dbe9c048..cf564f82 100644
--- a/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArray.java
+++ b/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArray.java
@@ -24,13 +24,36 @@ import org.apache.datasketches.memory.WritableMemory;
 
 final class DirectBitArray extends DirectBitArrayR {
 
-  DirectBitArray(final int dataLength, final long storedNumBitsSet, final WritableMemory mem) {
-    super(dataLength, 0, mem); // we'll set numBitsSet_ ourselves so pass 0
+  DirectBitArray(final int dataLength, final long storedNumBitsSet, final WritableMemory wmem) {
+    super(dataLength, 0, wmem); // we'll set numBitsSet_ ourselves so pass 0
 
     // can recompute later if needed
     numBitsSet_ = storedNumBitsSet;
   }
 
+  DirectBitArray(final int dataLength, final WritableMemory wmem) {
+    super(dataLength, 0, wmem);
+
+    wmem_.putInt(0, dataLength_);
+    setNumBitsSet(0);
+    wmem_.clear(DATA_OFFSET, dataLength_ * Long.BYTES);
+  }
+
+  static DirectBitArray initialize(final long numBits, final WritableMemory wmem) {
+    if (numBits <= 0) {
+      throw new SketchesArgumentException("Number of bits must be strictly positive. Found: " + numBits);
+    }
+
+    final int arrayLength = (int) numBits >>> 3; // we know it'll fit in an int based on above checks
+    final long requiredBytes = (2L + arrayLength) * Long.BYTES;
+    if (wmem.getCapacity() < requiredBytes) {
+      throw new SketchesArgumentException("Provided WritableMemory too small for requested array length. "
+        + "Requited: " + requiredBytes + ", provided capcity: " + wmem.getCapacity());
+    }
+
+    return new DirectBitArray(arrayLength, wmem);
+  }
+
   static DirectBitArray writableWrap(final WritableMemory mem, final boolean isEmpty) {
     final int arrayLength = mem.getInt(0);
     final long storedNumBitsSet = isEmpty ? 0L : mem.getLong(NUM_BITS_OFFSET);
@@ -46,7 +69,7 @@ final class DirectBitArray extends DirectBitArrayR {
 
     // required capacity is arrayLength plus room for
     // arrayLength (in longs) and numBitsSet
-    if (storedNumBitsSet > 0 && mem.getCapacity() < arrayLength + 2) {
+    if (storedNumBitsSet != 0 && mem.getCapacity() < arrayLength + 2) {
       throw new SketchesArgumentException("Memory capacity insufficient for Bloom Filter. Needed: "
         + (arrayLength + 2) + " , found: " + mem.getCapacity());
     }
@@ -165,7 +188,7 @@ final class DirectBitArray extends DirectBitArrayR {
 
   @Override
   protected void setLong(final int arrayIndex, final long value) {
-    wmem_.putLong(DATA_OFFSET + arrayIndex, value);
+    wmem_.putLong(DATA_OFFSET + (arrayIndex << 3), value);
   }
 
   private final void setNumBitsSet(final long numBitsSet) {
diff --git a/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayR.java b/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayR.java
index e7c073fb..6e29099e 100644
--- a/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayR.java
+++ b/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayR.java
@@ -41,7 +41,7 @@ public class DirectBitArrayR extends BitArray {
     if (storedNumBitsSet == -1) {
       numBitsSet_ = 0;
       for (int i = 0; i < dataLength_; ++i) {
-        numBitsSet_ += getLong(i);
+        numBitsSet_ += Long.bitCount(wmem_.getLong(DATA_OFFSET + (i << 3)));
       }
     } else {
       numBitsSet_ = storedNumBitsSet;
@@ -54,13 +54,13 @@ public class DirectBitArrayR extends BitArray {
     final int arrayLength = mem.getInt(0);
     final long storedNumBitsSet = isEmpty ? 0L : mem.getLong(NUM_BITS_OFFSET);
 
-    if (arrayLength * (long) Long.SIZE > MAX_BITS) {
-      throw new SketchesArgumentException("Possible corruption: Serialized image indicates array beyond maximum filter capacity");
+    if (arrayLength < 0) {
+      throw new SketchesArgumentException("Possible corruption: Serialized image indicates non-positive array length");
     }
 
     // required capacity is arrayLength plus room for
     // arrayLength (in longs) and numBitsSet
-    if (storedNumBitsSet > 0 && mem.getCapacity() < arrayLength + 2) {
+    if (storedNumBitsSet != 0 && mem.getCapacity() < arrayLength + 2) {
       throw new SketchesArgumentException("Memory capacity insufficient for Bloom Filter. Needed: "
         + (arrayLength + 2) + " , found: " + mem.getCapacity());
     }
@@ -91,13 +91,14 @@ public class DirectBitArrayR extends BitArray {
   @Override
   boolean getBit(final long index) {
     if (isEmpty()) { return false; }
-    return (wmem_.getByte(DATA_OFFSET + ((int) index >>> 6)) & (1L << index)) != 0 ? true : false;
+    // offset is bytes, not longs, so (index >>> 6) << 3 = index >>> 3
+    return (wmem_.getLong(DATA_OFFSET + ((int) index >>> 3)) & (1L << index)) != 0;
   }
 
   @Override
   protected long getLong(final int arrayIndex) {
     if (isEmpty()) { return 0L; }
-    return wmem_.getLong(DATA_OFFSET + arrayIndex);
+    return wmem_.getLong(DATA_OFFSET + (arrayIndex << 3));
   }
 
   @Override
diff --git a/src/test/java/org/apache/datasketches/filters/bloomfilter/BloomFilterBuilderTest.java b/src/test/java/org/apache/datasketches/filters/bloomfilter/BloomFilterBuilderTest.java
index 5f726d88..fd86c651 100644
--- a/src/test/java/org/apache/datasketches/filters/bloomfilter/BloomFilterBuilderTest.java
+++ b/src/test/java/org/apache/datasketches/filters/bloomfilter/BloomFilterBuilderTest.java
@@ -64,9 +64,9 @@ public class BloomFilterBuilderTest {
 
     // invalid number of bits
     assertThrows(SketchesArgumentException.class, () -> BloomFilterBuilder.createBySize(0, 3, 456));
-    assertThrows(SketchesArgumentException.class, () -> BloomFilterBuilder.createBySize(HeapBloomFilter.MAX_SIZE + 1, 3, 456));
+    assertThrows(SketchesArgumentException.class, () -> BloomFilterBuilder.createBySize(BloomFilter.MAX_SIZE + 1, 3, 456));
 
-    final HeapBloomFilter bf = BloomFilterBuilder.createBySize(1L << 21, 3);
+    final BloomFilter bf = BloomFilterBuilder.createBySize(1L << 21, 3);
     assertEquals(bf.getCapacity(), 1 << 21L);
     assertEquals(bf.getNumHashes(), 3);
     assertEquals(bf.getBitsUsed(), 0);
@@ -85,7 +85,7 @@ public class BloomFilterBuilderTest {
 
     // manually computed values based on formula
     assertEquals(BloomFilterBuilder.suggestNumFilterBits(250_000, 0.01), 2396265);
-    HeapBloomFilter bf = BloomFilterBuilder.createByAccuracy(250_000, 0.01);
+    BloomFilter bf = BloomFilterBuilder.createByAccuracy(250_000, 0.01);
     assertEquals(bf.getCapacity(), 2396288); // next smallest multiple of 64
     assertEquals(bf.getNumHashes(), BloomFilterBuilder.suggestNumHashes(250_000, 2396288));
 
diff --git a/src/test/java/org/apache/datasketches/filters/bloomfilter/BloomFilterTest.java b/src/test/java/org/apache/datasketches/filters/bloomfilter/BloomFilterTest.java
index e6c2a048..bb3e73b1 100644
--- a/src/test/java/org/apache/datasketches/filters/bloomfilter/BloomFilterTest.java
+++ b/src/test/java/org/apache/datasketches/filters/bloomfilter/BloomFilterTest.java
@@ -35,7 +35,7 @@ public class BloomFilterTest {
     final long numBits = 8192;
     final int numHashes = 3;
 
-    final HeapBloomFilter bf = BloomFilterBuilder.createBySize(numBits, numHashes);
+    final BloomFilter bf = BloomFilterBuilder.createBySize(numBits, numHashes);
     assertTrue(bf.isEmpty());
     assertEquals(bf.getCapacity(), numBits); // n is multiple of 64 so should be exact
     assertEquals(bf.getNumHashes(), numHashes);
@@ -65,7 +65,7 @@ public class BloomFilterTest {
     final long numBits = 8192;
     final int numHashes = 3;
 
-    final HeapBloomFilter bf = BloomFilterBuilder.createBySize(numBits, numHashes);
+    final BloomFilter bf = BloomFilterBuilder.createBySize(numBits, numHashes);
 
     final int n = 500;
     for (int i = 0; i < n; ++i) {
@@ -96,18 +96,18 @@ public class BloomFilterTest {
   public void incompatibleSetOperationsTest() {
     final int numBits = 128;
     final int numHashes = 4;
-    final HeapBloomFilter bf1 = BloomFilterBuilder.createBySize(numBits, numHashes);
+    final BloomFilter bf1 = BloomFilterBuilder.createBySize(numBits, numHashes);
 
     // mismatched num bits
-    final HeapBloomFilter bf2 = BloomFilterBuilder.createBySize(numBits * 2, numHashes, bf1.getSeed());
+    final BloomFilter bf2 = BloomFilterBuilder.createBySize(numBits * 2, numHashes, bf1.getSeed());
     assertThrows(SketchesArgumentException.class, () -> bf1.union(bf2));
 
     // mismatched num hashes
-    final HeapBloomFilter bf3 = BloomFilterBuilder.createBySize(numBits, numHashes * 2, bf1.getSeed());
+    final BloomFilter bf3 = BloomFilterBuilder.createBySize(numBits, numHashes * 2, bf1.getSeed());
     assertThrows(SketchesArgumentException.class, () -> bf1.intersect(bf3));
 
     // mismatched seed
-    final HeapBloomFilter bf4 = BloomFilterBuilder.createBySize(numBits, numHashes, bf1.getSeed() - 1);
+    final BloomFilter bf4 = BloomFilterBuilder.createBySize(numBits, numHashes, bf1.getSeed() - 1);
     assertThrows(SketchesArgumentException.class, () -> bf1.union(bf4));
   }
 
@@ -116,8 +116,8 @@ public class BloomFilterTest {
     final long numBits = 12288;
     final int numHashes = 4;
 
-    final HeapBloomFilter bf1 = BloomFilterBuilder.createBySize(numBits, numHashes);
-    final HeapBloomFilter bf2 = BloomFilterBuilder.createBySize(numBits, numHashes, bf1.getSeed());
+    final BloomFilter bf1 = BloomFilterBuilder.createBySize(numBits, numHashes);
+    final BloomFilter bf2 = BloomFilterBuilder.createBySize(numBits, numHashes, bf1.getSeed());
 
     final int n = 1000;
     final int maxItem = 3 * n / 2 - 1;
@@ -145,8 +145,8 @@ public class BloomFilterTest {
     final long numBits = 8192;
     final int numHashes = 5;
 
-    final HeapBloomFilter bf1 = BloomFilterBuilder.createBySize(numBits, numHashes);
-    final HeapBloomFilter bf2 = BloomFilterBuilder.createBySize(numBits, numHashes, bf1.getSeed());
+    final BloomFilter bf1 = BloomFilterBuilder.createBySize(numBits, numHashes);
+    final BloomFilter bf2 = BloomFilterBuilder.createBySize(numBits, numHashes, bf1.getSeed());
 
     final int n = 1024;
     final int maxItem = 3 * n / 2 - 1;
@@ -176,18 +176,18 @@ public class BloomFilterTest {
   public void emptySerializationTest() {
     final long numBits = 32768;
     final int numHashes = 7;
-    final HeapBloomFilter bf = BloomFilterBuilder.createBySize(numBits, numHashes);
+    final BloomFilter bf = BloomFilterBuilder.createBySize(numBits, numHashes);
 
     final byte[] bytes = bf.toByteArray();
     Memory mem = Memory.wrap(bytes);
-    final HeapBloomFilter fromBytes = HeapBloomFilter.heapify(mem);
+    final BloomFilter fromBytes = BloomFilter.heapify(mem);
     assertTrue(fromBytes.isEmpty());
     assertEquals(fromBytes.getCapacity(), numBits);
     assertEquals(fromBytes.getNumHashes(), numHashes);
 
     final long[] longs = bf.toLongArray();
     mem = Memory.wrap(longs);
-    final HeapBloomFilter fromLongs = HeapBloomFilter.heapify(mem);
+    final BloomFilter fromLongs = BloomFilter.heapify(mem);
     assertTrue(fromLongs.isEmpty());
     assertEquals(fromLongs.getCapacity(), numBits);
     assertEquals(fromLongs.getNumHashes(), numHashes);
@@ -197,7 +197,7 @@ public class BloomFilterTest {
   public void nonEmptySerializationTest() {
     final long numBits = 32768;
     final int numHashes = 5;
-    final HeapBloomFilter bf = BloomFilterBuilder.createBySize(numBits, numHashes);
+    final BloomFilter bf = BloomFilterBuilder.createBySize(numBits, numHashes);
 
     final int n = 2500;
     for (int i = 0; i < n; ++i) {
@@ -213,7 +213,7 @@ public class BloomFilterTest {
 
     final byte[] bytes = bf.toByteArray();
     Memory mem = Memory.wrap(bytes);
-    final HeapBloomFilter fromBytes = HeapBloomFilter.heapify(mem);
+    final BloomFilter fromBytes = BloomFilter.heapify(mem);
     assertFalse(fromBytes.isEmpty());
     assertEquals(fromBytes.getCapacity(), numBits);
     assertEquals(fromBytes.getBitsUsed(), numBitsSet);
@@ -228,7 +228,7 @@ public class BloomFilterTest {
 
     final long[] longs = bf.toLongArray();
     mem = Memory.wrap(longs);
-    final HeapBloomFilter fromLongs = HeapBloomFilter.heapify(mem);
+    final BloomFilter fromLongs = BloomFilter.heapify(mem);
     assertFalse(fromLongs.isEmpty());
     assertEquals(fromLongs.getCapacity(), numBits);
     assertEquals(fromLongs.getBitsUsed(), numBitsSet);
@@ -246,7 +246,7 @@ public class BloomFilterTest {
   public void testBasicUpdateMethods() {
     final int numDistinct = 100;
     final double fpp = 1e-6;
-    final HeapBloomFilter bf = BloomFilterBuilder.createByAccuracy(numDistinct, fpp);
+    final BloomFilter bf = BloomFilterBuilder.createByAccuracy(numDistinct, fpp);
 
     // empty/null String should do nothing
     bf.update("");
@@ -277,14 +277,14 @@ public class BloomFilterTest {
     // for each BloomFilter update type, call update() then queryAndUpdate(), where
     // the latter should return true. query() should likewise return true.
     // A final intersection should have the same number of bits set as the raw input.
-    final HeapBloomFilter bfMem = BloomFilterBuilder.createByAccuracy(numDistinct, fpp);
+    final BloomFilter bfMem = BloomFilterBuilder.createByAccuracy(numDistinct, fpp);
     bfMem.update(mem);
     assertTrue(bfMem.queryAndUpdate(mem));
     assertTrue(bfMem.query(mem));
     final long numBitsSet = bfMem.getBitsUsed();
     final long seed = bfMem.getSeed();
 
-    final HeapBloomFilter bfBytes = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
+    final BloomFilter bfBytes = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
     final byte[] bytes = new byte[24];
     mem.getByteArray(0, bytes, 0, 24);
     bfBytes.update(bytes);
@@ -292,7 +292,7 @@ public class BloomFilterTest {
     assertTrue(bfBytes.query(bytes));
     assertEquals(bfBytes.getBitsUsed(), numBitsSet);
 
-    final HeapBloomFilter bfChars = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
+    final BloomFilter bfChars = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
     final char[] chars = new char[12];
     mem.getCharArray(0, chars, 0, 12);
     bfChars.update(chars);
@@ -300,7 +300,7 @@ public class BloomFilterTest {
     assertTrue(bfChars.query(chars));
     assertEquals(bfChars.getBitsUsed(), numBitsSet);
 
-    final HeapBloomFilter bfShorts = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
+    final BloomFilter bfShorts = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
     final short[] shorts = new short[12];
     mem.getShortArray(0, shorts, 0, 12);
     bfShorts.update(shorts);
@@ -308,7 +308,7 @@ public class BloomFilterTest {
     assertTrue(bfShorts.query(shorts));
     assertEquals(bfShorts.getBitsUsed(), numBitsSet);
 
-    final HeapBloomFilter bfInts = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
+    final BloomFilter bfInts = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
     final int[] ints = new int[6];
     mem.getIntArray(0, ints, 0, 6);
     bfInts.update(ints);
@@ -316,7 +316,7 @@ public class BloomFilterTest {
     assertTrue(bfInts.query(ints));
     assertEquals(bfInts.getBitsUsed(), numBitsSet);
 
-    final HeapBloomFilter bfLongs = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
+    final BloomFilter bfLongs = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
     final long[] longs = new long[3];
     mem.getLongArray(0, longs, 0, 3);
     bfLongs.update(longs);
@@ -325,7 +325,7 @@ public class BloomFilterTest {
     assertEquals(bfLongs.getBitsUsed(), numBitsSet);
 
     // intersect all the sketches into a new one
-    final HeapBloomFilter bf = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
+    final BloomFilter bf = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
     bf.intersect(bfMem);
     bf.intersect(bfBytes);
     bf.intersect(bfChars);
diff --git a/src/test/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayRTest.java b/src/test/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayRTest.java
new file mode 100644
index 00000000..403cd1cf
--- /dev/null
+++ b/src/test/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayRTest.java
@@ -0,0 +1,134 @@
+/*
+ * 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.filters.bloomfilter;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.assertFalse;
+import static org.testng.Assert.assertThrows;
+import static org.testng.Assert.assertTrue;
+
+import org.apache.datasketches.common.SketchesArgumentException;
+import org.apache.datasketches.common.SketchesReadOnlyException;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+import org.testng.annotations.Test;
+
+public class DirectBitArrayRTest {
+
+  private Memory bitArrayToMemory(HeapBitArray ba) {
+    // assumes we're using small enough an array to test that
+    // size can be measured with an int
+    final int numBytes = (int) ba.getSerializedSizeBytes();
+    final byte[] bytes = new byte[numBytes];
+    final WritableMemory wmem = WritableMemory.writableWrap(bytes);
+    ba.writeToBuffer(wmem.asWritableBuffer());
+
+    return wmem;
+  }
+
+  @Test
+  public void createBitArrayTest() {
+    final HeapBitArray hba = new HeapBitArray(119);
+    assertTrue(hba.isEmpty());
+
+    final Memory mem = bitArrayToMemory(hba);
+    DirectBitArrayR dba = DirectBitArrayR.wrap(mem, hba.isEmpty());
+    assertTrue(dba.isEmpty());
+    assertEquals(dba.getCapacity(), 128); // nearest multiple of 64
+    assertEquals(dba.getArrayLength(), 2);
+    assertEquals(dba.getNumBitsSet(), 0);
+  }
+
+  @Test(expectedExceptions = SketchesArgumentException.class)
+  public void createNegativeSizeBitArrayTest() {
+    final byte[] bytes = new byte[32];
+    final WritableMemory wmem = WritableMemory.writableWrap(bytes);
+    wmem.putInt(0, -1); // negative length
+    DirectBitArrayR.wrap(wmem, true);
+  }
+
+  // no text of max size because the BitArray allows up to Integer.MAX_VALUE
+
+  @Test
+  public void basicOperationTest() {
+    final HeapBitArray hba = new HeapBitArray(128);
+    assertEquals(hba.getAndSetBit(1), false);
+    assertEquals(hba.getAndSetBit(2), false);
+    for (int i = 4; i < 64; i <<= 1) {
+      hba.getAndSetBit(64 + i);
+    }
+    assertEquals(hba.getNumBitsSet(), 6);
+    assertEquals(hba.getBit(68), true);
+    assertFalse(hba.isEmpty());
+
+    final Memory mem = bitArrayToMemory(hba);
+    DirectBitArrayR dba = DirectBitArrayR.wrap(mem, hba.isEmpty());
+    assertEquals(dba.getNumBitsSet(), 6);
+    assertEquals(dba.getBit(68), true);
+    assertFalse(dba.isEmpty());
+    assertFalse(dba.isDirty());
+
+    assertTrue(dba.hasMemory());
+    assertFalse(dba.isDirect());
+    assertTrue(dba.isReadOnly());
+  }
+
+  @Test
+  public void countBitsWhenDirty() {
+    // like basicOperationTest but with setBit which does
+    // not neecssarily track numBitsSet_
+    final HeapBitArray hba = new HeapBitArray(128);
+    assertEquals(hba.getAndSetBit(1), false);
+    assertEquals(hba.getAndSetBit(2), false);
+    for (int i = 4; i < 64; i <<= 1) {
+      hba.setBit(64 + i);
+    }
+    assertEquals(hba.getNumBitsSet(), 6);
+    assertEquals(hba.getBit(68), true);
+    assertFalse(hba.isEmpty());
+
+    final Memory mem = bitArrayToMemory(hba);
+    DirectBitArrayR dba = DirectBitArrayR.wrap(mem, hba.isEmpty());
+    assertEquals(dba.getNumBitsSet(), 6);
+    assertEquals(dba.getBit(68), true);
+    assertFalse(dba.isEmpty());
+    assertFalse(dba.isDirty());
+  }
+
+  @Test
+  public void checkInvalidMethods() {
+    final int numBits = 1024;
+    final HeapBitArray hba = new HeapBitArray(numBits);
+    for (int i = 0; i < numBits; i += numBits / 8) {
+      hba.getAndSetBit(i);
+    }
+
+    final Memory mem = bitArrayToMemory(hba);
+    DirectBitArrayR dba = DirectBitArrayR.wrap(mem, hba.isEmpty());
+
+    // all of these try to modify a read-only memory
+    assertThrows(SketchesReadOnlyException.class, () -> dba.setBit(14));
+    assertThrows(SketchesReadOnlyException.class, () -> dba.getAndSetBit(100));
+    assertThrows(SketchesReadOnlyException.class, () -> dba.reset());
+    assertThrows(SketchesReadOnlyException.class, () -> dba.invert());
+    assertThrows(SketchesReadOnlyException.class, () -> dba.intersect(hba));
+    assertThrows(SketchesReadOnlyException.class, () -> dba.union(hba));
+  }
+}
diff --git a/src/test/java/org/apache/datasketches/filters/bloomfilter/BitArrayTest.java b/src/test/java/org/apache/datasketches/filters/bloomfilter/HeapBitArrayTest.java
similarity index 99%
rename from src/test/java/org/apache/datasketches/filters/bloomfilter/BitArrayTest.java
rename to src/test/java/org/apache/datasketches/filters/bloomfilter/HeapBitArrayTest.java
index e1340708..e741b1b6 100644
--- a/src/test/java/org/apache/datasketches/filters/bloomfilter/BitArrayTest.java
+++ b/src/test/java/org/apache/datasketches/filters/bloomfilter/HeapBitArrayTest.java
@@ -28,7 +28,7 @@ import org.apache.datasketches.memory.WritableBuffer;
 import org.apache.datasketches.memory.WritableMemory;
 import org.testng.annotations.Test;
 
-public class BitArrayTest {
+public class HeapBitArrayTest {
 
   @Test
   public void createBitArrayTest() {


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


(datasketches-java) 06/06: still WIP: add untested methods for direct operation

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

jmalkin pushed a commit to branch bloom-direct
in repository https://gitbox.apache.org/repos/asf/datasketches-java.git

commit 9062343922823e3614b008e182c6d712e97dfd1f
Author: jmalkin <78...@users.noreply.github.com>
AuthorDate: Fri Mar 29 14:19:39 2024 -0700

    still WIP: add untested methods for direct operation
---
 .../filters/bloomfilter/BloomFilter.java           | 72 ++++++++++++++++++++-
 .../filters/bloomfilter/BloomFilterBuilder.java    | 74 +++++++++++++++++++++-
 2 files changed, 140 insertions(+), 6 deletions(-)

diff --git a/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilter.java b/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilter.java
index 8987766e..ca4da16e 100644
--- a/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilter.java
+++ b/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilter.java
@@ -84,6 +84,37 @@ public final class BloomFilter {
     wmem_ = null;
   }
 
+  /**
+   * Creates a BloomFilter with given number of bits and number of hash functions,
+   * and a user-specified seed in the provided WritableMemory
+   *
+   * @param numBits The size of the BloomFilter, in bits
+   * @param numHashes The number of hash functions to apply to items
+   * @param seed The base hash seed
+   * @param wmem A WritableMemory that will be initialized to hold the filter
+   */
+  BloomFilter(final long numBits, final int numHashes, final long seed, final WritableMemory wmem) {
+    if (wmem.getCapacity() < Family.BLOOMFILTER.getMaxPreLongs()) {
+      throw new SketchesArgumentException("Provided WritableMemory capacity insufficient to initialize BloomFilter");
+    }
+
+    // we don't resize so initialize with non-empty preLongs value
+    // and no empty flag
+    final WritableBuffer wbuf = wmem.asWritableBuffer();
+    wbuf.putByte((byte) Family.BLOOMFILTER.getMaxPreLongs());
+    wbuf.putByte((byte) SER_VER);
+    wbuf.putByte((byte) Family.BLOOMFILTER.getID());
+    wbuf.putByte((byte) 0); // instead of (bitArray_.isEmpty() ? EMPTY_FLAG_MASK : 0);
+    wbuf.putShort((short) numHashes);
+    wbuf.putShort((short) 0); // unused
+    wbuf.putLong(seed);
+
+    seed_ = seed;
+    numHashes_ = (short) numHashes;
+    bitArray_ = DirectBitArray.initialize(numBits, wmem.writableRegion(BIT_ARRAY_OFFSET, wmem.getCapacity() - BIT_ARRAY_OFFSET));
+    wmem_ = wmem;
+  }
+
   // Constructor used with internalHeapifyOrWrap()
   BloomFilter(final short numHashes, final long seed, final BitArray bitArray, final WritableMemory wmem) {
     seed_ = seed;
@@ -182,6 +213,30 @@ public final class BloomFilter {
    */
   public long getSeed() { return seed_; }
 
+  /**
+   * Returns whether the filter has a backing Memory object
+   * @return true if backed by Memory, otherwise false
+   */
+  public boolean hasMemory() { return wmem_ != null; }
+
+  /**
+   * Returns whether the filter is in read-only mode. That is possible
+   * only if there is a backing Memory in read-only mode.
+   * @return true if read-only, otherwise false
+   */
+  public boolean isReadOnly() {
+    return wmem_ != null && wmem_.isReadOnly();
+  }
+
+  /**
+   * Returns whether the filter is a direct (off-heap) or on-heap object.
+   * That is possible only if there is a backing Memory.
+   * @return true if using direct memory access, otherwise false
+   */
+  public boolean isDirect() {
+    return wmem_ != null && wmem_.isDirect();
+  }
+
   /**
    * Returns the percentage of all bits in the BloomFilter set to 1.
    * @return the percentage of bits in the filter set to 1
@@ -664,6 +719,15 @@ public final class BloomFilter {
     return sizeBytes;
   }
 
+  /**
+   * Returns the serialized length of a non-empty BloomFilter of the given size, in bytes
+   * @param numBits The number of bits of to use for size computation
+   * @return The serialized length of a non-empty BloomFilter of the given size, in bytes
+   */
+  public static long getSerializedSize(final long numBits) {
+    return (2L * Long.BYTES) + BitArray.getSerializedSizeBytes(numBits);
+  }
+
 /*
  * A Bloom Filter's serialized image always uses 3 longs of preamble when empty,
  * otherwise 4 longs:
@@ -704,8 +768,9 @@ public final class BloomFilter {
     if (wmem_ == null) {
       final WritableBuffer wbuf = WritableMemory.writableWrap(bytes).asWritableBuffer();
 
-      wbuf.putByte((byte) Family.BLOOMFILTER.getMinPreLongs());
-      wbuf.putByte((byte) SER_VER); // to do: add constant
+      final int numPreLongs = isEmpty() ? Family.BLOOMFILTER.getMinPreLongs() : Family.BLOOMFILTER.getMaxPreLongs();
+      wbuf.putByte((byte) numPreLongs);
+      wbuf.putByte((byte) SER_VER);
       wbuf.putByte((byte) Family.BLOOMFILTER.getID());
       wbuf.putByte((byte) (bitArray_.isEmpty() ? EMPTY_FLAG_MASK : 0));
       wbuf.putShort(numHashes_);
@@ -732,7 +797,8 @@ public final class BloomFilter {
     if (wmem_ == null) {
       final WritableBuffer wbuf = WritableMemory.writableWrap(longs).asWritableBuffer();
 
-      wbuf.putByte((byte) Family.BLOOMFILTER.getMinPreLongs());
+      final int numPreLongs = isEmpty() ? Family.BLOOMFILTER.getMinPreLongs() : Family.BLOOMFILTER.getMaxPreLongs();
+      wbuf.putByte((byte) numPreLongs);
       wbuf.putByte((byte) SER_VER); // to do: add constant
       wbuf.putByte((byte) Family.BLOOMFILTER.getID());
       wbuf.putByte((byte) (bitArray_.isEmpty() ? EMPTY_FLAG_MASK : 0));
diff --git a/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilterBuilder.java b/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilterBuilder.java
index 3e3659bd..3397ca2c 100644
--- a/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilterBuilder.java
+++ b/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilterBuilder.java
@@ -22,6 +22,7 @@ package org.apache.datasketches.filters.bloomfilter;
 import java.util.concurrent.ThreadLocalRandom;
 
 import org.apache.datasketches.common.SketchesArgumentException;
+import org.apache.datasketches.memory.WritableMemory;
 
 /**
  * <p>This class provides methods to help estimate the correct paramters to use when
@@ -79,7 +80,30 @@ public final class BloomFilterBuilder {
   }
 
   /**
-   * Creates a new BloomFilter with an optimal number of bits and hash functions for the given inputs.
+   * Returns the minimum memory size, in bytes, needed for a serizlied BloomFilter with an optimal number of bits
+   * and hash functions for the given inputs. This is also the minimum size of a WritableMemory for
+   * in-place filter intialization.
+   * @param maxDistinctItems The maximum expected number of distinct items to add to the filter
+   * @param targetFalsePositiveProb A desired false positive probability per item
+   * @return The size, in bytes, required to hold the specified BloomFilter when serialized
+   */
+  public static long getSerializedFilterSizeByAccuracy(final long maxDistinctItems, final double targetFalsePositiveProb) {
+    return BloomFilter.getSerializedSize(suggestNumFilterBits(maxDistinctItems, targetFalsePositiveProb));
+  }
+
+  /**
+   * Returns the minimum memory size, in bytes, needed for a serizlied BloomFilter with the given number of bits.
+   * This is also the minimum size of a WritableMemory for in-place filter intialization.
+   * @param numBits The number of bits in the target BloomFilter's bit array.
+   * @return The size, in bytes, required to hold the specified BloomFilter when serialized
+   */
+  public static long getSerializedFilterSize(final long numBits) {
+    return BloomFilter.getSerializedSize(numBits);
+  }
+
+  /**
+   * Creates a new BloomFilter with an optimal number of bits and hash functions for the given inputs,
+   * using a random base seed for the hash function.
    * @param maxDistinctItems The maximum expected number of distinct items to add to the filter
    * @param targetFalsePositiveProb A desired false positive probability per item
    * @return A new BloomFilter configured for the given input parameters
@@ -110,7 +134,7 @@ public final class BloomFilterBuilder {
 
   /**
    * Creates a BloomFilter with given number of bits and number of hash functions,
-   * and a random seed.
+   * using a rnadom base seed for the hash function.
    *
    * @param numBits The size of the BloomFilter, in bits
    * @param numHashes The number of hash functions to apply to items
@@ -122,7 +146,7 @@ public final class BloomFilterBuilder {
 
    /**
    * Creates a BloomFilter with given number of bits and number of hash functions,
-   * and a random seed.
+   * using the provided base seed for the hash function.
    *
    * @param numBits The size of the BloomFilter, in bits
    * @param numHashes The number of hash functions to apply to items
@@ -145,4 +169,48 @@ public final class BloomFilterBuilder {
 
     return new BloomFilter(numBits, numHashes, seed);
   }
+
+   /**
+   * Initializes a BloomFilter with given number of bits and number of hash functions,
+   * using a random base seed for the hash function and writing into the provided WritableMemory.
+   *
+   * @param numBits The size of the BloomFilter, in bits
+   * @param numHashes The number of hash functions to apply to items
+   * @param dstMem A WritableMemory to hold the initialized filter
+   * @return A new BloomFilter configured for the given input parameters
+   */
+  public static BloomFilter initializeFilter(final long numBits, final int numHashes, final WritableMemory wmem) {
+    return initializeFilter(numBits, numHashes, ThreadLocalRandom.current().nextLong(), wmem);
+  }
+
+   /**
+   * Initializes a BloomFilter with given number of bits and number of hash functions,
+   * using the provided base seed for the hash function and writing into the provided WritableMemory.
+   *
+   * @param numBits The size of the BloomFilter, in bits
+   * @param numHashes The number of hash functions to apply to items
+   * @param seed A base hash seed
+   * @param dstMem A WritableMemory to hold the initialized filter
+   * @return A new BloomFilter configured for the given input parameters
+   */
+  public static BloomFilter initializeFilter(final long numBits, final int numHashes, final long seed, final WritableMemory wmem) {
+    if (numBits > BloomFilter.MAX_SIZE) {
+      throw new SketchesArgumentException("Size of BloomFilter must be <= "
+      + BloomFilter.MAX_SIZE + ". Requested: " + numBits);
+    }
+    if (numHashes < 1) {
+      throw new SketchesArgumentException("Must specify a strictly positive number of hash functions. "
+      + "Requested: " + numHashes);
+    }
+    if (numHashes > Short.MAX_VALUE) {
+      throw new SketchesArgumentException("Number of hashes cannot exceed " + Short.MAX_VALUE
+      + ". Requested: " + numHashes);
+    }
+    if (wmem.getCapacity() < BloomFilter.getSerializedSize(numBits)) {
+      throw new SketchesArgumentException("Provided WritableMemory is insufficint to hold requested filter");
+    }
+
+    return new BloomFilter(numBits, numHashes, seed, wmem);
+  }
+
 }


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


(datasketches-java) 05/06: fix that empty filter is only 3 longs, add (untested) wrap methods to base filter class

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

jmalkin pushed a commit to branch bloom-direct
in repository https://gitbox.apache.org/repos/asf/datasketches-java.git

commit 895c42fe23219ade64a0716f3ee6ee2eba2e622b
Author: Jon <jm...@apache.org>
AuthorDate: Fri Mar 29 10:59:31 2024 -0700

    fix that empty filter is only 3 longs, add (untested) wrap methods to base filter class
---
 .../org/apache/datasketches/common/Family.java     |  4 +-
 .../filters/bloomfilter/BloomFilter.java           | 45 +++++++++++++++++-----
 2 files changed, 37 insertions(+), 12 deletions(-)

diff --git a/src/main/java/org/apache/datasketches/common/Family.java b/src/main/java/org/apache/datasketches/common/Family.java
index f369997b..22524899 100644
--- a/src/main/java/org/apache/datasketches/common/Family.java
+++ b/src/main/java/org/apache/datasketches/common/Family.java
@@ -157,11 +157,11 @@ public enum Family {
    * t-Digest for estimating quantiles and ranks
    */
   TDIGEST(20, "TDigest", 1, 2),
-  
+
   /**
    * Bloom Filter
    */
-  BLOOMFILTER(21, "BLOOMFILTER", 4, 4);
+  BLOOMFILTER(21, "BLOOMFILTER", 3, 4);
 
   private static final Map<Integer, Family> lookupID = new HashMap<>();
   private static final Map<String, Family> lookupFamName = new HashMap<>();
diff --git a/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilter.java b/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilter.java
index 7cf051e1..8987766e 100644
--- a/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilter.java
+++ b/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilter.java
@@ -62,6 +62,7 @@ public final class BloomFilter {
   static final long MAX_SIZE = (Integer.MAX_VALUE - Family.BLOOMFILTER.getMaxPreLongs()) * (long) Long.SIZE;
   private static final int SER_VER = 1;
   private static final int EMPTY_FLAG_MASK = 4;
+  private static final long BIT_ARRAY_OFFSET = 16;
 
   private long seed_;            // hash seed
   private short numHashes_;      // number of hash values
@@ -70,7 +71,7 @@ public final class BloomFilter {
 
   /**
    * Creates a BloomFilter with given number of bits and number of hash functions,
-   * and a user-specified  seed.
+   * and a user-specified seed.
    *
    * @param numBits The size of the BloomFilter, in bits
    * @param numHashes The number of hash functions to apply to items
@@ -83,12 +84,12 @@ public final class BloomFilter {
     wmem_ = null;
   }
 
-  // Constructor used with heapify()
-  BloomFilter(final short numHashes, final long seed, final BitArray bitArray) {
+  // Constructor used with internalHeapifyOrWrap()
+  BloomFilter(final short numHashes, final long seed, final BitArray bitArray, final WritableMemory wmem) {
     seed_ = seed;
     numHashes_ = numHashes;
     bitArray_ = bitArray;
-    wmem_ = null;
+    wmem_ = wmem;
   }
 
   /**
@@ -97,7 +98,21 @@ public final class BloomFilter {
    * @return a BloomFilter object
    */
   public static BloomFilter heapify(final Memory mem) {
-    final Buffer buf = mem.asBuffer();
+    // casting to writable, but heapify so only reading
+    return internalHeapifyOrWrap((WritableMemory) mem, false, false);
+  }
+
+  public static BloomFilter wrap(final Memory mem) {
+    // casting to writable, but tracking that the object is read-only
+    return internalHeapifyOrWrap((WritableMemory) mem, true, false);
+  }
+
+  public static BloomFilter writableWrap(final WritableMemory wmem) {
+    return internalHeapifyOrWrap(wmem, true, true);
+  }
+
+  private static BloomFilter internalHeapifyOrWrap(final WritableMemory wmem, final boolean isWrap, final boolean isWritable) {
+    final Buffer buf = wmem.asBuffer();
     final int preLongs = buf.getByte();
     final int serVer = buf.getByte();
     final int familyID = buf.getByte();
@@ -116,9 +131,18 @@ public final class BloomFilter {
 
     final boolean isEmpty = (flags & EMPTY_FLAG_MASK) != 0;
 
-    final BitArray bitArray = BitArray.heapify(buf, isEmpty);
-
-    return new BloomFilter(numHashes, seed, bitArray);
+    final BitArray bitArray;
+    if (isWrap) {
+      if (isWritable) {
+        bitArray = DirectBitArray.writableWrap(wmem.writableRegion(BIT_ARRAY_OFFSET, wmem.getCapacity() - BIT_ARRAY_OFFSET), isEmpty);
+      } else {
+        bitArray = DirectBitArrayR.wrap(wmem.region(BIT_ARRAY_OFFSET, wmem.getCapacity() - BIT_ARRAY_OFFSET), isEmpty);
+      }
+      return new BloomFilter(numHashes, seed, bitArray, wmem);
+    } else { // if heapify
+      bitArray = BitArray.heapify(buf, isEmpty);
+      return new BloomFilter(numHashes, seed, bitArray, null);
+    }
   }
 
   /**
@@ -641,7 +665,8 @@ public final class BloomFilter {
   }
 
 /*
- * A Bloom Filter's serialized image always uses 4 longs of preamble, whether empty or not:
+ * A Bloom Filter's serialized image always uses 3 longs of preamble when empty,
+ * otherwise 4 longs:
  *
  * <pre>
  * Long || Start Byte Adr:
@@ -656,7 +681,7 @@ public final class BloomFilter {
  *  2   ||-------BitArray Length (in longs)----------|-----------Unused------------------|
  *
  *      ||      24        |   25   |   26   |   27   |   28   |   29   |   30   |   31   |
- *  2   ||---------------------------------NumBitsSet------------------------------------|
+ *  3   ||---------------------------------NumBitsSet------------------------------------|
  *  </pre>
  *
  * The raw BitArray bits, if non-empty start at byte 24.


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


(datasketches-java) 01/06: WIP: start adding direct memory bloom filter

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

jmalkin pushed a commit to branch bloom-direct
in repository https://gitbox.apache.org/repos/asf/datasketches-java.git

commit d913fe7edc27d069ab36f43553332dca8308d249
Author: Jon <jm...@apache.org>
AuthorDate: Thu Mar 28 00:14:05 2024 -0700

    WIP: start adding direct memory bloom filter
---
 .../datasketches/filters/bloomfilter/BitArray.java | 207 +++++----------------
 .../filters/bloomfilter/BloomFilter.java           |  45 ++---
 .../filters/bloomfilter/BloomFilterBuilder.java    |  16 +-
 .../filters/bloomfilter/DirectBitArray.java        | 175 +++++++++++++++++
 .../filters/bloomfilter/DirectBitArrayR.java       | 152 +++++++++++++++
 .../{BitArray.java => HeapBitArray.java}           |  89 ++++-----
 .../filters/bloomfilter/BitArrayTest.java          |  38 ++--
 .../bloomfilter/BloomFilterBuilderTest.java        |  10 +-
 .../filters/bloomfilter/BloomFilterTest.java       |  62 +++---
 9 files changed, 506 insertions(+), 288 deletions(-)

diff --git a/src/main/java/org/apache/datasketches/filters/bloomfilter/BitArray.java b/src/main/java/org/apache/datasketches/filters/bloomfilter/BitArray.java
index 91c4f5ef..76fe2f85 100644
--- a/src/main/java/org/apache/datasketches/filters/bloomfilter/BitArray.java
+++ b/src/main/java/org/apache/datasketches/filters/bloomfilter/BitArray.java
@@ -17,194 +17,83 @@
  * under the License.
  */
 
-package org.apache.datasketches.filters.bloomfilter;
-
-import java.util.Arrays;
-
-import org.apache.datasketches.common.SketchesArgumentException;
-import org.apache.datasketches.memory.Buffer;
-import org.apache.datasketches.memory.WritableBuffer;
-
-/**
- * This class holds an array of bits suitable for use in a Bloom Filter
- *
- * <p>Rounds the number of bits up to the smallest multiple of 64 (one long)
- * that is not smaller than the specified number.
- */
-final class BitArray {
+ package org.apache.datasketches.filters.bloomfilter;
+
+ /**
+  * This class holds an array of bits suitable for use in a Bloom Filter
+  *
+  * <p>Rounds the number of bits up to the smallest multiple of 64 (one long)
+  * that is not smaller than the specified number.
+  */
+abstract class BitArray {
   // MAX_BITS using longs, based on array indices being capped at Integer.MAX_VALUE
-  private static final long MAX_BITS = Integer.MAX_VALUE * (long) Long.SIZE;
+  protected static final long MAX_BITS = Integer.MAX_VALUE * (long) Long.SIZE;
 
-  private long numBitsSet_;  // if -1, need to recompute value
-  private boolean isDirty_;
-  private long[] data_;
+  protected BitArray() {}
 
-  // creates an array of a given size
-  BitArray(final long numBits) {
-    if (numBits <= 0) {
-      throw new SketchesArgumentException("Number of bits must be strictly positive. Found: " + numBits);
-    }
-    if (numBits > MAX_BITS) {
-      throw new SketchesArgumentException("Number of bits may not exceed " + MAX_BITS + ". Found: " + numBits);
-    }
-
-    final int numLongs = (int) Math.ceil(numBits / 64.0);
-    numBitsSet_ = 0;
-    isDirty_ = false;
-    data_ = new long[numLongs];
-  }
-
-  // uses the provided array
-  BitArray(final long numBitsSet, final long[] data) {
-    data_ = data;
-    isDirty_ = numBitsSet < 0;
-    numBitsSet_ = numBitsSet;
+  boolean isEmpty() {
+    return !isDirty() && getNumBitsSet() == 0;
   }
 
-  // reads a serialized image, but the BitArray is not fully self-describing so requires
-  // a flag to indicate whether the array is empty
-  static BitArray heapify(final Buffer buffer, final boolean isEmpty) {
-    final int numLongs = buffer.getInt();
-    if (numLongs < 0) {
-      throw new SketchesArgumentException("Possible corruption: Must have strictly positive array size. Found: " + numLongs);
-    }
+  abstract boolean hasMemory();
 
-    if (isEmpty) {
-      return new BitArray((long) numLongs * Long.SIZE);
-    }
+  abstract boolean isDirect();
 
-    buffer.getInt(); // unused
-
-    // will be -1 if dirty
-    final long numBitsSet = buffer.getLong();
-
-    final long[] data = new long[numLongs];
-    buffer.getLongArray(data, 0, numLongs);
-    return new BitArray(numBitsSet, data);
+  boolean isReadOnly() {
+    return false;
   }
 
-  boolean isEmpty() {
-    return getNumBitsSet() == 0 && !isDirty_;
-  }
+  abstract boolean getBit(final long index);
 
-  // queries a single bit in the array
-  boolean getBit(final long index) {
-    return (data_[(int) index >>> 6] & (1L << index)) != 0 ? true : false;
-  }
+  abstract boolean getAndSetBit(final long index);
 
-  // sets a single bit in the array without querying, meaning the method
-  // cannot properly track the number of bits set so set isDirty = true
-  void setBit(final long index) {
-    data_[(int) index >>> 6] |= 1L << index;
-    isDirty_ = true;
-  }
+  abstract void setBit(final long index);
 
-  // returns existing value of bit
-  boolean getAndSetBit(final long index) {
-    final int offset = (int) index >>> 6;
-    final long mask = 1L << index;
-    if ((data_[offset] & mask) != 0) {
-      return true; // already seen
-    } else {
-      data_[offset] |= mask;
-      ++numBitsSet_; // increment regardless of isDirty_
-      return false; // new set
-    }
-  }
+  abstract long getNumBitsSet();
 
-  // may need to recompute value:
-  // O(1) if only getAndSetBit() has been used
-  // O(data_.length) if setBit() has ever been used
-  long getNumBitsSet() {
-    if (isDirty_) {
-      numBitsSet_ = 0;
-      for (final long val : data_) {
-        numBitsSet_ += Long.bitCount(val);
-      }
-    }
-    return numBitsSet_;
-  }
+  abstract void reset();
 
-  long getCapacity() { return (long) data_.length * Long.SIZE; }
+  abstract long getCapacity();
 
-  int getArrayLength() { return data_.length; }
+  abstract int getArrayLength();
 
-  // applies logical OR
-  void union(final BitArray other) {
-    if (data_.length != other.data_.length) {
-      throw new SketchesArgumentException("Cannot union bit arrays with unequal lengths");
-    }
+  abstract void union(final BitArray other);
 
-    numBitsSet_ = 0;
-    for (int i = 0; i < data_.length; ++i) {
-      data_[i] |= other.data_[i];
-      numBitsSet_ += Long.bitCount(data_[i]);
-    }
-    isDirty_ = false;
-  }
-
-  // applies logical AND
-  void intersect(final BitArray other) {
-    if (data_.length != other.data_.length) {
-      throw new SketchesArgumentException("Cannot intersect bit arrays with unequal lengths");
-    }
+  abstract void intersect(final BitArray other);
 
-    numBitsSet_ = 0;
-    for (int i = 0; i < data_.length; ++i) {
-      data_[i] &= other.data_[i];
-      numBitsSet_ += Long.bitCount(data_[i]);
-    }
-    isDirty_ = false;
-  }
+  abstract void invert();
 
-  // applies bitwise inversion
-  void invert() {
-    if (isDirty_) {
-      numBitsSet_ = 0;
-      for (int i = 0; i < data_.length; ++i) {
-        data_[i] = ~data_[i];
-        numBitsSet_ += Long.bitCount(data_[i]);
-      }
-      isDirty_ = false;
-    } else {
-      for (int i = 0; i < data_.length; ++i) {
-        data_[i] = ~data_[i];
-      }
-      numBitsSet_ = getCapacity() - numBitsSet_;
+  // prints the raw BitArray as 0s and 1s, one long per row
+  @Override
+  public String toString() {
+    final StringBuilder sb = new StringBuilder();
+    for (int i = 0; i < getArrayLength(); ++i) {
+      sb.append(i + ": ")
+        .append(printLong(getLong(i)))
+        .append("\n");
     }
+    return sb.toString();
   }
 
   long getSerializedSizeBytes() {
     // We only really need an int for array length but this will keep everything
     // aligned to 8 bytes.
     // Always write array length and numBitsSet, even if empty
-    return isEmpty() ? Long.BYTES : Long.BYTES * (2L + data_.length);
+    return Long.BYTES * (isEmpty() ? 2L : (2L + getArrayLength()));
   }
 
-  void writeToBuffer(final WritableBuffer wbuf) {
-    wbuf.putInt(data_.length);
-    wbuf.putInt(0); // unused
+  abstract protected boolean isDirty();
 
-    if (!isEmpty()) {
-      wbuf.putLong(isDirty_ ? -1 : numBitsSet_);
-      wbuf.putLongArray(data_, 0, data_.length);
-    }
-  }
+  // used to get a long from the array regardless of underlying storage
+  // NOT used to query individual bits
+  abstract protected long getLong(final int arrayIndex);
 
-  // prints the raw BitArray as 0s and 1s, one long per row
-  @Override
-  public String toString() {
-    final StringBuilder sb = new StringBuilder();
-    for (int i = 0; i < data_.length; ++i) {
-      sb.append(i + ": ")
-        .append(printLong(data_[i]))
-        .append("\n");
-    }
-    return sb.toString();
-  }
+  // used to set a long in the array regardless of underlying storage
+  // NOT used to set individual bits
+  abstract protected void setLong(final int arrayIndex, final long value);
 
   // prints a long as a series of 0s and 1s as little endian
-  private static String printLong(final long val) {
+  protected static String printLong(final long val) {
     final StringBuilder sb = new StringBuilder();
     for (int j = 0; j < Long.SIZE; ++j) {
       sb.append((val & (1L << j)) != 0 ? "1" : "0");
@@ -213,10 +102,4 @@ final class BitArray {
     return sb.toString();
   }
 
-  // clears the array
-  void reset() {
-    Arrays.fill(data_, 0);
-    numBitsSet_ = 0;
-    isDirty_ = false;
-  }
 }
diff --git a/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilter.java b/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilter.java
index 74cc2e4f..0c45f780 100644
--- a/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilter.java
+++ b/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilter.java
@@ -66,11 +66,12 @@ public final class BloomFilter {
   private long seed_;            // hash seed
   private short numHashes_;      // number of hash values
   private BitArray bitArray_;    // the actual data bits
+  private WritableMemory wmem_;  // used only for direct mode BitArray
 
   /**
    * Creates a BloomFilter with given number of bits and number of hash functions,
    * and a user-specified  seed.
-   * 
+   *
    * @param numBits The size of the BloomFilter, in bits
    * @param numHashes The number of hash functions to apply to items
    * @param seed The base hash seed
@@ -78,7 +79,8 @@ public final class BloomFilter {
   BloomFilter(final long numBits, final int numHashes, final long seed) {
     seed_ = seed;
     numHashes_ = (short) numHashes;
-    bitArray_ = new BitArray(numBits);
+    bitArray_ = new HeapBitArray(numBits);
+    wmem_ = null;
   }
 
   // Constructor used with heapify()
@@ -86,6 +88,7 @@ public final class BloomFilter {
     seed_ = seed;
     numHashes_ = numHashes;
     bitArray_ = bitArray;
+    wmem_ = null;
   }
 
   /**
@@ -104,7 +107,7 @@ public final class BloomFilter {
       "Possible corruption: Incorrect number of preamble bytes specified in header");
     checkArgument(serVer != SER_VER, "Possible corruption: Unrecognized serialization version: " + serVer);
     checkArgument(familyID != Family.BLOOMFILTER.getID(), "Possible corruption: Incorrect FamilyID for bloom filter. Found: " + familyID);
-    
+
     final short numHashes = buf.getShort();
     buf.getShort(); // unused
     checkArgument(numHashes < 1, "Possible corruption: Need strictly positive number of hash functions. Found: " + numHashes);
@@ -130,7 +133,7 @@ public final class BloomFilter {
    * @return True if the BloomFilter is empty, otherwise False
    */
   public boolean isEmpty() { return bitArray_.isEmpty(); }
-  
+
   /**
    * Returns the number of bits in the BloomFilter that are set to 1.
    * @return The number of bits in use in this filter
@@ -180,7 +183,7 @@ public final class BloomFilter {
    * @param item an item with which to update the filter
    */
   public void update(final double item) {
-    // canonicalize all NaN & +/- infinity forms    
+    // canonicalize all NaN & +/- infinity forms
     final long[] data = { Double.doubleToLongBits(item) };
     final long h0 = XxHash.hashLongArr(data, 0, 1, seed_);
     final long h1 = XxHash.hashLongArr(data, 0, 1, h0);
@@ -281,10 +284,10 @@ public final class BloomFilter {
     }
   }
 
-  // QUERY-AND-UPDATE METHODS 
+  // QUERY-AND-UPDATE METHODS
 
   /**
-   * Updates the filter with the provided long and 
+   * Updates the filter with the provided long and
    * returns the result from quering that value prior to the update.
    * @param item an item with which to update the filter
    * @return The query result prior to applying the update
@@ -294,16 +297,16 @@ public final class BloomFilter {
     final long h1 = XxHash.hashLong(item, h0);
     return queryAndUpdateInternal(h0, h1);
   }
-  
+
   /**
-   * Updates the filter with the provided double and 
+   * Updates the filter with the provided double and
    * returns the result from quering that value prior to the update.
    * The double is canonicalized (NaN and +/- infinity) in the call.
    * @param item an item with which to update the filter
    * @return The query result prior to applying the update
    */
   public boolean queryAndUpdate(final double item) {
-    // canonicalize all NaN & +/- infinity forms    
+    // canonicalize all NaN & +/- infinity forms
     final long[] data = { Double.doubleToLongBits(item) };
     final long h0 = XxHash.hashLongArr(data, 0, 1, seed_);
     final long h1 = XxHash.hashLongArr(data, 0, 1, h0);
@@ -311,7 +314,7 @@ public final class BloomFilter {
   }
 
   /**
-   * Updates the filter with the provided String and 
+   * Updates the filter with the provided String and
    * returns the result from quering that value prior to the update.
    * The string is converted to a byte array using UTF8 encoding.
    *
@@ -331,7 +334,7 @@ public final class BloomFilter {
   }
 
   /**
-   * Updates the filter with the provided byte[] and 
+   * Updates the filter with the provided byte[] and
    * returns the result from quering that array prior to the update.
    * @param data an array with which to update the filter
    * @return The query result prior to applying the update, or false if data is null
@@ -341,9 +344,9 @@ public final class BloomFilter {
     final long h1 = XxHash.hashByteArr(data, 0, data.length, h0);
     return queryAndUpdateInternal(h0, h1);
   }
-  
+
   /**
-   * Updates the filter with the provided char[] and 
+   * Updates the filter with the provided char[] and
    * returns the result from quering that array prior to the update.
    * @param data an array with which to update the filter
    * @return The query result prior to applying the update, or false if data is null
@@ -356,7 +359,7 @@ public final class BloomFilter {
   }
 
   /**
-   * Updates the filter with the provided short[] and 
+   * Updates the filter with the provided short[] and
    * returns the result from quering that array prior to the update.
    * @param data an array with which to update the filter
    * @return The query result prior to applying the update, or false if data is null
@@ -369,7 +372,7 @@ public final class BloomFilter {
   }
 
   /**
-   * Updates the filter with the provided int[] and 
+   * Updates the filter with the provided int[] and
    * returns the result from quering that array prior to the update.
    * @param data an array with which to update the filter
    * @return The query result prior to applying the update, or false if data is null
@@ -382,7 +385,7 @@ public final class BloomFilter {
   }
 
   /**
-   * Updates the filter with the provided long[] and 
+   * Updates the filter with the provided long[] and
    * returns the result from quering that array prior to the update.
    * @param data an array with which to update the filter
    * @return The query result prior to applying the update, or false if data is null
@@ -395,7 +398,7 @@ public final class BloomFilter {
   }
 
   /**
-   * Updates the filter with the provided Memory and 
+   * Updates the filter with the provided Memory and
    * returns the result from quering that Memory prior to the update.
    * @param mem an array with which to update the filter
    * @return The query result prior to applying the update, or false if mem is null
@@ -444,7 +447,7 @@ public final class BloomFilter {
    * @return The result of querying the filter with the given item
    */
   public boolean query(final double item) {
-    // canonicalize all NaN & +/- infinity forms    
+    // canonicalize all NaN & +/- infinity forms
     final long[] data = { Double.doubleToLongBits(item) };
     final long h0 = XxHash.hashLongArr(data, 0, 1, seed_);
     final long h1 = XxHash.hashLongArr(data, 0, 1, h0);
@@ -466,7 +469,7 @@ public final class BloomFilter {
    * @return The result of querying the filter with the given item, or false if item is null
    */
   public boolean query(final String item) {
-    if (item == null || item.isEmpty()) { return false; }    
+    if (item == null || item.isEmpty()) { return false; }
     final byte[] strBytes = item.getBytes(StandardCharsets.UTF_8);
     final long h0 = XxHash.hashByteArr(strBytes, 0, strBytes.length, seed_);
     final long h1 = XxHash.hashByteArr(strBytes, 0, strBytes.length, h0);
@@ -655,7 +658,7 @@ public final class BloomFilter {
  *      ||      24        |   25   |   26   |   27   |   28   |   29   |   30   |   31   |
  *  2   ||---------------------------------NumBitsSet------------------------------------|
  *  </pre>
- * 
+ *
  * The raw BitArray bits, if non-empty start at byte 24.
  */
 
diff --git a/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilterBuilder.java b/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilterBuilder.java
index 3e3659bd..6d4c86d6 100644
--- a/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilterBuilder.java
+++ b/src/main/java/org/apache/datasketches/filters/bloomfilter/BloomFilterBuilder.java
@@ -84,7 +84,7 @@ public final class BloomFilterBuilder {
    * @param targetFalsePositiveProb A desired false positive probability per item
    * @return A new BloomFilter configured for the given input parameters
    */
-  public static BloomFilter createByAccuracy(final long maxDistinctItems, final double targetFalsePositiveProb) {
+  public static HeapBloomFilter createByAccuracy(final long maxDistinctItems, final double targetFalsePositiveProb) {
     return createByAccuracy(maxDistinctItems, targetFalsePositiveProb, ThreadLocalRandom.current().nextLong());
   }
 
@@ -96,7 +96,7 @@ public final class BloomFilterBuilder {
    * @param seed A base hash seed
    * @return A new BloomFilter configured for the given input parameters
    */
-  public static BloomFilter createByAccuracy(final long maxDistinctItems, final double targetFalsePositiveProb, final long seed) {
+  public static HeapBloomFilter createByAccuracy(final long maxDistinctItems, final double targetFalsePositiveProb, final long seed) {
     if (maxDistinctItems <= 0) {
       throw new SketchesArgumentException("maxDistinctItems must be strictly positive");
     }
@@ -105,7 +105,7 @@ public final class BloomFilterBuilder {
     }
     final long numBits = suggestNumFilterBits(maxDistinctItems, targetFalsePositiveProb);
     final short numHashes = suggestNumHashes(maxDistinctItems, numBits);
-    return new BloomFilter(numBits, numHashes, seed);
+    return new HeapBloomFilter(numBits, numHashes, seed);
   }
 
   /**
@@ -116,7 +116,7 @@ public final class BloomFilterBuilder {
    * @param numHashes The number of hash functions to apply to items
    * @return A new BloomFilter configured for the given input parameters
    */
-  public static BloomFilter createBySize(final long numBits, final int numHashes) {
+  public static HeapBloomFilter createBySize(final long numBits, final int numHashes) {
     return createBySize(numBits, numHashes, ThreadLocalRandom.current().nextLong());
   }
 
@@ -129,10 +129,10 @@ public final class BloomFilterBuilder {
    * @param seed A base hash seed
    * @return A new BloomFilter configured for the given input parameters
    */
-  public static BloomFilter createBySize(final long numBits, final int numHashes, final long seed) {
-    if (numBits > BloomFilter.MAX_SIZE) {
+  public static HeapBloomFilter createBySize(final long numBits, final int numHashes, final long seed) {
+    if (numBits > HeapBloomFilter.MAX_SIZE) {
       throw new SketchesArgumentException("Size of BloomFilter must be <= "
-      + BloomFilter.MAX_SIZE + ". Requested: " + numBits);
+      + HeapBloomFilter.MAX_SIZE + ". Requested: " + numBits);
     }
     if (numHashes < 1) {
       throw new SketchesArgumentException("Must specify a strictly positive number of hash functions. "
@@ -143,6 +143,6 @@ public final class BloomFilterBuilder {
       + ". Requested: " + numHashes);
     }
 
-    return new BloomFilter(numBits, numHashes, seed);
+    return new HeapBloomFilter(numBits, numHashes, seed);
   }
 }
diff --git a/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArray.java b/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArray.java
new file mode 100644
index 00000000..4d0509bd
--- /dev/null
+++ b/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArray.java
@@ -0,0 +1,175 @@
+/*
+ * 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.filters.bloomfilter;
+
+import org.apache.datasketches.common.SketchesArgumentException;
+import org.apache.datasketches.memory.WritableMemory;
+
+final class DirectBitArray extends DirectBitArrayR {
+
+  DirectBitArray(final int dataLength, final long storedNumBitsSet, final WritableMemory mem) {
+    super(dataLength, 0, mem); // we'll set numBitsSet_ ourselves so pass 0
+
+    // can recompute later if needed
+    numBitsSet_ = storedNumBitsSet;
+  }
+
+  static DirectBitArray wrap(final WritableMemory mem) {
+    final int arrayLength = mem.getInt(0);
+    final long storedNumBitsSet = mem.getLong(NUM_BITS_OFFSET);
+
+    if (arrayLength * (long) Long.SIZE > MAX_BITS) {
+      throw new SketchesArgumentException("Possible corruption: Serialized image indicates array beyond maximum filter capacity");
+    }
+
+    // if empty cannot wrap as writable
+    if (storedNumBitsSet == 0) {
+      throw new SketchesArgumentException("Cannot wrap an empty filter for writing as there is no backing data array");
+    }
+
+    // required capacity is arrayLength plus room for
+    // arrayLength (in longs) and numBitsSet
+    if (storedNumBitsSet > 0 && mem.getCapacity() < arrayLength + 2) {
+      throw new SketchesArgumentException("Memory capacity insufficient for Bloom Filter. Needed: "
+        + (arrayLength + 2) + " , found: " + mem.getCapacity());
+    }
+    return new DirectBitArray(arrayLength, storedNumBitsSet, mem);
+  }
+
+  @Override
+  long getNumBitsSet() {
+    // update numBitsSet and store in array
+    if (isDirty()) {
+      numBitsSet_ = 0;
+      for (int i = 0; i < dataLength_; ++i) {
+        numBitsSet_ += Long.bitCount(getLong(i));
+      }
+      wmem_.putLong(NUM_BITS_OFFSET, numBitsSet_);
+    }
+
+    return numBitsSet_;
+  }
+
+  @Override
+  protected boolean isDirty() {
+    return numBitsSet_ == -1;
+  }
+
+  @Override
+  boolean getBit(final long index) {
+    return (wmem_.getByte(DATA_OFFSET + ((int) index >>> 6)) & (1L << index)) != 0 ? true : false;
+  }
+
+  @Override
+  protected long getLong(final int arrayIndex) {
+    return wmem_.getLong(DATA_OFFSET + arrayIndex);
+  }
+
+  @Override
+  public boolean isReadOnly() {
+    return false;
+  }
+
+  @Override
+  void reset() {
+    setNumBitsSet(0);
+    wmem_.clear(DATA_OFFSET, dataLength_ * Long.BYTES);
+  }
+
+  @Override
+  void setBit(final long index) {
+    final int idx = (int) index >>> 6;
+    final long val = getLong(idx);
+    setLong(idx, val | 1L << index);
+    setNumBitsSet(-1); // mark dirty
+  }
+
+  @Override
+  boolean getAndSetBit(final long index) {
+    final int offset = (int) index >>> 6;
+    final long mask = 1L << index;
+    final long val = getLong(offset);
+    if ((val & mask) != 0) {
+      return true; // already seen
+    } else {
+      setLong(offset, val | mask);
+      if (!isDirty()) { setNumBitsSet(numBitsSet_ + 1); }
+      return false; // new set
+    }
+  }
+
+  @Override
+  void intersect(final BitArray other) {
+    if (getCapacity() != other.getCapacity()) {
+      throw new SketchesArgumentException("Cannot intersect bit arrays with unequal lengths");
+    }
+
+    numBitsSet_ = 0;
+    for (int i = 0; i < dataLength_; ++i) {
+      final long val = getLong(i) & other.getLong(i);
+      numBitsSet_ += Long.bitCount(val);
+      setLong(i, val);
+    }
+    wmem_.putLong(NUM_BITS_OFFSET, numBitsSet_);
+  }
+
+  @Override
+  void union(final BitArray other) {
+    if (getCapacity() != other.getCapacity()) {
+      throw new SketchesArgumentException("Cannot intersect bit arrays with unequal lengths");
+    }
+
+    numBitsSet_ = 0;
+    for (int i = 0; i < dataLength_; ++i) {
+      final long val = getLong(i) | other.getLong(i);
+      numBitsSet_ += Long.bitCount(val);
+      setLong(i, val);
+    }
+    wmem_.putLong(NUM_BITS_OFFSET, numBitsSet_);
+  }
+
+  @Override
+  void invert() {
+    if (isDirty()) {
+      numBitsSet_ = 0;
+      for (int i = 0; i < dataLength_; ++i) {
+        final long val = ~getLong(i);
+        setLong(i, val);
+        numBitsSet_ += Long.bitCount(val);
+      }
+    } else {
+      for (int i = 0; i < dataLength_; ++i) {
+        setLong(i, ~getLong(i));
+      }
+      numBitsSet_ = getCapacity() - numBitsSet_;
+    }
+    wmem_.putLong(NUM_BITS_OFFSET, numBitsSet_);
+  }
+
+  @Override
+  protected void setLong(final int arrayIndex, final long value) {
+    wmem_.putLong(DATA_OFFSET + arrayIndex, value);
+  }
+
+  private final void setNumBitsSet(final long numBitsSet) {
+    numBitsSet_ = 0;
+    wmem_.putLong(NUM_BITS_OFFSET, numBitsSet);
+  }
+}
diff --git a/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayR.java b/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayR.java
new file mode 100644
index 00000000..d938380d
--- /dev/null
+++ b/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayR.java
@@ -0,0 +1,152 @@
+/*
+ * 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.filters.bloomfilter;
+
+import org.apache.datasketches.common.SketchesArgumentException;
+import org.apache.datasketches.common.SketchesReadOnlyException;
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+
+public class DirectBitArrayR extends BitArray {
+  final static protected long NUM_BITS_OFFSET = Long.BYTES;
+  final static protected long DATA_OFFSET = 2 * Long.BYTES;
+
+  final protected int dataLength_;
+  final protected WritableMemory wmem_; // for inheritance; we won't write to it
+  protected long numBitsSet_; // could be final here but writable direct will update it
+
+  protected DirectBitArrayR(final int dataLength, final long storedNumBitsSet, final Memory mem) {
+    super();
+
+    dataLength_ = dataLength;
+    wmem_ = (WritableMemory) mem;
+
+    if (storedNumBitsSet == -1) {
+      numBitsSet_ = 0;
+      for (int i = 0; i < dataLength_; ++i) {
+        numBitsSet_ += getLong(i);
+      }
+    } else {
+      numBitsSet_ = storedNumBitsSet;
+    }
+  }
+
+  // assumes we have a region with only the portion of Memory
+  // the BitArray cares about
+  static DirectBitArrayR wrap(final Memory mem) {
+    final int arrayLength = mem.getInt(0);
+    final long storedNumBitsSet = mem.getLong(NUM_BITS_OFFSET);
+
+    if (arrayLength * (long) Long.SIZE > MAX_BITS) {
+      throw new SketchesArgumentException("Possible corruption: Serialized image indicates array beyond maximum filter capacity");
+    }
+
+    // required capacity is arrayLength plus room for
+    // arrayLength (in longs) and numBitsSet
+    if (storedNumBitsSet > 0 && mem.getCapacity() < arrayLength + 2) {
+      throw new SketchesArgumentException("Memory capacity insufficient for Bloom Filter. Needed: "
+        + (arrayLength + 2) + " , found: " + mem.getCapacity());
+    }
+    return new DirectBitArrayR(arrayLength, storedNumBitsSet, mem);
+  }
+
+  @Override
+  long getCapacity() {
+    return dataLength_ * Long.SIZE;
+  }
+
+  @Override
+  long getNumBitsSet() {
+    return numBitsSet_;
+  }
+
+  @Override
+  protected boolean isDirty() {
+    // read-only so necessarily false
+    return false;
+  }
+
+  @Override
+  int getArrayLength() {
+    return dataLength_;
+  }
+
+  @Override
+  boolean getBit(final long index) {
+    if (isEmpty()) { return false; }
+    return (wmem_.getByte(DATA_OFFSET + ((int) index >>> 6)) & (1L << index)) != 0 ? true : false;
+  }
+
+  @Override
+  protected long getLong(final int arrayIndex) {
+    if (isEmpty()) { return 0L; }
+    return wmem_.getLong(DATA_OFFSET + arrayIndex);
+  }
+
+  @Override
+  public boolean hasMemory() {
+    return (wmem_ != null);
+  }
+
+  @Override
+  public boolean isDirect() {
+    return (wmem_ != null) ? wmem_.isDirect() : false;
+  }
+
+  @Override
+  public boolean isReadOnly() {
+    return true;
+  }
+
+  @Override
+  void reset() {
+    throw new SketchesReadOnlyException("Attempt to call reset() on read-only memory");
+  }
+
+  @Override
+  void setBit(final long index) {
+    throw new SketchesReadOnlyException("Attempt to call setBit() on read-only memory");
+  }
+
+  @Override
+  boolean getAndSetBit(final long index) {
+    throw new SketchesReadOnlyException("Attempt to call getAndSetBit() on read-only memory");
+  }
+
+  @Override
+  void intersect(final BitArray other) {
+    throw new SketchesReadOnlyException("Attempt to call intersect() on read-only memory");
+  }
+
+  @Override
+  void union(final BitArray other) {
+    throw new SketchesReadOnlyException("Attempt to call union() on read-only memory");
+  }
+
+  @Override
+  void invert() {
+    throw new SketchesReadOnlyException("Attempt to call invert() on read-only memory");
+  }
+
+  @Override
+  protected void setLong(final int arrayIndex, final long value) {
+    throw new SketchesReadOnlyException("Attempt to call setLong() on read-only memory");
+  }
+}
diff --git a/src/main/java/org/apache/datasketches/filters/bloomfilter/BitArray.java b/src/main/java/org/apache/datasketches/filters/bloomfilter/HeapBitArray.java
similarity index 74%
copy from src/main/java/org/apache/datasketches/filters/bloomfilter/BitArray.java
copy to src/main/java/org/apache/datasketches/filters/bloomfilter/HeapBitArray.java
index 91c4f5ef..a003ce2a 100644
--- a/src/main/java/org/apache/datasketches/filters/bloomfilter/BitArray.java
+++ b/src/main/java/org/apache/datasketches/filters/bloomfilter/HeapBitArray.java
@@ -31,16 +31,15 @@ import org.apache.datasketches.memory.WritableBuffer;
  * <p>Rounds the number of bits up to the smallest multiple of 64 (one long)
  * that is not smaller than the specified number.
  */
-final class BitArray {
-  // MAX_BITS using longs, based on array indices being capped at Integer.MAX_VALUE
-  private static final long MAX_BITS = Integer.MAX_VALUE * (long) Long.SIZE;
-
+final class HeapBitArray extends BitArray {
   private long numBitsSet_;  // if -1, need to recompute value
   private boolean isDirty_;
-  private long[] data_;
+  final private long[] data_;
 
   // creates an array of a given size
-  BitArray(final long numBits) {
+  HeapBitArray(final long numBits) {
+    super();
+
     if (numBits <= 0) {
       throw new SketchesArgumentException("Number of bits must be strictly positive. Found: " + numBits);
     }
@@ -55,7 +54,9 @@ final class BitArray {
   }
 
   // uses the provided array
-  BitArray(final long numBitsSet, final long[] data) {
+  HeapBitArray(final long numBitsSet, final long[] data) {
+    super();
+
     data_ = data;
     isDirty_ = numBitsSet < 0;
     numBitsSet_ = numBitsSet;
@@ -63,14 +64,14 @@ final class BitArray {
 
   // reads a serialized image, but the BitArray is not fully self-describing so requires
   // a flag to indicate whether the array is empty
-  static BitArray heapify(final Buffer buffer, final boolean isEmpty) {
+  static HeapBitArray heapify(final Buffer buffer, final boolean isEmpty) {
     final int numLongs = buffer.getInt();
     if (numLongs < 0) {
       throw new SketchesArgumentException("Possible corruption: Must have strictly positive array size. Found: " + numLongs);
     }
 
     if (isEmpty) {
-      return new BitArray((long) numLongs * Long.SIZE);
+      return new HeapBitArray((long) numLongs * Long.SIZE);
     }
 
     buffer.getInt(); // unused
@@ -80,26 +81,40 @@ final class BitArray {
 
     final long[] data = new long[numLongs];
     buffer.getLongArray(data, 0, numLongs);
-    return new BitArray(numBitsSet, data);
+    return new HeapBitArray(numBitsSet, data);
   }
 
-  boolean isEmpty() {
-    return getNumBitsSet() == 0 && !isDirty_;
+  @Override
+  protected boolean isDirty() {
+    return isDirty_;
+  }
+
+  @Override
+  boolean hasMemory() {
+    return false;
+  }
+
+  @Override
+  boolean isDirect() {
+    return false;
   }
 
   // queries a single bit in the array
+  @Override
   boolean getBit(final long index) {
     return (data_[(int) index >>> 6] & (1L << index)) != 0 ? true : false;
   }
 
   // sets a single bit in the array without querying, meaning the method
   // cannot properly track the number of bits set so set isDirty = true
+  @Override
   void setBit(final long index) {
     data_[(int) index >>> 6] |= 1L << index;
     isDirty_ = true;
   }
 
   // returns existing value of bit
+  @Override
   boolean getAndSetBit(final long index) {
     final int offset = (int) index >>> 6;
     final long mask = 1L << index;
@@ -115,6 +130,7 @@ final class BitArray {
   // may need to recompute value:
   // O(1) if only getAndSetBit() has been used
   // O(data_.length) if setBit() has ever been used
+  @Override
   long getNumBitsSet() {
     if (isDirty_) {
       numBitsSet_ = 0;
@@ -125,39 +141,46 @@ final class BitArray {
     return numBitsSet_;
   }
 
+  @Override
   long getCapacity() { return (long) data_.length * Long.SIZE; }
 
+  @Override
   int getArrayLength() { return data_.length; }
 
   // applies logical OR
+  @Override
   void union(final BitArray other) {
-    if (data_.length != other.data_.length) {
+    if (getCapacity() != other.getCapacity()) {
       throw new SketchesArgumentException("Cannot union bit arrays with unequal lengths");
     }
 
     numBitsSet_ = 0;
     for (int i = 0; i < data_.length; ++i) {
-      data_[i] |= other.data_[i];
-      numBitsSet_ += Long.bitCount(data_[i]);
+      final long val = data_[i] | other.getLong(i);
+      numBitsSet_ += Long.bitCount(val);
+      data_[i] = val;
     }
     isDirty_ = false;
   }
 
   // applies logical AND
+  @Override
   void intersect(final BitArray other) {
-    if (data_.length != other.data_.length) {
+    if (getCapacity() != other.getCapacity()) {
       throw new SketchesArgumentException("Cannot intersect bit arrays with unequal lengths");
     }
 
     numBitsSet_ = 0;
     for (int i = 0; i < data_.length; ++i) {
-      data_[i] &= other.data_[i];
-      numBitsSet_ += Long.bitCount(data_[i]);
+      final long val = data_[i] & other.getLong(i);
+      numBitsSet_ += Long.bitCount(val);
+      data_[i] = val;
     }
     isDirty_ = false;
   }
 
   // applies bitwise inversion
+  @Override
   void invert() {
     if (isDirty_) {
       numBitsSet_ = 0;
@@ -174,13 +197,6 @@ final class BitArray {
     }
   }
 
-  long getSerializedSizeBytes() {
-    // We only really need an int for array length but this will keep everything
-    // aligned to 8 bytes.
-    // Always write array length and numBitsSet, even if empty
-    return isEmpty() ? Long.BYTES : Long.BYTES * (2L + data_.length);
-  }
-
   void writeToBuffer(final WritableBuffer wbuf) {
     wbuf.putInt(data_.length);
     wbuf.putInt(0); // unused
@@ -191,29 +207,18 @@ final class BitArray {
     }
   }
 
-  // prints the raw BitArray as 0s and 1s, one long per row
   @Override
-  public String toString() {
-    final StringBuilder sb = new StringBuilder();
-    for (int i = 0; i < data_.length; ++i) {
-      sb.append(i + ": ")
-        .append(printLong(data_[i]))
-        .append("\n");
-    }
-    return sb.toString();
+  protected long getLong(final int arrayIndex) {
+    return data_[arrayIndex];
   }
 
-  // prints a long as a series of 0s and 1s as little endian
-  private static String printLong(final long val) {
-    final StringBuilder sb = new StringBuilder();
-    for (int j = 0; j < Long.SIZE; ++j) {
-      sb.append((val & (1L << j)) != 0 ? "1" : "0");
-      if (j % 8 == 7) { sb.append(" "); }
-    }
-    return sb.toString();
+  @Override
+  protected void setLong(final int arrayIndex, final long value) {
+    data_[arrayIndex] = value;
   }
 
   // clears the array
+  @Override
   void reset() {
     Arrays.fill(data_, 0);
     numBitsSet_ = 0;
diff --git a/src/test/java/org/apache/datasketches/filters/bloomfilter/BitArrayTest.java b/src/test/java/org/apache/datasketches/filters/bloomfilter/BitArrayTest.java
index d3249f65..e1340708 100644
--- a/src/test/java/org/apache/datasketches/filters/bloomfilter/BitArrayTest.java
+++ b/src/test/java/org/apache/datasketches/filters/bloomfilter/BitArrayTest.java
@@ -32,7 +32,7 @@ public class BitArrayTest {
 
   @Test
   public void createBitArrayTest() {
-    final BitArray ba = new BitArray(119);
+    final HeapBitArray ba = new HeapBitArray(119);
     assertEquals(ba.getCapacity(), 128); // nearest multiple of 64
     assertEquals(ba.getArrayLength(), 2);
     assertEquals(ba.getNumBitsSet(), 0);
@@ -41,17 +41,17 @@ public class BitArrayTest {
 
   @Test(expectedExceptions = SketchesArgumentException.class)
   public void createNegativeSizeBitArrayTest() {
-    new BitArray(-64);
+    new HeapBitArray(-64);
   }
 
   @Test(expectedExceptions = SketchesArgumentException.class)
   public void createTooLargeBitArrayTest() {
-    new BitArray(1L + (long) Integer.MAX_VALUE * Long.SIZE);
+    new HeapBitArray(1L + (long) Integer.MAX_VALUE * Long.SIZE);
   }
 
   @Test
   public void basicOperationTest() {
-    final BitArray ba = new BitArray(128);
+    final HeapBitArray ba = new HeapBitArray(128);
     assertEquals(ba.getAndSetBit(1), false);
     assertEquals(ba.getAndSetBit(2), false);
     for (int i = 4; i < 64; i <<= 1) {
@@ -60,7 +60,7 @@ public class BitArrayTest {
     assertEquals(ba.getNumBitsSet(), 6);
     assertEquals(ba.getBit(68), true);
     assertFalse(ba.isEmpty());
-    
+
     assertFalse(ba.getBit(5));
     ba.setBit(5);
     assertTrue(ba.getAndSetBit(5));
@@ -74,7 +74,7 @@ public class BitArrayTest {
   @Test
   public void inversionTest() {
     final int numBits = 1024;
-    final BitArray ba = new BitArray(numBits);
+    final HeapBitArray ba = new HeapBitArray(numBits);
     for (int i = 0; i < numBits; i += numBits / 8) {
       ba.getAndSetBit(i);
     }
@@ -84,27 +84,27 @@ public class BitArrayTest {
     ba.invert();
 
     assertEquals(ba.getNumBitsSet(), numBits - numSet);
-    assertFalse(ba.getBit(0));    
+    assertFalse(ba.getBit(0));
   }
 
   @Test(expectedExceptions = SketchesArgumentException.class)
   public void invalidUnionTest() {
-    final BitArray ba = new BitArray(128);
-    ba.union(new BitArray(64));
+    final HeapBitArray ba = new HeapBitArray(128);
+    ba.union(new HeapBitArray(64));
   }
 
   @Test(expectedExceptions = SketchesArgumentException.class)
   public void invalidIntersectionTest() {
-    final BitArray ba = new BitArray(128);
-    ba.intersect(new BitArray(64));
+    final HeapBitArray ba = new HeapBitArray(128);
+    ba.intersect(new HeapBitArray(64));
   }
 
   @Test
   public void validUnionAndIntersectionTest() {
-    final BitArray ba1 = new BitArray(64);
-    final BitArray ba2 = new BitArray(64);
-    final BitArray ba3 = new BitArray(64);
-    
+    final HeapBitArray ba1 = new HeapBitArray(64);
+    final HeapBitArray ba2 = new HeapBitArray(64);
+    final HeapBitArray ba3 = new HeapBitArray(64);
+
     final int n = 10;
     for (int i = 0; i < n; ++i) {
       ba1.getAndSetBit(i);
@@ -124,11 +124,11 @@ public class BitArrayTest {
 
   @Test
   public void serializeEmptyTest() {
-    final BitArray ba = new BitArray(64);
+    final HeapBitArray ba = new HeapBitArray(64);
     final WritableBuffer wbuf = WritableMemory.allocate((int) ba.getSerializedSizeBytes()).asWritableBuffer();
     ba.writeToBuffer(wbuf);
     wbuf.resetPosition();
-    final BitArray newBA = BitArray.heapify(wbuf, true);
+    final HeapBitArray newBA = HeapBitArray.heapify(wbuf, true);
     assertEquals(newBA.getArrayLength(), ba.getArrayLength());
     assertEquals(newBA.getCapacity(), ba.getCapacity());
     assertEquals(newBA.getNumBitsSet(), ba.getNumBitsSet());
@@ -138,13 +138,13 @@ public class BitArrayTest {
   @Test
   public void serializeNonEmptyTest() {
     final long n = 8192;
-    final BitArray ba = new BitArray(n);
+    final HeapBitArray ba = new HeapBitArray(n);
     for (int i = 0; i < n; i += 3)
       ba.getAndSetBit(i);
     final WritableBuffer wbuf = WritableMemory.allocate((int) ba.getSerializedSizeBytes()).asWritableBuffer();
     ba.writeToBuffer(wbuf);
     wbuf.resetPosition();
-    final BitArray newBA = BitArray.heapify(wbuf, false);
+    final HeapBitArray newBA = HeapBitArray.heapify(wbuf, false);
     assertEquals(newBA.getArrayLength(), ba.getArrayLength());
     assertEquals(newBA.getCapacity(), ba.getCapacity());
     assertEquals(newBA.getNumBitsSet(), ba.getNumBitsSet());
diff --git a/src/test/java/org/apache/datasketches/filters/bloomfilter/BloomFilterBuilderTest.java b/src/test/java/org/apache/datasketches/filters/bloomfilter/BloomFilterBuilderTest.java
index c3b927d4..5f726d88 100644
--- a/src/test/java/org/apache/datasketches/filters/bloomfilter/BloomFilterBuilderTest.java
+++ b/src/test/java/org/apache/datasketches/filters/bloomfilter/BloomFilterBuilderTest.java
@@ -61,12 +61,12 @@ public class BloomFilterBuilderTest {
     // invalid number of hashes
     assertThrows(SketchesArgumentException.class, () -> BloomFilterBuilder.createBySize(1000, -1, 123));
     assertThrows(SketchesArgumentException.class, () -> BloomFilterBuilder.createBySize(1000, 65536, 123));
-  
+
     // invalid number of bits
     assertThrows(SketchesArgumentException.class, () -> BloomFilterBuilder.createBySize(0, 3, 456));
-    assertThrows(SketchesArgumentException.class, () -> BloomFilterBuilder.createBySize(BloomFilter.MAX_SIZE + 1, 3, 456));
-  
-    final BloomFilter bf = BloomFilterBuilder.createBySize(1L << 21, 3);
+    assertThrows(SketchesArgumentException.class, () -> BloomFilterBuilder.createBySize(HeapBloomFilter.MAX_SIZE + 1, 3, 456));
+
+    final HeapBloomFilter bf = BloomFilterBuilder.createBySize(1L << 21, 3);
     assertEquals(bf.getCapacity(), 1 << 21L);
     assertEquals(bf.getNumHashes(), 3);
     assertEquals(bf.getBitsUsed(), 0);
@@ -85,7 +85,7 @@ public class BloomFilterBuilderTest {
 
     // manually computed values based on formula
     assertEquals(BloomFilterBuilder.suggestNumFilterBits(250_000, 0.01), 2396265);
-    BloomFilter bf = BloomFilterBuilder.createByAccuracy(250_000, 0.01);
+    HeapBloomFilter bf = BloomFilterBuilder.createByAccuracy(250_000, 0.01);
     assertEquals(bf.getCapacity(), 2396288); // next smallest multiple of 64
     assertEquals(bf.getNumHashes(), BloomFilterBuilder.suggestNumHashes(250_000, 2396288));
 
diff --git a/src/test/java/org/apache/datasketches/filters/bloomfilter/BloomFilterTest.java b/src/test/java/org/apache/datasketches/filters/bloomfilter/BloomFilterTest.java
index 1fbf1ec0..e6c2a048 100644
--- a/src/test/java/org/apache/datasketches/filters/bloomfilter/BloomFilterTest.java
+++ b/src/test/java/org/apache/datasketches/filters/bloomfilter/BloomFilterTest.java
@@ -34,13 +34,13 @@ public class BloomFilterTest {
   public void basicFilterOperationsTest() {
     final long numBits = 8192;
     final int numHashes = 3;
-    
-    final BloomFilter bf = BloomFilterBuilder.createBySize(numBits, numHashes);
+
+    final HeapBloomFilter bf = BloomFilterBuilder.createBySize(numBits, numHashes);
     assertTrue(bf.isEmpty());
     assertEquals(bf.getCapacity(), numBits); // n is multiple of 64 so should be exact
     assertEquals(bf.getNumHashes(), numHashes);
     assertEquals(bf.getBitsUsed(), 0);
-    
+
     final long n = 1000;
     for (long i = 0; i < n; ++i) {
       bf.queryAndUpdate(i);
@@ -64,14 +64,14 @@ public class BloomFilterTest {
   public void inversionTest() {
     final long numBits = 8192;
     final int numHashes = 3;
-    
-    final BloomFilter bf = BloomFilterBuilder.createBySize(numBits, numHashes);
+
+    final HeapBloomFilter bf = BloomFilterBuilder.createBySize(numBits, numHashes);
 
     final int n = 500;
     for (int i = 0; i < n; ++i) {
       bf.queryAndUpdate(Integer.toString(i));
     }
-    
+
     final long numBitsSet = bf.getBitsUsed();
     bf.invert();
     assertEquals(bf.getBitsUsed(), numBits - numBitsSet);
@@ -96,18 +96,18 @@ public class BloomFilterTest {
   public void incompatibleSetOperationsTest() {
     final int numBits = 128;
     final int numHashes = 4;
-    final BloomFilter bf1 = BloomFilterBuilder.createBySize(numBits, numHashes);
+    final HeapBloomFilter bf1 = BloomFilterBuilder.createBySize(numBits, numHashes);
 
     // mismatched num bits
-    final BloomFilter bf2 = BloomFilterBuilder.createBySize(numBits * 2, numHashes, bf1.getSeed());
+    final HeapBloomFilter bf2 = BloomFilterBuilder.createBySize(numBits * 2, numHashes, bf1.getSeed());
     assertThrows(SketchesArgumentException.class, () -> bf1.union(bf2));
 
     // mismatched num hashes
-    final BloomFilter bf3 = BloomFilterBuilder.createBySize(numBits, numHashes * 2, bf1.getSeed());
+    final HeapBloomFilter bf3 = BloomFilterBuilder.createBySize(numBits, numHashes * 2, bf1.getSeed());
     assertThrows(SketchesArgumentException.class, () -> bf1.intersect(bf3));
 
     // mismatched seed
-    final BloomFilter bf4 = BloomFilterBuilder.createBySize(numBits, numHashes, bf1.getSeed() - 1);
+    final HeapBloomFilter bf4 = BloomFilterBuilder.createBySize(numBits, numHashes, bf1.getSeed() - 1);
     assertThrows(SketchesArgumentException.class, () -> bf1.union(bf4));
   }
 
@@ -116,8 +116,8 @@ public class BloomFilterTest {
     final long numBits = 12288;
     final int numHashes = 4;
 
-    final BloomFilter bf1 = BloomFilterBuilder.createBySize(numBits, numHashes);
-    final BloomFilter bf2 = BloomFilterBuilder.createBySize(numBits, numHashes, bf1.getSeed());
+    final HeapBloomFilter bf1 = BloomFilterBuilder.createBySize(numBits, numHashes);
+    final HeapBloomFilter bf2 = BloomFilterBuilder.createBySize(numBits, numHashes, bf1.getSeed());
 
     final int n = 1000;
     final int maxItem = 3 * n / 2 - 1;
@@ -125,7 +125,7 @@ public class BloomFilterTest {
       bf1.queryAndUpdate(i);
       bf2.queryAndUpdate(n / 2 + i);
     }
-    
+
     bf1.union(null); // no-op
     bf1.union(bf2);
     for (int i = 0; i < maxItem; ++i) {
@@ -145,8 +145,8 @@ public class BloomFilterTest {
     final long numBits = 8192;
     final int numHashes = 5;
 
-    final BloomFilter bf1 = BloomFilterBuilder.createBySize(numBits, numHashes);
-    final BloomFilter bf2 = BloomFilterBuilder.createBySize(numBits, numHashes, bf1.getSeed());
+    final HeapBloomFilter bf1 = BloomFilterBuilder.createBySize(numBits, numHashes);
+    final HeapBloomFilter bf2 = BloomFilterBuilder.createBySize(numBits, numHashes, bf1.getSeed());
 
     final int n = 1024;
     final int maxItem = 3 * n / 2 - 1;
@@ -154,13 +154,13 @@ public class BloomFilterTest {
       bf1.queryAndUpdate(i);
       bf2.queryAndUpdate(n / 2 + i);
     }
-    
+
     bf1.intersect(null); // no-op
     bf1.intersect(bf2);
     for (int i = n / 2; i < n; ++i) {
       assertTrue(bf1.query(i));
     }
-    
+
     int count = 0;
     for (int i = 0; i < n / 2; ++i) {
       count += bf1.query(i) ? 1 : 0;
@@ -176,18 +176,18 @@ public class BloomFilterTest {
   public void emptySerializationTest() {
     final long numBits = 32768;
     final int numHashes = 7;
-    final BloomFilter bf = BloomFilterBuilder.createBySize(numBits, numHashes);
+    final HeapBloomFilter bf = BloomFilterBuilder.createBySize(numBits, numHashes);
 
     final byte[] bytes = bf.toByteArray();
     Memory mem = Memory.wrap(bytes);
-    final BloomFilter fromBytes = BloomFilter.heapify(mem);
+    final HeapBloomFilter fromBytes = HeapBloomFilter.heapify(mem);
     assertTrue(fromBytes.isEmpty());
     assertEquals(fromBytes.getCapacity(), numBits);
     assertEquals(fromBytes.getNumHashes(), numHashes);
 
     final long[] longs = bf.toLongArray();
     mem = Memory.wrap(longs);
-    final BloomFilter fromLongs = BloomFilter.heapify(mem);
+    final HeapBloomFilter fromLongs = HeapBloomFilter.heapify(mem);
     assertTrue(fromLongs.isEmpty());
     assertEquals(fromLongs.getCapacity(), numBits);
     assertEquals(fromLongs.getNumHashes(), numHashes);
@@ -197,7 +197,7 @@ public class BloomFilterTest {
   public void nonEmptySerializationTest() {
     final long numBits = 32768;
     final int numHashes = 5;
-    final BloomFilter bf = BloomFilterBuilder.createBySize(numBits, numHashes);
+    final HeapBloomFilter bf = BloomFilterBuilder.createBySize(numBits, numHashes);
 
     final int n = 2500;
     for (int i = 0; i < n; ++i) {
@@ -213,7 +213,7 @@ public class BloomFilterTest {
 
     final byte[] bytes = bf.toByteArray();
     Memory mem = Memory.wrap(bytes);
-    final BloomFilter fromBytes = BloomFilter.heapify(mem);
+    final HeapBloomFilter fromBytes = HeapBloomFilter.heapify(mem);
     assertFalse(fromBytes.isEmpty());
     assertEquals(fromBytes.getCapacity(), numBits);
     assertEquals(fromBytes.getBitsUsed(), numBitsSet);
@@ -228,7 +228,7 @@ public class BloomFilterTest {
 
     final long[] longs = bf.toLongArray();
     mem = Memory.wrap(longs);
-    final BloomFilter fromLongs = BloomFilter.heapify(mem);
+    final HeapBloomFilter fromLongs = HeapBloomFilter.heapify(mem);
     assertFalse(fromLongs.isEmpty());
     assertEquals(fromLongs.getCapacity(), numBits);
     assertEquals(fromLongs.getBitsUsed(), numBitsSet);
@@ -246,7 +246,7 @@ public class BloomFilterTest {
   public void testBasicUpdateMethods() {
     final int numDistinct = 100;
     final double fpp = 1e-6;
-    final BloomFilter bf = BloomFilterBuilder.createByAccuracy(numDistinct, fpp);
+    final HeapBloomFilter bf = BloomFilterBuilder.createByAccuracy(numDistinct, fpp);
 
     // empty/null String should do nothing
     bf.update("");
@@ -277,14 +277,14 @@ public class BloomFilterTest {
     // for each BloomFilter update type, call update() then queryAndUpdate(), where
     // the latter should return true. query() should likewise return true.
     // A final intersection should have the same number of bits set as the raw input.
-    final BloomFilter bfMem = BloomFilterBuilder.createByAccuracy(numDistinct, fpp);
+    final HeapBloomFilter bfMem = BloomFilterBuilder.createByAccuracy(numDistinct, fpp);
     bfMem.update(mem);
     assertTrue(bfMem.queryAndUpdate(mem));
     assertTrue(bfMem.query(mem));
     final long numBitsSet = bfMem.getBitsUsed();
     final long seed = bfMem.getSeed();
 
-    final BloomFilter bfBytes = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
+    final HeapBloomFilter bfBytes = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
     final byte[] bytes = new byte[24];
     mem.getByteArray(0, bytes, 0, 24);
     bfBytes.update(bytes);
@@ -292,7 +292,7 @@ public class BloomFilterTest {
     assertTrue(bfBytes.query(bytes));
     assertEquals(bfBytes.getBitsUsed(), numBitsSet);
 
-    final BloomFilter bfChars = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
+    final HeapBloomFilter bfChars = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
     final char[] chars = new char[12];
     mem.getCharArray(0, chars, 0, 12);
     bfChars.update(chars);
@@ -300,7 +300,7 @@ public class BloomFilterTest {
     assertTrue(bfChars.query(chars));
     assertEquals(bfChars.getBitsUsed(), numBitsSet);
 
-    final BloomFilter bfShorts = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
+    final HeapBloomFilter bfShorts = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
     final short[] shorts = new short[12];
     mem.getShortArray(0, shorts, 0, 12);
     bfShorts.update(shorts);
@@ -308,7 +308,7 @@ public class BloomFilterTest {
     assertTrue(bfShorts.query(shorts));
     assertEquals(bfShorts.getBitsUsed(), numBitsSet);
 
-    final BloomFilter bfInts = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
+    final HeapBloomFilter bfInts = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
     final int[] ints = new int[6];
     mem.getIntArray(0, ints, 0, 6);
     bfInts.update(ints);
@@ -316,7 +316,7 @@ public class BloomFilterTest {
     assertTrue(bfInts.query(ints));
     assertEquals(bfInts.getBitsUsed(), numBitsSet);
 
-    final BloomFilter bfLongs = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
+    final HeapBloomFilter bfLongs = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
     final long[] longs = new long[3];
     mem.getLongArray(0, longs, 0, 3);
     bfLongs.update(longs);
@@ -325,7 +325,7 @@ public class BloomFilterTest {
     assertEquals(bfLongs.getBitsUsed(), numBitsSet);
 
     // intersect all the sketches into a new one
-    final BloomFilter bf = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
+    final HeapBloomFilter bf = BloomFilterBuilder.createByAccuracy(numDistinct, fpp, seed);
     bf.intersect(bfMem);
     bf.intersect(bfBytes);
     bf.intersect(bfChars);


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


(datasketches-java) 02/06: WIP: fix wrap methods to take input isEmpty flag

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

jmalkin pushed a commit to branch bloom-direct
in repository https://gitbox.apache.org/repos/asf/datasketches-java.git

commit 312f3d491fb44c14d8b5ac837aa0eea4f3224402
Author: Jon <jm...@apache.org>
AuthorDate: Thu Mar 28 00:36:10 2024 -0700

    WIP: fix wrap methods to take input isEmpty flag
---
 .../datasketches/filters/bloomfilter/BitArray.java  | 21 ++++++++++++++++++---
 .../filters/bloomfilter/DirectBitArray.java         |  6 +++---
 .../filters/bloomfilter/DirectBitArrayR.java        |  4 ++--
 3 files changed, 23 insertions(+), 8 deletions(-)

diff --git a/src/main/java/org/apache/datasketches/filters/bloomfilter/BitArray.java b/src/main/java/org/apache/datasketches/filters/bloomfilter/BitArray.java
index 76fe2f85..22715126 100644
--- a/src/main/java/org/apache/datasketches/filters/bloomfilter/BitArray.java
+++ b/src/main/java/org/apache/datasketches/filters/bloomfilter/BitArray.java
@@ -19,7 +19,10 @@
 
  package org.apache.datasketches.filters.bloomfilter;
 
- /**
+import org.apache.datasketches.memory.Memory;
+import org.apache.datasketches.memory.WritableMemory;
+
+/**
   * This class holds an array of bits suitable for use in a Bloom Filter
   *
   * <p>Rounds the number of bits up to the smallest multiple of 64 (one long)
@@ -31,6 +34,18 @@ abstract class BitArray {
 
   protected BitArray() {}
 
+  static BitArray heapify(final Memory mem, final boolean isEmpty) {
+    return HeapBitArray.heapify(mem);
+  }
+
+  static BitArray wrap(final Memory mem) {
+    return DirectBitArrayR.wrap(mem);
+  }
+
+  static BitArray writableWrap(final WritableMemory wmem) {
+    return DirectBitArray.writableWrap(wmem);
+  }
+
   boolean isEmpty() {
     return !isDirty() && getNumBitsSet() == 0;
   }
@@ -78,8 +93,8 @@ abstract class BitArray {
   long getSerializedSizeBytes() {
     // We only really need an int for array length but this will keep everything
     // aligned to 8 bytes.
-    // Always write array length and numBitsSet, even if empty
-    return Long.BYTES * (isEmpty() ? 2L : (2L + getArrayLength()));
+    // Always write array length, but write numBitsSet only if empty
+    return Long.BYTES * (isEmpty() ? 1L : (2L + getArrayLength()));
   }
 
   abstract protected boolean isDirty();
diff --git a/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArray.java b/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArray.java
index 4d0509bd..dbe9c048 100644
--- a/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArray.java
+++ b/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArray.java
@@ -31,16 +31,16 @@ final class DirectBitArray extends DirectBitArrayR {
     numBitsSet_ = storedNumBitsSet;
   }
 
-  static DirectBitArray wrap(final WritableMemory mem) {
+  static DirectBitArray writableWrap(final WritableMemory mem, final boolean isEmpty) {
     final int arrayLength = mem.getInt(0);
-    final long storedNumBitsSet = mem.getLong(NUM_BITS_OFFSET);
+    final long storedNumBitsSet = isEmpty ? 0L : mem.getLong(NUM_BITS_OFFSET);
 
     if (arrayLength * (long) Long.SIZE > MAX_BITS) {
       throw new SketchesArgumentException("Possible corruption: Serialized image indicates array beyond maximum filter capacity");
     }
 
     // if empty cannot wrap as writable
-    if (storedNumBitsSet == 0) {
+    if (isEmpty) {
       throw new SketchesArgumentException("Cannot wrap an empty filter for writing as there is no backing data array");
     }
 
diff --git a/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayR.java b/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayR.java
index d938380d..e7c073fb 100644
--- a/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayR.java
+++ b/src/main/java/org/apache/datasketches/filters/bloomfilter/DirectBitArrayR.java
@@ -50,9 +50,9 @@ public class DirectBitArrayR extends BitArray {
 
   // assumes we have a region with only the portion of Memory
   // the BitArray cares about
-  static DirectBitArrayR wrap(final Memory mem) {
+  static DirectBitArrayR wrap(final Memory mem, final boolean isEmpty) {
     final int arrayLength = mem.getInt(0);
-    final long storedNumBitsSet = mem.getLong(NUM_BITS_OFFSET);
+    final long storedNumBitsSet = isEmpty ? 0L : mem.getLong(NUM_BITS_OFFSET);
 
     if (arrayLength * (long) Long.SIZE > MAX_BITS) {
       throw new SketchesArgumentException("Possible corruption: Serialized image indicates array beyond maximum filter capacity");


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