From 3c5a843b3bb86951955e12fd4e9cdc9c5cf013cf Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Mon, 12 May 2025 10:40:31 +0200 Subject: [PATCH 01/44] SPARK-47547 BloomFilter fpp degradation: addressing the int32 truncation --- common/sketch/pom.xml | 7 + .../spark/util/sketch/BloomFilterImpl.java | 16 +- .../util/sketch/TestSparkBloomFilter.java | 142 ++++++++++++++++++ 3 files changed, 157 insertions(+), 8 deletions(-) create mode 100644 common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index f3f17a74fc99d..fdc5a9720687d 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -51,6 +51,13 @@ test + + org.junit-pioneer + junit-pioneer + 2.3.0 + test + + diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java index 3bd04a531fe75..57e2d8b466506 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java @@ -87,8 +87,8 @@ public boolean putBinary(byte[] item) { long bitSize = bits.bitSize(); boolean bitsChanged = false; - for (int i = 1; i <= numHashFunctions; i++) { - int combinedHash = h1 + (i * h2); + for (long i = 1; i <= numHashFunctions; i++) { + long combinedHash = h1 + (i * h2); // Flip all the bits if it's negative (guaranteed positive number) if (combinedHash < 0) { combinedHash = ~combinedHash; @@ -109,8 +109,8 @@ public boolean mightContainBinary(byte[] item) { int h2 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, h1); long bitSize = bits.bitSize(); - for (int i = 1; i <= numHashFunctions; i++) { - int combinedHash = h1 + (i * h2); + for (long i = 1; i <= numHashFunctions; i++) { + long combinedHash = h1 + (i * h2); // Flip all the bits if it's negative (guaranteed positive number) if (combinedHash < 0) { combinedHash = ~combinedHash; @@ -134,8 +134,8 @@ public boolean putLong(long item) { long bitSize = bits.bitSize(); boolean bitsChanged = false; - for (int i = 1; i <= numHashFunctions; i++) { - int combinedHash = h1 + (i * h2); + for (long i = 1; i <= numHashFunctions; i++) { + long combinedHash = h1 + (i * h2); // Flip all the bits if it's negative (guaranteed positive number) if (combinedHash < 0) { combinedHash = ~combinedHash; @@ -151,8 +151,8 @@ public boolean mightContainLong(long item) { int h2 = Murmur3_x86_32.hashLong(item, h1); long bitSize = bits.bitSize(); - for (int i = 1; i <= numHashFunctions; i++) { - int combinedHash = h1 + (i * h2); + for (long i = 1; i <= numHashFunctions; i++) { + long combinedHash = h1 + (i * h2); // Flip all the bits if it's negative (guaranteed positive number) if (combinedHash < 0) { combinedHash = ~combinedHash; diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java new file mode 100644 index 0000000000000..a0b9174549502 --- /dev/null +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java @@ -0,0 +1,142 @@ +/* + * 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.spark.util.sketch; + +import org.junit.jupiter.api.*; +import org.junit.jupiter.params.provider.ValueSource; +import org.junitpioneer.jupiter.cartesian.CartesianTest; +import org.junitpioneer.jupiter.cartesian.CartesianTest.Values; + +import java.time.Duration; +import java.time.Instant; +import java.util.Arrays; +import java.util.List; +import java.util.stream.Stream; + +public class TestSparkBloomFilter { + + // the implemented fpp limit is only approximating the hard boundary, + // so we'll need an error threshold for the assertion + final double FPP_ERROR_FACTOR = 1.0; + + final long ONE_GB = 1024L * 1024L * 1024L; + + private static Instant START; + private Instant start; + + @BeforeAll + public static void beforeAll() { + START = Instant.now(); + } + + @AfterAll + public static void afterAll() { + Duration duration = Duration.between(START, Instant.now()); + System.err.println(duration + " TOTAL"); + } + + @BeforeEach + public void beforeEach() { + start = Instant.now(); + } + + @AfterEach + public void afterEach(TestInfo testInfo) { + Duration duration = Duration.between(start, Instant.now()); + System.err.println(duration + " " + testInfo.getDisplayName()); + } + + @CartesianTest + public void testAccuracy( + @Values(longs = {1_000_000L, 1_000_000_000L, 5_000_000_000L}) long numItems, + @Values(doubles = {0.05, 0.03, 0.01, 0.001}) double expectedFpp + ) { + long optimalNumOfBytes = BloomFilter.optimalNumOfBits(numItems, expectedFpp) / Byte.SIZE; + System.err.printf("bitArray: %d MB", optimalNumOfBytes / 1024 / 1024); + Assumptions.assumeTrue( + optimalNumOfBytes < 4 * ONE_GB, + "this testcase would require allocating more than 4GB of heap mem (" + optimalNumOfBytes + ")" + ); + + // + + BloomFilter bloomFilter = BloomFilter.create(numItems, expectedFpp); + + for (long i = 0; i < numItems; i++) { + if (i % 10_000_000 == 0) { + System.err.printf( + "i: %d, bitCount: %d, b/i: %f, size: %d\n", + i, + bloomFilter.cardinality(), + (double) bloomFilter.cardinality() / i, + bloomFilter.bitSize() + ); + } + bloomFilter.putLong(2 * i); + } + + long mightContainEven = 0; + long mightContainOdd = 0; + + for (long i = 0; i < numItems; i++) { + if (i % 10_000_000 == 0) { + System.err.printf("i: %d\n", i); + } + + long even = 2 * i; + if (bloomFilter.mightContainLong(even)) { + mightContainEven++; + } + + long odd = 2 * i + 1; + if (bloomFilter.mightContainLong(odd)) { + mightContainOdd++; + } + } + + Assertions.assertEquals( + numItems, mightContainEven, + "mightContainLong must return true for all inserted numbers" + ); + + double actualFpp = (double) mightContainOdd / numItems; + double acceptableFpp = expectedFpp * (1 + FPP_ERROR_FACTOR); + + System.err.printf("expectedFpp: %f %%\n", 100 * expectedFpp); + System.err.printf("acceptableFpp: %f %%\n", 100 * acceptableFpp); + System.err.printf("actualFpp: %f %%\n", 100 * actualFpp); + + Assumptions.assumeTrue( + actualFpp <= acceptableFpp, + String.format( + "acceptableFpp(%f %%) < actualFpp (%f %%)", + 100 * acceptableFpp, + 100 * actualFpp + ) + ); + + Assertions.assertTrue( + actualFpp <= acceptableFpp, + String.format( + "acceptableFpp(%f %%) < actualFpp (%f %%)", + 100 * acceptableFpp, + 100 * actualFpp + ) + ); + } +} From 08cbfebb805d1adb0840b291904a62a5a647dcbf Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Tue, 13 May 2025 12:40:39 +0200 Subject: [PATCH 02/44] SPARK-47547 BloomFilter fpp degradation: fixing test data repetition errors in scala suite --- .../org/apache/spark/util/sketch/TestSparkBloomFilter.java | 1 + .../scala/org/apache/spark/util/sketch/BloomFilterSuite.scala | 4 +++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java index a0b9174549502..dc82479ceb6ed 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java @@ -28,6 +28,7 @@ import java.util.List; import java.util.stream.Stream; +@Disabled public class TestSparkBloomFilter { // the implemented fpp limit is only approximating the hard boundary, diff --git a/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala b/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala index 4d0ba66637b46..d1e709e83a5a9 100644 --- a/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala +++ b/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala @@ -46,7 +46,9 @@ class BloomFilterSuite extends AnyFunSuite { // scalastyle:ignore funsuite val fpp = 0.05 val numInsertion = numItems / 10 - val allItems = Array.fill(numItems)(itemGen(r)) + // using a Set to avoid duplicates, + // inserting twice as many random values as used, to compensate for lost dupes + val allItems = Set.fill(2 * numItems)(itemGen(r)).take(numItems) val filter = BloomFilter.create(numInsertion, fpp) From e3cb08e6490ddcdb585ba99e64c6cde6fd7746d1 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Tue, 13 May 2025 14:46:22 +0200 Subject: [PATCH 03/44] SPARK-47547 BloomFilter fpp degradation: scrambling the high 32bytes of the combined hash --- .../spark/util/sketch/BloomFilterImpl.java | 30 ++++++++++++++----- .../util/sketch/TestSparkBloomFilter.java | 2 +- 2 files changed, 23 insertions(+), 9 deletions(-) diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java index 57e2d8b466506..8dd7f7acc0268 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java @@ -87,8 +87,11 @@ public boolean putBinary(byte[] item) { long bitSize = bits.bitSize(); boolean bitsChanged = false; - for (long i = 1; i <= numHashFunctions; i++) { - long combinedHash = h1 + (i * h2); + + // Integer.MAX_VALUE takes care of scrambling the higher four bytes of combinedHash + long combinedHash = (long) h1 * Integer.MAX_VALUE; + for (long i = 0; i < numHashFunctions; i++) { + combinedHash += h2; // Flip all the bits if it's negative (guaranteed positive number) if (combinedHash < 0) { combinedHash = ~combinedHash; @@ -109,8 +112,11 @@ public boolean mightContainBinary(byte[] item) { int h2 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, h1); long bitSize = bits.bitSize(); - for (long i = 1; i <= numHashFunctions; i++) { - long combinedHash = h1 + (i * h2); + + // Integer.MAX_VALUE takes care of scrambling the higher four bytes of combinedHash + long combinedHash = (long) h1 * Integer.MAX_VALUE; + for (long i = 0; i < numHashFunctions; i++) { + combinedHash += h2; // Flip all the bits if it's negative (guaranteed positive number) if (combinedHash < 0) { combinedHash = ~combinedHash; @@ -134,8 +140,12 @@ public boolean putLong(long item) { long bitSize = bits.bitSize(); boolean bitsChanged = false; - for (long i = 1; i <= numHashFunctions; i++) { - long combinedHash = h1 + (i * h2); + + // Integer.MAX_VALUE takes care of scrambling the higher four bytes of combinedHash + long combinedHash = (long) h1 * Integer.MAX_VALUE; + for (long i = 0; i < numHashFunctions; i++) { + combinedHash += h2; + // Flip all the bits if it's negative (guaranteed positive number) if (combinedHash < 0) { combinedHash = ~combinedHash; @@ -151,8 +161,12 @@ public boolean mightContainLong(long item) { int h2 = Murmur3_x86_32.hashLong(item, h1); long bitSize = bits.bitSize(); - for (long i = 1; i <= numHashFunctions; i++) { - long combinedHash = h1 + (i * h2); + + // Integer.MAX_VALUE takes care of scrambling the higher four bytes of combinedHash + long combinedHash = (long) h1 * Integer.MAX_VALUE; + for (long i = 0; i < numHashFunctions; i++) { + combinedHash += h2; + // Flip all the bits if it's negative (guaranteed positive number) if (combinedHash < 0) { combinedHash = ~combinedHash; diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java index dc82479ceb6ed..17d38897556c4 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java @@ -33,7 +33,7 @@ public class TestSparkBloomFilter { // the implemented fpp limit is only approximating the hard boundary, // so we'll need an error threshold for the assertion - final double FPP_ERROR_FACTOR = 1.0; + final double FPP_ERROR_FACTOR = 0.05; final long ONE_GB = 1024L * 1024L * 1024L; From c4e3f585898e170768009db27627a208b7c1152e Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Tue, 13 May 2025 18:30:10 +0200 Subject: [PATCH 04/44] SPARK-47547 BloomFilter fpp degradation: random distribution fpp test --- .../apache/spark/util/sketch/BloomFilter.java | 5 +- .../spark/util/sketch/BloomFilterImpl.java | 28 +++- .../util/sketch/TestSparkBloomFilter.java | 148 ++++++++++++++++-- 3 files changed, 158 insertions(+), 23 deletions(-) diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java index 172b394689ca9..50a79c14937e3 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java @@ -256,6 +256,9 @@ public static BloomFilter create(long expectedNumItems, double fpp) { * pick an optimal {@code numHashFunctions} which can minimize {@code fpp} for the bloom filter. */ public static BloomFilter create(long expectedNumItems, long numBits) { + return create(expectedNumItems, numBits, BloomFilterImpl.DEFAULT_SEED); + } + public static BloomFilter create(long expectedNumItems, long numBits, int seed) { if (expectedNumItems <= 0) { throw new IllegalArgumentException("Expected insertions must be positive"); } @@ -264,6 +267,6 @@ public static BloomFilter create(long expectedNumItems, long numBits) { throw new IllegalArgumentException("Number of bits must be positive"); } - return new BloomFilterImpl(optimalNumOfHashFunctions(expectedNumItems, numBits), numBits); + return new BloomFilterImpl(optimalNumOfHashFunctions(expectedNumItems, numBits), numBits, seed); } } diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java index 8dd7f7acc0268..07b9de4afc721 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java @@ -21,17 +21,25 @@ class BloomFilterImpl extends BloomFilter implements Serializable { + public static final int DEFAULT_SEED = 0; + + private int seed; private int numHashFunctions; private BitArray bits; BloomFilterImpl(int numHashFunctions, long numBits) { - this(new BitArray(numBits), numHashFunctions); + this(numHashFunctions, numBits, DEFAULT_SEED); + } + + BloomFilterImpl(int numHashFunctions, long numBits, int seed) { + this(new BitArray(numBits), numHashFunctions, seed); } - private BloomFilterImpl(BitArray bits, int numHashFunctions) { + private BloomFilterImpl(BitArray bits, int numHashFunctions, int seed) { this.bits = bits; this.numHashFunctions = numHashFunctions; + this.seed = seed; } private BloomFilterImpl() {} @@ -82,7 +90,7 @@ public boolean putString(String item) { @Override public boolean putBinary(byte[] item) { - int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, 0); + int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, seed); int h2 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, h1); long bitSize = bits.bitSize(); @@ -108,7 +116,7 @@ public boolean mightContainString(String item) { @Override public boolean mightContainBinary(byte[] item) { - int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, 0); + int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, seed); int h2 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, h1); long bitSize = bits.bitSize(); @@ -135,7 +143,7 @@ public boolean putLong(long item) { // Note that `CountMinSketch` use a different strategy, it hash the input long element with // every i to produce n hash values. // TODO: the strategy of `CountMinSketch` looks more advanced, should we follow it here? - int h1 = Murmur3_x86_32.hashLong(item, 0); + int h1 = Murmur3_x86_32.hashLong(item, seed); int h2 = Murmur3_x86_32.hashLong(item, h1); long bitSize = bits.bitSize(); @@ -157,7 +165,7 @@ public boolean putLong(long item) { @Override public boolean mightContainLong(long item) { - int h1 = Murmur3_x86_32.hashLong(item, 0); + int h1 = Murmur3_x86_32.hashLong(item, seed); int h2 = Murmur3_x86_32.hashLong(item, h1); long bitSize = bits.bitSize(); @@ -240,6 +248,12 @@ private BloomFilterImpl checkCompatibilityForMerge(BloomFilter other) throw new IncompatibleMergeException("Cannot merge bloom filters with different bit size"); } + if (this.seed != that.seed) { + throw new IncompatibleMergeException( + "Cannot merge bloom filters with different seeds" + ); + } + if (this.numHashFunctions != that.numHashFunctions) { throw new IncompatibleMergeException( "Cannot merge bloom filters with different number of hash functions" @@ -253,6 +267,7 @@ public void writeTo(OutputStream out) throws IOException { DataOutputStream dos = new DataOutputStream(out); dos.writeInt(Version.V1.getVersionNumber()); + dos.writeInt(seed); dos.writeInt(numHashFunctions); bits.writeTo(dos); } @@ -265,6 +280,7 @@ private void readFrom0(InputStream in) throws IOException { throw new IOException("Unexpected Bloom filter version number (" + version + ")"); } + this.seed = dis.readInt(); this.numHashFunctions = dis.readInt(); this.bits = BitArray.readFrom(dis); } diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java index 17d38897556c4..46c005364fadf 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java @@ -26,6 +26,7 @@ import java.time.Instant; import java.util.Arrays; import java.util.List; +import java.util.Random; import java.util.stream.Stream; @Disabled @@ -33,7 +34,8 @@ public class TestSparkBloomFilter { // the implemented fpp limit is only approximating the hard boundary, // so we'll need an error threshold for the assertion - final double FPP_ERROR_FACTOR = 0.05; + final double FPP_EVEN_ODD_ERROR_FACTOR = 0.05; + final double FPP_RANDOM_ERROR_FACTOR = 0.04; final long ONE_GB = 1024L * 1024L * 1024L; @@ -63,29 +65,36 @@ public void afterEach(TestInfo testInfo) { } @CartesianTest - public void testAccuracy( + public void testAccuracyEvenOdd( @Values(longs = {1_000_000L, 1_000_000_000L, 5_000_000_000L}) long numItems, - @Values(doubles = {0.05, 0.03, 0.01, 0.001}) double expectedFpp + @Values(doubles = {0.05, 0.03, 0.01, 0.001}) double expectedFpp, + @Values(ints = {BloomFilterImpl.DEFAULT_SEED, 1, 127}) int deterministicSeed ) { - long optimalNumOfBytes = BloomFilter.optimalNumOfBits(numItems, expectedFpp) / Byte.SIZE; - System.err.printf("bitArray: %d MB", optimalNumOfBytes / 1024 / 1024); + long optimalNumOfBits = BloomFilter.optimalNumOfBits(numItems, expectedFpp) / Byte.SIZE; + System.err.printf( + "optimal bitArray: %d (%d MB)\n", + optimalNumOfBits, + optimalNumOfBits / Byte.SIZE / 1024 / 1024 + ); Assumptions.assumeTrue( - optimalNumOfBytes < 4 * ONE_GB, - "this testcase would require allocating more than 4GB of heap mem (" + optimalNumOfBytes + ")" + optimalNumOfBits / Byte.SIZE < 4 * ONE_GB, + "this testcase would require allocating more than 4GB of heap mem (" + optimalNumOfBits + " bits)" ); - // - - BloomFilter bloomFilter = BloomFilter.create(numItems, expectedFpp); + BloomFilter bloomFilter = BloomFilter.create(numItems, optimalNumOfBits, deterministicSeed); + System.err.printf( + "allocated bitArray: %d (%d MB)\n", + bloomFilter.bitSize(), + bloomFilter.bitSize() / Byte.SIZE / 1024 / 1024 + ); for (long i = 0; i < numItems; i++) { if (i % 10_000_000 == 0) { System.err.printf( - "i: %d, bitCount: %d, b/i: %f, size: %d\n", + "i: %d, bitCount: %d, saturation: %f\n", i, bloomFilter.cardinality(), - (double) bloomFilter.cardinality() / i, - bloomFilter.bitSize() + (double) bloomFilter.cardinality() / bloomFilter.bitSize() ); } bloomFilter.putLong(2 * i); @@ -95,8 +104,9 @@ public void testAccuracy( long mightContainOdd = 0; for (long i = 0; i < numItems; i++) { - if (i % 10_000_000 == 0) { - System.err.printf("i: %d\n", i); + if (i % (numItems / 100) == 0) { + System.err.print("."); + System.err.flush(); } long even = 2 * i; @@ -109,6 +119,7 @@ public void testAccuracy( mightContainOdd++; } } + System.err.println(); Assertions.assertEquals( numItems, mightContainEven, @@ -116,7 +127,7 @@ public void testAccuracy( ); double actualFpp = (double) mightContainOdd / numItems; - double acceptableFpp = expectedFpp * (1 + FPP_ERROR_FACTOR); + double acceptableFpp = expectedFpp * (1 + FPP_EVEN_ODD_ERROR_FACTOR); System.err.printf("expectedFpp: %f %%\n", 100 * expectedFpp); System.err.printf("acceptableFpp: %f %%\n", 100 * acceptableFpp); @@ -140,4 +151,109 @@ public void testAccuracy( ) ); } + + @CartesianTest + public void testAccuracyRandom( + @Values(longs = {1_000_000L, 1_000_000_000L}) long numItems, + @Values(doubles = {0.05, 0.03, 0.01, 0.001}) double expectedFpp, + @Values(ints = {BloomFilterImpl.DEFAULT_SEED, 1, 127}) int deterministicSeed + ) { + long optimalNumOfBits = BloomFilter.optimalNumOfBits(numItems, expectedFpp); + System.err.printf( + "optimal bitArray: %d (%d MB)\n", + optimalNumOfBits, + optimalNumOfBits / Byte.SIZE / 1024 / 1024 + ); + Assumptions.assumeTrue( + 2 * optimalNumOfBits / Byte.SIZE < 4 * ONE_GB, + "this testcase would require allocating more than 4GB of heap mem (2x " + optimalNumOfBits + " bits)" + ); + + BloomFilter bloomFilterPrimary = BloomFilter.create(numItems, optimalNumOfBits, deterministicSeed); + BloomFilter bloomFilterSecondary = BloomFilter.create(numItems, optimalNumOfBits, 0xCAFEBABE); + System.err.printf( + "allocated bitArray: %d (%d MB)\n", + bloomFilterPrimary.bitSize(), + bloomFilterPrimary.bitSize() / Byte.SIZE / 1024 / 1024 + ); + + + Random pseudoRandom = new Random(); + long iterationCount = 2 * numItems; + + pseudoRandom.setSeed(deterministicSeed); + for (long i = 0; i < iterationCount; i++) { + if (i % 10_000_000 == 0) { + System.err.printf( + "i: %d, bitCount: %d, saturation: %f\n", + i, + bloomFilterPrimary.cardinality(), + (double) bloomFilterPrimary.cardinality() / bloomFilterPrimary.bitSize() + ); + } + + long candidate = pseudoRandom.nextLong(); + if (i % 2 == 0) { + bloomFilterPrimary.putLong(candidate); + bloomFilterSecondary.putLong(candidate); + } + } + + long mightContainEven = 0; + long mightContainOdd = 0; + + pseudoRandom.setSeed(deterministicSeed); + for (long i = 0; i < iterationCount; i++) { + if (i % (iterationCount / 100) == 0) { + System.err.print("."); + System.err.flush(); + } + + long candidate = pseudoRandom.nextLong(); + if (bloomFilterPrimary.mightContainLong(candidate)) { + if (i % 2 == 0) { + mightContainEven++; + } else { + // only count those cases as false positives, + // where the secondary has confirmed, + // that we haven't inserted before + // (mitigating duplicates in input sequence) + if (!bloomFilterSecondary.mightContainLong(candidate)) { + mightContainOdd++; + } + } + } + } + System.err.println(); + + Assertions.assertEquals( + numItems, mightContainEven, + "mightContainLong must return true for all inserted numbers" + ); + + double actualFpp = (double) mightContainOdd / numItems; + double acceptableFpp = expectedFpp * (1 + FPP_RANDOM_ERROR_FACTOR); + + System.err.printf("expectedFpp: %f %%\n", 100 * expectedFpp); + System.err.printf("acceptableFpp: %f %%\n", 100 * acceptableFpp); + System.err.printf("actualFpp: %f %%\n", 100 * actualFpp); + + Assumptions.assumeTrue( + actualFpp <= acceptableFpp, + String.format( + "acceptableFpp(%f %%) < actualFpp (%f %%)", + 100 * acceptableFpp, + 100 * actualFpp + ) + ); + + Assertions.assertTrue( + actualFpp <= acceptableFpp, + String.format( + "acceptableFpp(%f %%) < actualFpp (%f %%)", + 100 * acceptableFpp, + 100 * actualFpp + ) + ); + } } From 1a0b66f77dca9a49dc0ff7063ce292105e7d96aa Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Mon, 19 May 2025 15:57:27 +0200 Subject: [PATCH 05/44] SPARK-47547 BloomFilter fpp degradation: javadoc for test methods, checkstyle errors, renaming test vars --- .../util/sketch/TestSparkBloomFilter.java | 67 ++++++++++++++----- 1 file changed, 51 insertions(+), 16 deletions(-) diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java index 46c005364fadf..d370ae6e38a8b 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java @@ -18,16 +18,12 @@ package org.apache.spark.util.sketch; import org.junit.jupiter.api.*; -import org.junit.jupiter.params.provider.ValueSource; import org.junitpioneer.jupiter.cartesian.CartesianTest; import org.junitpioneer.jupiter.cartesian.CartesianTest.Values; import java.time.Duration; import java.time.Instant; -import java.util.Arrays; -import java.util.List; import java.util.Random; -import java.util.stream.Stream; @Disabled public class TestSparkBloomFilter { @@ -38,6 +34,7 @@ public class TestSparkBloomFilter { final double FPP_RANDOM_ERROR_FACTOR = 0.04; final long ONE_GB = 1024L * 1024L * 1024L; + final long REQUIRED_HEAP_UPPER_BOUND_IN_BYTES = 4 * ONE_GB; private static Instant START; private Instant start; @@ -64,6 +61,18 @@ public void afterEach(TestInfo testInfo) { System.err.println(duration + " " + testInfo.getDisplayName()); } + /** + * This test, in N number of iterations, inserts N even numbers (2*i) int, + * and leaves out N odd numbers (2*i+1) from the tested BloomFilter instance. + * + * It checks the 100% accuracy of mightContain=true on all of the even items, + * and measures the mightContain=true (false positive) rate on the not-inserted odd numbers. + * + * @param numItems the number of items to be inserted + * @param expectedFpp the expected fpp rate of the tested BloomFilter instance + * @param deterministicSeed the deterministic seed to use to initialize + * the primary BloomFilter instance. + */ @CartesianTest public void testAccuracyEvenOdd( @Values(longs = {1_000_000L, 1_000_000_000L, 5_000_000_000L}) long numItems, @@ -77,8 +86,10 @@ public void testAccuracyEvenOdd( optimalNumOfBits / Byte.SIZE / 1024 / 1024 ); Assumptions.assumeTrue( - optimalNumOfBits / Byte.SIZE < 4 * ONE_GB, - "this testcase would require allocating more than 4GB of heap mem (" + optimalNumOfBits + " bits)" + optimalNumOfBits / Byte.SIZE < REQUIRED_HEAP_UPPER_BOUND_IN_BYTES, + "this testcase would require allocating more than 4GB of heap mem (" + + optimalNumOfBits + + " bits)" ); BloomFilter bloomFilter = BloomFilter.create(numItems, optimalNumOfBits, deterministicSeed); @@ -152,6 +163,25 @@ public void testAccuracyEvenOdd( ); } + /** + * This test inserts N pseudorandomly generated numbers in 2N number of iterations in two + * differently seeded (theoretically independent) BloomFilter instances. All the random + * numbers generated in an even-iteration will be inserted into both filters, all the + * random numbers generated in an odd-iteration will be left out from both. + * + * The test checks the 100% accuracy of 'mightContain=true' for all the items inserted + * in an even-loop. It counts the false positives as the number of odd-loop items for + * which the primary filter reports 'mightContain=true', but secondary reports + * 'mightContain=false'. Since we inserted the same elements into both instances, + * and the secondary reports non-insertion, the 'mightContain=true' from the primary + * can only be a false positive. + * + * @param numItems the number of items to be inserted + * @param expectedFpp the expected fpp rate of the tested BloomFilter instance + * @param deterministicSeed the deterministic seed to use to initialize + * the primary BloomFilter instance. (The secondary will be + * initialized with the constant seed of 0xCAFEBABE) + */ @CartesianTest public void testAccuracyRandom( @Values(longs = {1_000_000L, 1_000_000_000L}) long numItems, @@ -165,12 +195,17 @@ public void testAccuracyRandom( optimalNumOfBits / Byte.SIZE / 1024 / 1024 ); Assumptions.assumeTrue( - 2 * optimalNumOfBits / Byte.SIZE < 4 * ONE_GB, - "this testcase would require allocating more than 4GB of heap mem (2x " + optimalNumOfBits + " bits)" + 2 * optimalNumOfBits / Byte.SIZE < REQUIRED_HEAP_UPPER_BOUND_IN_BYTES, + "this testcase would require allocating more than 4GB of heap mem (2x " + + optimalNumOfBits + + " bits)" ); - BloomFilter bloomFilterPrimary = BloomFilter.create(numItems, optimalNumOfBits, deterministicSeed); - BloomFilter bloomFilterSecondary = BloomFilter.create(numItems, optimalNumOfBits, 0xCAFEBABE); + BloomFilter bloomFilterPrimary = + BloomFilter.create(numItems, optimalNumOfBits, deterministicSeed); + BloomFilter bloomFilterSecondary = + BloomFilter.create(numItems, optimalNumOfBits, 0xCAFEBABE); + System.err.printf( "allocated bitArray: %d (%d MB)\n", bloomFilterPrimary.bitSize(), @@ -199,8 +234,8 @@ public void testAccuracyRandom( } } - long mightContainEven = 0; - long mightContainOdd = 0; + long mightContainEvenIndexed = 0; + long mightContainOddIndexed = 0; pseudoRandom.setSeed(deterministicSeed); for (long i = 0; i < iterationCount; i++) { @@ -212,14 +247,14 @@ public void testAccuracyRandom( long candidate = pseudoRandom.nextLong(); if (bloomFilterPrimary.mightContainLong(candidate)) { if (i % 2 == 0) { - mightContainEven++; + mightContainEvenIndexed++; } else { // only count those cases as false positives, // where the secondary has confirmed, // that we haven't inserted before // (mitigating duplicates in input sequence) if (!bloomFilterSecondary.mightContainLong(candidate)) { - mightContainOdd++; + mightContainOddIndexed++; } } } @@ -227,11 +262,11 @@ public void testAccuracyRandom( System.err.println(); Assertions.assertEquals( - numItems, mightContainEven, + numItems, mightContainEvenIndexed, "mightContainLong must return true for all inserted numbers" ); - double actualFpp = (double) mightContainOdd / numItems; + double actualFpp = (double) mightContainOddIndexed / numItems; double acceptableFpp = expectedFpp * (1 + FPP_RANDOM_ERROR_FACTOR); System.err.printf("expectedFpp: %f %%\n", 100 * expectedFpp); From d912b6648abbc0f0705ed41e6e6a1bb6e588efd7 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Mon, 19 May 2025 16:30:22 +0200 Subject: [PATCH 06/44] SPARK-47547 BloomFilter fpp degradation: make seed serialization backward compatible with previously serialized streams --- .../org/apache/spark/util/sketch/BloomFilterImpl.java | 10 ++++++++-- .../spark/sql/BloomFilterAggregateQuerySuite.scala | 10 +++++++++- 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java index 07b9de4afc721..6ce33ff47945c 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java @@ -267,9 +267,9 @@ public void writeTo(OutputStream out) throws IOException { DataOutputStream dos = new DataOutputStream(out); dos.writeInt(Version.V1.getVersionNumber()); - dos.writeInt(seed); dos.writeInt(numHashFunctions); bits.writeTo(dos); + dos.writeInt(seed); } private void readFrom0(InputStream in) throws IOException { @@ -280,9 +280,15 @@ private void readFrom0(InputStream in) throws IOException { throw new IOException("Unexpected Bloom filter version number (" + version + ")"); } - this.seed = dis.readInt(); this.numHashFunctions = dis.readInt(); this.bits = BitArray.readFrom(dis); + + // compatibility with "seedless" serialization streams. + try { + this.seed = dis.readInt(); + } catch (EOFException e) { + this.seed = DEFAULT_SEED; + } } public static BloomFilterImpl readFrom(InputStream in) throws IOException { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala index af97856fd222e..7b20a7fcb9239 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala @@ -349,7 +349,7 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { ) } - test("Test that might_contain can take a constant value input") { + test("Test that might_contain can take a constant value input (seedless version)") { checkAnswer(spark.sql( """SELECT might_contain( |X'00000001000000050000000343A2EC6EA8C117E2D3CDB767296B144FC5BFBCED9737F267', @@ -357,6 +357,14 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { Row(false)) } + test("Test that might_contain can take a constant value input (seeded version)") { + checkAnswer(spark.sql( + """SELECT might_contain( + |X'00000001000000050000000343A2EC6EA8C117E2D3CDB767296B144FC5BFBCED9737F26700000000', + |cast(201 as long))""".stripMargin), + Row(false)) + } + test("Test that bloom_filter_agg produces a NULL with empty input") { checkAnswer(spark.sql("""SELECT bloom_filter_agg(cast(id as long)) from range(1, 1)"""), Row(null)) From f589e2c399c46472928a041341ec98fbae214cee Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Mon, 19 May 2025 17:23:25 +0200 Subject: [PATCH 07/44] SPARK-47547 BloomFilter fpp degradation: counting discarded odd items in random test + test formatting --- .../spark/util/sketch/TestSparkBloomFilter.java | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java index d370ae6e38a8b..ad5141e84f0f5 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java @@ -51,14 +51,16 @@ public static void afterAll() { } @BeforeEach - public void beforeEach() { + public void beforeEach(TestInfo testInfo) { start = Instant.now(); + System.err.println("---"); + System.err.println(testInfo.getTestMethod().get().getName() + " " + testInfo.getDisplayName()); } @AfterEach public void afterEach(TestInfo testInfo) { Duration duration = Duration.between(start, Instant.now()); - System.err.println(duration + " " + testInfo.getDisplayName()); + System.err.println(duration ); } /** @@ -236,6 +238,7 @@ public void testAccuracyRandom( long mightContainEvenIndexed = 0; long mightContainOddIndexed = 0; + long discardedOddIndexed = 0; pseudoRandom.setSeed(deterministicSeed); for (long i = 0; i < iterationCount; i++) { @@ -255,6 +258,8 @@ public void testAccuracyRandom( // (mitigating duplicates in input sequence) if (!bloomFilterSecondary.mightContainLong(candidate)) { mightContainOddIndexed++; + } else { + discardedOddIndexed++; } } } @@ -266,9 +271,11 @@ public void testAccuracyRandom( "mightContainLong must return true for all inserted numbers" ); - double actualFpp = (double) mightContainOddIndexed / numItems; + double actualFpp = (double) mightContainOddIndexed / (numItems - discardedOddIndexed); double acceptableFpp = expectedFpp * (1 + FPP_RANDOM_ERROR_FACTOR); + System.err.printf("numItems: %10d\n", numItems); + System.err.printf("discardedOddIndexed: %10d\n", discardedOddIndexed); System.err.printf("expectedFpp: %f %%\n", 100 * expectedFpp); System.err.printf("acceptableFpp: %f %%\n", 100 * acceptableFpp); System.err.printf("actualFpp: %f %%\n", 100 * actualFpp); From f597c762cf06517004901f7a4de8c8eb2593f74f Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Mon, 19 May 2025 19:03:42 +0200 Subject: [PATCH 08/44] SPARK-47547 BloomFilter fpp degradation: refactoring FPP counting logic in random test --- .../util/sketch/TestSparkBloomFilter.java | 38 +++++++++++-------- 1 file changed, 23 insertions(+), 15 deletions(-) diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java index ad5141e84f0f5..41fddc99897ab 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java @@ -238,7 +238,7 @@ public void testAccuracyRandom( long mightContainEvenIndexed = 0; long mightContainOddIndexed = 0; - long discardedOddIndexed = 0; + long confirmedAsNotInserted = 0; pseudoRandom.setSeed(deterministicSeed); for (long i = 0; i < iterationCount; i++) { @@ -248,19 +248,26 @@ public void testAccuracyRandom( } long candidate = pseudoRandom.nextLong(); - if (bloomFilterPrimary.mightContainLong(candidate)) { - if (i % 2 == 0) { - mightContainEvenIndexed++; - } else { - // only count those cases as false positives, - // where the secondary has confirmed, - // that we haven't inserted before - // (mitigating duplicates in input sequence) - if (!bloomFilterSecondary.mightContainLong(candidate)) { + + if (i % 2 == 0) { // EVEN + mightContainEvenIndexed++; + } else { // ODD + // for fpp estimation, only consider the odd indexes + // (to avoid querying the secondary with elements known to be inserted) + + // since here we avoided all the even indexes, + // most of these secondary queries will return false + if (!bloomFilterSecondary.mightContainLong(candidate)) { + // from the odd indexes, we consider only those items + // where the secondary confirms the non-insertion + + // anything on which the primary and the secondary + // disagrees here is a false positive + if (bloomFilterPrimary.mightContainLong(candidate)) { mightContainOddIndexed++; - } else { - discardedOddIndexed++; } + // count the total number of considered items for a baseline + confirmedAsNotInserted++; } } } @@ -271,11 +278,12 @@ public void testAccuracyRandom( "mightContainLong must return true for all inserted numbers" ); - double actualFpp = (double) mightContainOddIndexed / (numItems - discardedOddIndexed); + double actualFpp = (double) mightContainOddIndexed / confirmedAsNotInserted; double acceptableFpp = expectedFpp * (1 + FPP_RANDOM_ERROR_FACTOR); - System.err.printf("numItems: %10d\n", numItems); - System.err.printf("discardedOddIndexed: %10d\n", discardedOddIndexed); + System.err.printf("mightContainOddIndexed: %10d\n", mightContainOddIndexed); + System.err.printf("confirmedAsNotInserted: %10d\n", confirmedAsNotInserted); + System.err.printf("numItems: %10d\n", numItems); System.err.printf("expectedFpp: %f %%\n", 100 * expectedFpp); System.err.printf("acceptableFpp: %f %%\n", 100 * acceptableFpp); System.err.printf("actualFpp: %f %%\n", 100 * actualFpp); From 4ea633ddbc65aa4b7931c064e7c9929639bcde2b Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Mon, 19 May 2025 20:59:18 +0200 Subject: [PATCH 09/44] SPARK-47547 BloomFilter fpp degradation: checkstyle fix --- .../org/apache/spark/util/sketch/TestSparkBloomFilter.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java index 41fddc99897ab..b310a6362e3f5 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java @@ -54,7 +54,10 @@ public static void afterAll() { public void beforeEach(TestInfo testInfo) { start = Instant.now(); System.err.println("---"); - System.err.println(testInfo.getTestMethod().get().getName() + " " + testInfo.getDisplayName()); + System.err.println( + testInfo.getTestMethod().get().getName() + + " " + + testInfo.getDisplayName()); } @AfterEach From 669610679b5734997532f3178b5c7a6b0a0d64c6 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Wed, 21 May 2025 17:54:09 +0200 Subject: [PATCH 10/44] SPARK-47547 BloomFilter fpp degradation: fix test bug --- .../java/org/apache/spark/util/sketch/TestSparkBloomFilter.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java index b310a6362e3f5..c4b093af51ccd 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java @@ -84,7 +84,7 @@ public void testAccuracyEvenOdd( @Values(doubles = {0.05, 0.03, 0.01, 0.001}) double expectedFpp, @Values(ints = {BloomFilterImpl.DEFAULT_SEED, 1, 127}) int deterministicSeed ) { - long optimalNumOfBits = BloomFilter.optimalNumOfBits(numItems, expectedFpp) / Byte.SIZE; + long optimalNumOfBits = BloomFilter.optimalNumOfBits(numItems, expectedFpp); System.err.printf( "optimal bitArray: %d (%d MB)\n", optimalNumOfBits, From b75e187e333a281490ace4b76a27e8953791ba48 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Mon, 26 May 2025 17:51:48 +0200 Subject: [PATCH 11/44] SPARK-47547 BloomFilter fpp degradation: parallelization friendly test output capture --- .../util/sketch/TestSparkBloomFilter.java | 96 +++++++++++-------- 1 file changed, 56 insertions(+), 40 deletions(-) diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java index c4b093af51ccd..2a3042f53e7e1 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java @@ -21,6 +21,9 @@ import org.junitpioneer.jupiter.cartesian.CartesianTest; import org.junitpioneer.jupiter.cartesian.CartesianTest.Values; +import java.io.PrintStream; +import java.nio.file.Files; +import java.nio.file.Path; import java.time.Duration; import java.time.Instant; import java.util.Random; @@ -37,7 +40,10 @@ public class TestSparkBloomFilter { final long REQUIRED_HEAP_UPPER_BOUND_IN_BYTES = 4 * ONE_GB; private static Instant START; + + private String testName; private Instant start; + private PrintStream oldOut; @BeforeAll public static void beforeAll() { @@ -51,19 +57,32 @@ public static void afterAll() { } @BeforeEach - public void beforeEach(TestInfo testInfo) { + public void beforeEach( + TestInfo testInfo + ) throws Exception { start = Instant.now(); - System.err.println("---"); - System.err.println( - testInfo.getTestMethod().get().getName() - + " " - + testInfo.getDisplayName()); + + oldOut = System.out; + PrintStream newOut = + new PrintStream( + Files.newOutputStream(Path.of(testInfo.getDisplayName() + ".log")) + ); + System.setOut(newOut); + + testName = testInfo.getDisplayName(); + System.out.println("testName: " + testName); } @AfterEach public void afterEach(TestInfo testInfo) { Duration duration = Duration.between(start, Instant.now()); - System.err.println(duration ); + System.out.println("duration: " + duration ); + + PrintStream newOut = System.out; + System.setOut(oldOut); + newOut.close(); + + testName = ""; } /** @@ -78,14 +97,14 @@ public void afterEach(TestInfo testInfo) { * @param deterministicSeed the deterministic seed to use to initialize * the primary BloomFilter instance. */ - @CartesianTest + @CartesianTest(name = "testAccuracyEvenOdd_{index}.n{0}_fpp{1}_seed{2}") public void testAccuracyEvenOdd( @Values(longs = {1_000_000L, 1_000_000_000L, 5_000_000_000L}) long numItems, @Values(doubles = {0.05, 0.03, 0.01, 0.001}) double expectedFpp, @Values(ints = {BloomFilterImpl.DEFAULT_SEED, 1, 127}) int deterministicSeed ) { long optimalNumOfBits = BloomFilter.optimalNumOfBits(numItems, expectedFpp); - System.err.printf( + System.out.printf( "optimal bitArray: %d (%d MB)\n", optimalNumOfBits, optimalNumOfBits / Byte.SIZE / 1024 / 1024 @@ -98,7 +117,7 @@ public void testAccuracyEvenOdd( ); BloomFilter bloomFilter = BloomFilter.create(numItems, optimalNumOfBits, deterministicSeed); - System.err.printf( + System.out.printf( "allocated bitArray: %d (%d MB)\n", bloomFilter.bitSize(), bloomFilter.bitSize() / Byte.SIZE / 1024 / 1024 @@ -106,23 +125,23 @@ public void testAccuracyEvenOdd( for (long i = 0; i < numItems; i++) { if (i % 10_000_000 == 0) { - System.err.printf( - "i: %d, bitCount: %d, saturation: %f\n", - i, - bloomFilter.cardinality(), - (double) bloomFilter.cardinality() / bloomFilter.bitSize() - ); + System.err.printf("i: %d\n", i); } + bloomFilter.putLong(2 * i); } + System.out.printf("bitCount: %d\nsaturation: %f\n", + bloomFilter.cardinality(), + (double) bloomFilter.cardinality() / bloomFilter.bitSize() + ); + long mightContainEven = 0; long mightContainOdd = 0; for (long i = 0; i < numItems; i++) { if (i % (numItems / 100) == 0) { - System.err.print("."); - System.err.flush(); + System.err.printf("%s: %2d %%\n", testName, 100 * i / numItems); } long even = 2 * i; @@ -135,7 +154,6 @@ public void testAccuracyEvenOdd( mightContainOdd++; } } - System.err.println(); Assertions.assertEquals( numItems, mightContainEven, @@ -145,9 +163,9 @@ public void testAccuracyEvenOdd( double actualFpp = (double) mightContainOdd / numItems; double acceptableFpp = expectedFpp * (1 + FPP_EVEN_ODD_ERROR_FACTOR); - System.err.printf("expectedFpp: %f %%\n", 100 * expectedFpp); - System.err.printf("acceptableFpp: %f %%\n", 100 * acceptableFpp); - System.err.printf("actualFpp: %f %%\n", 100 * actualFpp); + System.out.printf("expectedFpp: %f %%\n", 100 * expectedFpp); + System.out.printf("acceptableFpp: %f %%\n", 100 * acceptableFpp); + System.out.printf("actualFpp: %f %%\n", 100 * actualFpp); Assumptions.assumeTrue( actualFpp <= acceptableFpp, @@ -187,14 +205,15 @@ public void testAccuracyEvenOdd( * the primary BloomFilter instance. (The secondary will be * initialized with the constant seed of 0xCAFEBABE) */ - @CartesianTest + @Disabled + @CartesianTest(name = "testAccuracyRandom_{index}.n{0}_fpp{1}_seed{2}") public void testAccuracyRandom( @Values(longs = {1_000_000L, 1_000_000_000L}) long numItems, @Values(doubles = {0.05, 0.03, 0.01, 0.001}) double expectedFpp, @Values(ints = {BloomFilterImpl.DEFAULT_SEED, 1, 127}) int deterministicSeed ) { long optimalNumOfBits = BloomFilter.optimalNumOfBits(numItems, expectedFpp); - System.err.printf( + System.out.printf( "optimal bitArray: %d (%d MB)\n", optimalNumOfBits, optimalNumOfBits / Byte.SIZE / 1024 / 1024 @@ -211,7 +230,7 @@ public void testAccuracyRandom( BloomFilter bloomFilterSecondary = BloomFilter.create(numItems, optimalNumOfBits, 0xCAFEBABE); - System.err.printf( + System.out.printf( "allocated bitArray: %d (%d MB)\n", bloomFilterPrimary.bitSize(), bloomFilterPrimary.bitSize() / Byte.SIZE / 1024 / 1024 @@ -224,12 +243,7 @@ public void testAccuracyRandom( pseudoRandom.setSeed(deterministicSeed); for (long i = 0; i < iterationCount; i++) { if (i % 10_000_000 == 0) { - System.err.printf( - "i: %d, bitCount: %d, saturation: %f\n", - i, - bloomFilterPrimary.cardinality(), - (double) bloomFilterPrimary.cardinality() / bloomFilterPrimary.bitSize() - ); + System.err.printf("i: %d\n", i); } long candidate = pseudoRandom.nextLong(); @@ -238,6 +252,10 @@ public void testAccuracyRandom( bloomFilterSecondary.putLong(candidate); } } + System.out.printf("bitCount: %d\nsaturation: %f\n", + bloomFilterPrimary.cardinality(), + (double) bloomFilterPrimary.cardinality() / bloomFilterPrimary.bitSize() + ); long mightContainEvenIndexed = 0; long mightContainOddIndexed = 0; @@ -246,8 +264,7 @@ public void testAccuracyRandom( pseudoRandom.setSeed(deterministicSeed); for (long i = 0; i < iterationCount; i++) { if (i % (iterationCount / 100) == 0) { - System.err.print("."); - System.err.flush(); + System.err.printf("%s: %2d %%\n", testName, 100 * i / iterationCount); } long candidate = pseudoRandom.nextLong(); @@ -274,7 +291,6 @@ public void testAccuracyRandom( } } } - System.err.println(); Assertions.assertEquals( numItems, mightContainEvenIndexed, @@ -284,12 +300,12 @@ public void testAccuracyRandom( double actualFpp = (double) mightContainOddIndexed / confirmedAsNotInserted; double acceptableFpp = expectedFpp * (1 + FPP_RANDOM_ERROR_FACTOR); - System.err.printf("mightContainOddIndexed: %10d\n", mightContainOddIndexed); - System.err.printf("confirmedAsNotInserted: %10d\n", confirmedAsNotInserted); - System.err.printf("numItems: %10d\n", numItems); - System.err.printf("expectedFpp: %f %%\n", 100 * expectedFpp); - System.err.printf("acceptableFpp: %f %%\n", 100 * acceptableFpp); - System.err.printf("actualFpp: %f %%\n", 100 * actualFpp); + System.out.printf("mightContainOddIndexed: %10d\n", mightContainOddIndexed); + System.out.printf("confirmedAsNotInserted: %10d\n", confirmedAsNotInserted); + System.out.printf("numItems: %10d\n", numItems); + System.out.printf("expectedFpp: %f %%\n", 100 * expectedFpp); + System.out.printf("acceptableFpp: %f %%\n", 100 * acceptableFpp); + System.out.printf("actualFpp: %f %%\n", 100 * actualFpp); Assumptions.assumeTrue( actualFpp <= acceptableFpp, From 2d8a9f188be6732ee0b3d6856d1bc7c1eba60842 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Mon, 26 May 2025 19:07:11 +0200 Subject: [PATCH 12/44] SPARK-47547 BloomFilter fpp degradation: parallelization friendly test output capture - 2nd take --- .../util/sketch/TestSparkBloomFilter.java | 53 +++++++++---------- 1 file changed, 24 insertions(+), 29 deletions(-) diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java index 2a3042f53e7e1..ae07c384285b5 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java @@ -28,6 +28,8 @@ import java.time.Instant; import java.util.Random; +import static java.nio.file.StandardOpenOption.CREATE; + @Disabled public class TestSparkBloomFilter { @@ -43,7 +45,7 @@ public class TestSparkBloomFilter { private String testName; private Instant start; - private PrintStream oldOut; + private PrintStream testOut; @BeforeAll public static void beforeAll() { @@ -62,26 +64,19 @@ public void beforeEach( ) throws Exception { start = Instant.now(); - oldOut = System.out; - PrintStream newOut = - new PrintStream( - Files.newOutputStream(Path.of(testInfo.getDisplayName() + ".log")) - ); - System.setOut(newOut); + Path testLogPath = Path.of(testInfo.getDisplayName() + ".log"); + Files.deleteIfExists(testLogPath); + testOut = new PrintStream(Files.newOutputStream(testLogPath)); testName = testInfo.getDisplayName(); - System.out.println("testName: " + testName); + testOut.println("testName: " + testName); } @AfterEach public void afterEach(TestInfo testInfo) { Duration duration = Duration.between(start, Instant.now()); - System.out.println("duration: " + duration ); - - PrintStream newOut = System.out; - System.setOut(oldOut); - newOut.close(); - + testOut.println("duration: " + duration ); + testOut.close(); testName = ""; } @@ -104,7 +99,7 @@ public void testAccuracyEvenOdd( @Values(ints = {BloomFilterImpl.DEFAULT_SEED, 1, 127}) int deterministicSeed ) { long optimalNumOfBits = BloomFilter.optimalNumOfBits(numItems, expectedFpp); - System.out.printf( + testOut.printf( "optimal bitArray: %d (%d MB)\n", optimalNumOfBits, optimalNumOfBits / Byte.SIZE / 1024 / 1024 @@ -117,7 +112,7 @@ public void testAccuracyEvenOdd( ); BloomFilter bloomFilter = BloomFilter.create(numItems, optimalNumOfBits, deterministicSeed); - System.out.printf( + testOut.printf( "allocated bitArray: %d (%d MB)\n", bloomFilter.bitSize(), bloomFilter.bitSize() / Byte.SIZE / 1024 / 1024 @@ -131,7 +126,7 @@ public void testAccuracyEvenOdd( bloomFilter.putLong(2 * i); } - System.out.printf("bitCount: %d\nsaturation: %f\n", + testOut.printf("bitCount: %d\nsaturation: %f\n", bloomFilter.cardinality(), (double) bloomFilter.cardinality() / bloomFilter.bitSize() ); @@ -163,9 +158,9 @@ public void testAccuracyEvenOdd( double actualFpp = (double) mightContainOdd / numItems; double acceptableFpp = expectedFpp * (1 + FPP_EVEN_ODD_ERROR_FACTOR); - System.out.printf("expectedFpp: %f %%\n", 100 * expectedFpp); - System.out.printf("acceptableFpp: %f %%\n", 100 * acceptableFpp); - System.out.printf("actualFpp: %f %%\n", 100 * actualFpp); + testOut.printf("expectedFpp: %f %%\n", 100 * expectedFpp); + testOut.printf("acceptableFpp: %f %%\n", 100 * acceptableFpp); + testOut.printf("actualFpp: %f %%\n", 100 * actualFpp); Assumptions.assumeTrue( actualFpp <= acceptableFpp, @@ -213,7 +208,7 @@ public void testAccuracyRandom( @Values(ints = {BloomFilterImpl.DEFAULT_SEED, 1, 127}) int deterministicSeed ) { long optimalNumOfBits = BloomFilter.optimalNumOfBits(numItems, expectedFpp); - System.out.printf( + testOut.printf( "optimal bitArray: %d (%d MB)\n", optimalNumOfBits, optimalNumOfBits / Byte.SIZE / 1024 / 1024 @@ -230,7 +225,7 @@ public void testAccuracyRandom( BloomFilter bloomFilterSecondary = BloomFilter.create(numItems, optimalNumOfBits, 0xCAFEBABE); - System.out.printf( + testOut.printf( "allocated bitArray: %d (%d MB)\n", bloomFilterPrimary.bitSize(), bloomFilterPrimary.bitSize() / Byte.SIZE / 1024 / 1024 @@ -252,7 +247,7 @@ public void testAccuracyRandom( bloomFilterSecondary.putLong(candidate); } } - System.out.printf("bitCount: %d\nsaturation: %f\n", + testOut.printf("bitCount: %d\nsaturation: %f\n", bloomFilterPrimary.cardinality(), (double) bloomFilterPrimary.cardinality() / bloomFilterPrimary.bitSize() ); @@ -300,12 +295,12 @@ public void testAccuracyRandom( double actualFpp = (double) mightContainOddIndexed / confirmedAsNotInserted; double acceptableFpp = expectedFpp * (1 + FPP_RANDOM_ERROR_FACTOR); - System.out.printf("mightContainOddIndexed: %10d\n", mightContainOddIndexed); - System.out.printf("confirmedAsNotInserted: %10d\n", confirmedAsNotInserted); - System.out.printf("numItems: %10d\n", numItems); - System.out.printf("expectedFpp: %f %%\n", 100 * expectedFpp); - System.out.printf("acceptableFpp: %f %%\n", 100 * acceptableFpp); - System.out.printf("actualFpp: %f %%\n", 100 * actualFpp); + testOut.printf("mightContainOddIndexed: %10d\n", mightContainOddIndexed); + testOut.printf("confirmedAsNotInserted: %10d\n", confirmedAsNotInserted); + testOut.printf("numItems: %10d\n", numItems); + testOut.printf("expectedFpp: %f %%\n", 100 * expectedFpp); + testOut.printf("acceptableFpp: %f %%\n", 100 * acceptableFpp); + testOut.printf("actualFpp: %f %%\n", 100 * actualFpp); Assumptions.assumeTrue( actualFpp <= acceptableFpp, From 4a307945bfe61a7391764dbe896ca63c0ce6c51b Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Mon, 26 May 2025 19:47:24 +0200 Subject: [PATCH 13/44] SPARK-47547 BloomFilter fpp degradation: parallelization friendly test output capture - 3rd take --- .../util/sketch/TestSparkBloomFilter.java | 31 +++++++++++++------ 1 file changed, 22 insertions(+), 9 deletions(-) diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java index ae07c384285b5..03096314e46c7 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java @@ -26,9 +26,10 @@ import java.nio.file.Path; import java.time.Duration; import java.time.Instant; +import java.util.Map; import java.util.Random; +import java.util.concurrent.ConcurrentHashMap; -import static java.nio.file.StandardOpenOption.CREATE; @Disabled public class TestSparkBloomFilter { @@ -43,9 +44,8 @@ public class TestSparkBloomFilter { private static Instant START; - private String testName; private Instant start; - private PrintStream testOut; + private final Map testOutMap = new ConcurrentHashMap<>(); @BeforeAll public static void beforeAll() { @@ -64,20 +64,25 @@ public void beforeEach( ) throws Exception { start = Instant.now(); - Path testLogPath = Path.of(testInfo.getDisplayName() + ".log"); + String testName = testInfo.getDisplayName(); + + Path testLogPath = Path.of(testName + ".log"); Files.deleteIfExists(testLogPath); - testOut = new PrintStream(Files.newOutputStream(testLogPath)); + PrintStream testOut = new PrintStream(Files.newOutputStream(testLogPath)); + testOutMap.put(testName, testOut); - testName = testInfo.getDisplayName(); testOut.println("testName: " + testName); } @AfterEach public void afterEach(TestInfo testInfo) { Duration duration = Duration.between(start, Instant.now()); + + String testName = testInfo.getDisplayName(); + PrintStream testOut = testOutMap.get(testName); + testOut.println("duration: " + duration ); testOut.close(); - testName = ""; } /** @@ -96,8 +101,12 @@ public void afterEach(TestInfo testInfo) { public void testAccuracyEvenOdd( @Values(longs = {1_000_000L, 1_000_000_000L, 5_000_000_000L}) long numItems, @Values(doubles = {0.05, 0.03, 0.01, 0.001}) double expectedFpp, - @Values(ints = {BloomFilterImpl.DEFAULT_SEED, 1, 127}) int deterministicSeed + @Values(ints = {BloomFilterImpl.DEFAULT_SEED, 1, 127}) int deterministicSeed, + TestInfo testInfo ) { + String testName = testInfo.getDisplayName(); + PrintStream testOut = testOutMap.get(testName); + long optimalNumOfBits = BloomFilter.optimalNumOfBits(numItems, expectedFpp); testOut.printf( "optimal bitArray: %d (%d MB)\n", @@ -205,8 +214,12 @@ public void testAccuracyEvenOdd( public void testAccuracyRandom( @Values(longs = {1_000_000L, 1_000_000_000L}) long numItems, @Values(doubles = {0.05, 0.03, 0.01, 0.001}) double expectedFpp, - @Values(ints = {BloomFilterImpl.DEFAULT_SEED, 1, 127}) int deterministicSeed + @Values(ints = {BloomFilterImpl.DEFAULT_SEED, 1, 127}) int deterministicSeed, + TestInfo testInfo ) { + String testName = testInfo.getDisplayName(); + PrintStream testOut = testOutMap.get(testName); + long optimalNumOfBits = BloomFilter.optimalNumOfBits(numItems, expectedFpp); testOut.printf( "optimal bitArray: %d (%d MB)\n", From d9d69801ec7d5fdfd441f12b2f22587a29e3ffd8 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Mon, 16 Jun 2025 14:06:28 +0200 Subject: [PATCH 14/44] SPARK-47547 BloomFilter fpp degradation: addressing concerns around disabled test --- ...Filter.java => SparkBloomFilterSuite.java} | 12 ++++++-- .../java/org/apache/spark/tags/SlowTest.java | 30 +++++++++++++++++++ 2 files changed, 39 insertions(+), 3 deletions(-) rename common/sketch/src/test/java/org/apache/spark/util/sketch/{TestSparkBloomFilter.java => SparkBloomFilterSuite.java} (97%) create mode 100644 common/tags/src/test/java/org/apache/spark/tags/SlowTest.java diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java similarity index 97% rename from common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java rename to common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java index 03096314e46c7..9a00e4c9e1c20 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/TestSparkBloomFilter.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java @@ -31,8 +31,9 @@ import java.util.concurrent.ConcurrentHashMap; -@Disabled -public class TestSparkBloomFilter { +@Tag("BloomFilter") +@Tag("org.apache.spark.tags.SlowTest") +public class SparkBloomFilterSuite { // the implemented fpp limit is only approximating the hard boundary, // so we'll need an error threshold for the assertion @@ -66,8 +67,13 @@ public void beforeEach( String testName = testInfo.getDisplayName(); - Path testLogPath = Path.of(testName + ".log"); + String testClassName = getClass().getName(); + String logDir = System.getProperty(testClassName+ ".logDir", "./target/tmp"); + Path logDirPath = Path.of(logDir); + Files.createDirectories(logDirPath); + Path testLogPath = Path.of(logDir,testName + ".log"); Files.deleteIfExists(testLogPath); + PrintStream testOut = new PrintStream(Files.newOutputStream(testLogPath)); testOutMap.put(testName, testOut); diff --git a/common/tags/src/test/java/org/apache/spark/tags/SlowTest.java b/common/tags/src/test/java/org/apache/spark/tags/SlowTest.java new file mode 100644 index 0000000000000..6d14d384101eb --- /dev/null +++ b/common/tags/src/test/java/org/apache/spark/tags/SlowTest.java @@ -0,0 +1,30 @@ +/* + * 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.spark.tags; + +import org.scalatest.TagAnnotation; + +import java.lang.annotation.ElementType; +import java.lang.annotation.Retention; +import java.lang.annotation.RetentionPolicy; +import java.lang.annotation.Target; + +@TagAnnotation +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.METHOD, ElementType.TYPE}) +public @interface SlowTest { } From 39a46c97f3a5d06802eb10c3154b1e427f4f0610 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Tue, 17 Jun 2025 10:36:11 +0200 Subject: [PATCH 15/44] SPARK-47547 BloomFilter fpp degradation: cut down test cases to decrease running time --- .../util/sketch/SparkBloomFilterSuite.java | 31 +++++++++++++------ 1 file changed, 22 insertions(+), 9 deletions(-) diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java index 9a00e4c9e1c20..b827e651ea4a9 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java @@ -105,10 +105,17 @@ public void afterEach(TestInfo testInfo) { */ @CartesianTest(name = "testAccuracyEvenOdd_{index}.n{0}_fpp{1}_seed{2}") public void testAccuracyEvenOdd( - @Values(longs = {1_000_000L, 1_000_000_000L, 5_000_000_000L}) long numItems, - @Values(doubles = {0.05, 0.03, 0.01, 0.001}) double expectedFpp, - @Values(ints = {BloomFilterImpl.DEFAULT_SEED, 1, 127}) int deterministicSeed, - TestInfo testInfo + // temporary workaround: + // to reduce running time to acceptable levels, we test only one case, + // with the default FPP and the default seed only. + @Values(longs = {1_000_000_000L}) long numItems, + @Values(doubles = {0.03}) double expectedFpp, + @Values(ints = {BloomFilterImpl.DEFAULT_SEED}) int deterministicSeed, + // preferable minimum parameter space for tests: + // @Values(longs = {1_000_000L, 1_000_000_000L}) long numItems, + // @Values(doubles = {0.05, 0.03, 0.01, 0.001}) double expectedFpp, + // @Values(ints = {BloomFilterImpl.DEFAULT_SEED, 1, 127}) int deterministicSeed, + TestInfo testInfo ) { String testName = testInfo.getDisplayName(); PrintStream testOut = testOutMap.get(testName); @@ -215,13 +222,19 @@ public void testAccuracyEvenOdd( * the primary BloomFilter instance. (The secondary will be * initialized with the constant seed of 0xCAFEBABE) */ - @Disabled @CartesianTest(name = "testAccuracyRandom_{index}.n{0}_fpp{1}_seed{2}") public void testAccuracyRandom( - @Values(longs = {1_000_000L, 1_000_000_000L}) long numItems, - @Values(doubles = {0.05, 0.03, 0.01, 0.001}) double expectedFpp, - @Values(ints = {BloomFilterImpl.DEFAULT_SEED, 1, 127}) int deterministicSeed, - TestInfo testInfo + // temporary workaround: + // to reduce running time to acceptable levels, we test only one case, + // with the default FPP and the default seed only. + @Values(longs = {1_000_000_000L}) long numItems, + @Values(doubles = {0.03}) double expectedFpp, + @Values(ints = {BloomFilterImpl.DEFAULT_SEED}) int deterministicSeed, + // preferable minimum parameter space for tests: + // @Values(longs = {1_000_000L, 1_000_000_000L}) long numItems, + // @Values(doubles = {0.05, 0.03, 0.01, 0.001}) double expectedFpp, + // @Values(ints = {BloomFilterImpl.DEFAULT_SEED, 1, 127}) int deterministicSeed, + TestInfo testInfo ) { String testName = testInfo.getDisplayName(); PrintStream testOut = testOutMap.get(testName); From 16be3a9f71f9a2b3e5258736576b81bb82f16572 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Tue, 17 Jun 2025 15:45:01 +0200 Subject: [PATCH 16/44] SPARK-47547 BloomFilter fpp degradation: revert creating a new SlowTest tag --- .../util/sketch/SparkBloomFilterSuite.java | 2 -- .../java/org/apache/spark/tags/SlowTest.java | 30 ------------------- 2 files changed, 32 deletions(-) delete mode 100644 common/tags/src/test/java/org/apache/spark/tags/SlowTest.java diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java index b827e651ea4a9..a7a330013ed80 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java @@ -31,8 +31,6 @@ import java.util.concurrent.ConcurrentHashMap; -@Tag("BloomFilter") -@Tag("org.apache.spark.tags.SlowTest") public class SparkBloomFilterSuite { // the implemented fpp limit is only approximating the hard boundary, diff --git a/common/tags/src/test/java/org/apache/spark/tags/SlowTest.java b/common/tags/src/test/java/org/apache/spark/tags/SlowTest.java deleted file mode 100644 index 6d14d384101eb..0000000000000 --- a/common/tags/src/test/java/org/apache/spark/tags/SlowTest.java +++ /dev/null @@ -1,30 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.tags; - -import org.scalatest.TagAnnotation; - -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.RetentionPolicy; -import java.lang.annotation.Target; - -@TagAnnotation -@Retention(RetentionPolicy.RUNTIME) -@Target({ElementType.METHOD, ElementType.TYPE}) -public @interface SlowTest { } From e91b5ca727c9ba219fdc272b49c238b0c9a51814 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Tue, 17 Jun 2025 15:52:45 +0200 Subject: [PATCH 17/44] SPARK-47547 BloomFilter fpp degradation: disable progress logging by default --- .../util/sketch/SparkBloomFilterSuite.java | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java index a7a330013ed80..fe6051b3b2d28 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java @@ -42,6 +42,7 @@ public class SparkBloomFilterSuite { final long REQUIRED_HEAP_UPPER_BOUND_IN_BYTES = 4 * ONE_GB; private static Instant START; + private static boolean verbose; private Instant start; private final Map testOutMap = new ConcurrentHashMap<>(); @@ -49,12 +50,17 @@ public class SparkBloomFilterSuite { @BeforeAll public static void beforeAll() { START = Instant.now(); + String testClassName = SparkBloomFilterSuite.class.getName(); + String verboseString = System.getProperty(testClassName+ ".verbose", "false"); + verbose = Boolean.parseBoolean(verboseString); } @AfterAll public static void afterAll() { Duration duration = Duration.between(START, Instant.now()); - System.err.println(duration + " TOTAL"); + if (verbose) { + System.err.println(duration + " TOTAL"); + } } @BeforeEach @@ -65,7 +71,7 @@ public void beforeEach( String testName = testInfo.getDisplayName(); - String testClassName = getClass().getName(); + String testClassName = SparkBloomFilterSuite.class.getName(); String logDir = System.getProperty(testClassName+ ".logDir", "./target/tmp"); Path logDirPath = Path.of(logDir); Files.createDirectories(logDirPath); @@ -139,7 +145,7 @@ public void testAccuracyEvenOdd( ); for (long i = 0; i < numItems; i++) { - if (i % 10_000_000 == 0) { + if (verbose && i % 10_000_000 == 0) { System.err.printf("i: %d\n", i); } @@ -155,7 +161,7 @@ public void testAccuracyEvenOdd( long mightContainOdd = 0; for (long i = 0; i < numItems; i++) { - if (i % (numItems / 100) == 0) { + if (verbose && i % (numItems / 100) == 0) { System.err.printf("%s: %2d %%\n", testName, 100 * i / numItems); } @@ -267,7 +273,7 @@ public void testAccuracyRandom( pseudoRandom.setSeed(deterministicSeed); for (long i = 0; i < iterationCount; i++) { - if (i % 10_000_000 == 0) { + if (verbose && i % 10_000_000 == 0) { System.err.printf("i: %d\n", i); } @@ -288,7 +294,7 @@ public void testAccuracyRandom( pseudoRandom.setSeed(deterministicSeed); for (long i = 0; i < iterationCount; i++) { - if (i % (iterationCount / 100) == 0) { + if (verbose && i % (iterationCount / 100) == 0) { System.err.printf("%s: %2d %%\n", testName, 100 * i / iterationCount); } From 897c1d4a510eeaa3e82d4d3078e22361cff979a8 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Wed, 18 Jun 2025 11:25:38 +0200 Subject: [PATCH 18/44] SPARK-47547 BloomFilter fpp degradation: adjust tolerance and fail on fpp problems by default --- .../util/sketch/SparkBloomFilterSuite.java | 57 ++++++++++--------- 1 file changed, 31 insertions(+), 26 deletions(-) diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java index fe6051b3b2d28..f4309ed35978c 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java @@ -35,13 +35,14 @@ public class SparkBloomFilterSuite { // the implemented fpp limit is only approximating the hard boundary, // so we'll need an error threshold for the assertion - final double FPP_EVEN_ODD_ERROR_FACTOR = 0.05; - final double FPP_RANDOM_ERROR_FACTOR = 0.04; + final double FPP_EVEN_ODD_ERROR_FACTOR = 0.10; + final double FPP_RANDOM_ERROR_FACTOR = 0.10; final long ONE_GB = 1024L * 1024L * 1024L; final long REQUIRED_HEAP_UPPER_BOUND_IN_BYTES = 4 * ONE_GB; private static Instant START; + private static boolean strict; private static boolean verbose; private Instant start; @@ -51,8 +52,8 @@ public class SparkBloomFilterSuite { public static void beforeAll() { START = Instant.now(); String testClassName = SparkBloomFilterSuite.class.getName(); - String verboseString = System.getProperty(testClassName+ ".verbose", "false"); - verbose = Boolean.parseBoolean(verboseString); + strict = Boolean.parseBoolean(System.getProperty(testClassName+ ".strict", "true")); + verbose = Boolean.parseBoolean(System.getProperty(testClassName+ ".verbose", "false")); } @AfterAll @@ -188,23 +189,25 @@ public void testAccuracyEvenOdd( testOut.printf("acceptableFpp: %f %%\n", 100 * acceptableFpp); testOut.printf("actualFpp: %f %%\n", 100 * actualFpp); - Assumptions.assumeTrue( + if (!strict) { + Assumptions.assumeTrue( actualFpp <= acceptableFpp, String.format( - "acceptableFpp(%f %%) < actualFpp (%f %%)", - 100 * acceptableFpp, - 100 * actualFpp + "acceptableFpp(%f %%) < actualFpp (%f %%)", + 100 * acceptableFpp, + 100 * actualFpp ) - ); - - Assertions.assertTrue( + ); + } else { + Assertions.assertTrue( actualFpp <= acceptableFpp, String.format( - "acceptableFpp(%f %%) < actualFpp (%f %%)", - 100 * acceptableFpp, - 100 * actualFpp + "acceptableFpp(%f %%) < actualFpp (%f %%)", + 100 * acceptableFpp, + 100 * actualFpp ) - ); + ); + } } /** @@ -338,22 +341,24 @@ public void testAccuracyRandom( testOut.printf("acceptableFpp: %f %%\n", 100 * acceptableFpp); testOut.printf("actualFpp: %f %%\n", 100 * actualFpp); - Assumptions.assumeTrue( + if (!strict) { + Assumptions.assumeTrue( actualFpp <= acceptableFpp, String.format( - "acceptableFpp(%f %%) < actualFpp (%f %%)", - 100 * acceptableFpp, - 100 * actualFpp + "acceptableFpp(%f %%) < actualFpp (%f %%)", + 100 * acceptableFpp, + 100 * actualFpp ) - ); - - Assertions.assertTrue( + ); + } else { + Assertions.assertTrue( actualFpp <= acceptableFpp, String.format( - "acceptableFpp(%f %%) < actualFpp (%f %%)", - 100 * acceptableFpp, - 100 * actualFpp + "acceptableFpp(%f %%) < actualFpp (%f %%)", + 100 * acceptableFpp, + 100 * actualFpp ) - ); + ); + } } } From 013bfe4f756801f15c7ccdbbce358795f99957e6 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Sun, 6 Jul 2025 14:11:07 +0200 Subject: [PATCH 19/44] SPARK-47547 BloomFilter fpp degradation: make V1/V2 distinction in BloomFilter serialization --- .../apache/spark/util/sketch/BloomFilter.java | 58 +++- .../spark/util/sketch/BloomFilterImpl.java | 72 ++--- .../spark/util/sketch/BloomFilterImplV2.java | 301 ++++++++++++++++++ .../util/sketch/SparkBloomFilterSuite.java | 28 +- .../spark/util/sketch/BloomFilterSuite.scala | 6 + .../sql/BloomFilterAggregateQuerySuite.scala | 2 +- 6 files changed, 401 insertions(+), 66 deletions(-) create mode 100644 common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java index 50a79c14937e3..ca09155c1ced3 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java @@ -17,9 +17,12 @@ package org.apache.spark.util.sketch; +import java.io.BufferedInputStream; +import java.io.ByteArrayInputStream; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; +import java.nio.ByteBuffer; /** * A Bloom filter is a space-efficient probabilistic data structure that offers an approximate @@ -51,7 +54,8 @@ public enum Version { *
  • The words/longs (numWords * 64 bit)
  • * */ - V1(1); + V1(1), + V2(2); private final int versionNumber; @@ -175,14 +179,34 @@ public long cardinality() { * the stream. */ public static BloomFilter readFrom(InputStream in) throws IOException { - return BloomFilterImpl.readFrom(in); + // peek into the inputstream so we can determine the version + BufferedInputStream bin = new BufferedInputStream(in); + bin.mark(4); + int version = ByteBuffer.wrap(bin.readNBytes(4)).getInt(); + bin.reset(); + + BloomFilter result; + switch (version) { + case 1: + result = BloomFilterImpl.readFrom(bin); + break; + case 2: + result = BloomFilterImplV2.readFrom(bin); + break; + default: + throw new IllegalArgumentException("Unknown BloomFilter version: " + version); + } + + return result; } /** * Reads in a {@link BloomFilter} from a byte array. */ public static BloomFilter readFrom(byte[] bytes) throws IOException { - return BloomFilterImpl.readFrom(bytes); + try (ByteArrayInputStream bis = new ByteArrayInputStream(bytes)) { + return readFrom(bis); + } } /** @@ -247,7 +271,6 @@ public static BloomFilter create(long expectedNumItems, double fpp) { "False positive probability must be within range (0.0, 1.0)" ); } - return create(expectedNumItems, optimalNumOfBits(expectedNumItems, fpp)); } @@ -256,9 +279,19 @@ public static BloomFilter create(long expectedNumItems, double fpp) { * pick an optimal {@code numHashFunctions} which can minimize {@code fpp} for the bloom filter. */ public static BloomFilter create(long expectedNumItems, long numBits) { - return create(expectedNumItems, numBits, BloomFilterImpl.DEFAULT_SEED); + return create(Version.V2, expectedNumItems, numBits, BloomFilterImplV2.DEFAULT_SEED); } + public static BloomFilter create(long expectedNumItems, long numBits, int seed) { + return create(Version.V2, expectedNumItems, numBits, seed); + } + + public static BloomFilter create( + Version version, + long expectedNumItems, + long numBits, + int seed + ) { if (expectedNumItems <= 0) { throw new IllegalArgumentException("Expected insertions must be positive"); } @@ -267,6 +300,19 @@ public static BloomFilter create(long expectedNumItems, long numBits, int seed) throw new IllegalArgumentException("Number of bits must be positive"); } - return new BloomFilterImpl(optimalNumOfHashFunctions(expectedNumItems, numBits), numBits, seed); + int numHashFunctions = optimalNumOfHashFunctions(expectedNumItems, numBits); + + BloomFilter result; + switch (version) { + case V1: + result = new BloomFilterImpl(numHashFunctions, numBits); + break; + case V2: + result = new BloomFilterImplV2(numHashFunctions, numBits, seed); + break; + default: + throw new IllegalArgumentException("Unknown BloomFilter version: " + version); + } + return result; } } diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java index 6ce33ff47945c..ba569994cdf87 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java @@ -21,25 +21,17 @@ class BloomFilterImpl extends BloomFilter implements Serializable { - public static final int DEFAULT_SEED = 0; - - private int seed; private int numHashFunctions; private BitArray bits; BloomFilterImpl(int numHashFunctions, long numBits) { - this(numHashFunctions, numBits, DEFAULT_SEED); - } - - BloomFilterImpl(int numHashFunctions, long numBits, int seed) { - this(new BitArray(numBits), numHashFunctions, seed); + this(new BitArray(numBits), numHashFunctions); } - private BloomFilterImpl(BitArray bits, int numHashFunctions, int seed) { + private BloomFilterImpl(BitArray bits, int numHashFunctions) { this.bits = bits; this.numHashFunctions = numHashFunctions; - this.seed = seed; } private BloomFilterImpl() {} @@ -90,16 +82,13 @@ public boolean putString(String item) { @Override public boolean putBinary(byte[] item) { - int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, seed); + int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, 0); int h2 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, h1); long bitSize = bits.bitSize(); boolean bitsChanged = false; - - // Integer.MAX_VALUE takes care of scrambling the higher four bytes of combinedHash - long combinedHash = (long) h1 * Integer.MAX_VALUE; - for (long i = 0; i < numHashFunctions; i++) { - combinedHash += h2; + for (int i = 1; i <= numHashFunctions; i++) { + int combinedHash = h1 + (i * h2); // Flip all the bits if it's negative (guaranteed positive number) if (combinedHash < 0) { combinedHash = ~combinedHash; @@ -116,15 +105,12 @@ public boolean mightContainString(String item) { @Override public boolean mightContainBinary(byte[] item) { - int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, seed); + int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, 0); int h2 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, h1); long bitSize = bits.bitSize(); - - // Integer.MAX_VALUE takes care of scrambling the higher four bytes of combinedHash - long combinedHash = (long) h1 * Integer.MAX_VALUE; - for (long i = 0; i < numHashFunctions; i++) { - combinedHash += h2; + for (int i = 1; i <= numHashFunctions; i++) { + int combinedHash = h1 + (i * h2); // Flip all the bits if it's negative (guaranteed positive number) if (combinedHash < 0) { combinedHash = ~combinedHash; @@ -143,17 +129,13 @@ public boolean putLong(long item) { // Note that `CountMinSketch` use a different strategy, it hash the input long element with // every i to produce n hash values. // TODO: the strategy of `CountMinSketch` looks more advanced, should we follow it here? - int h1 = Murmur3_x86_32.hashLong(item, seed); + int h1 = Murmur3_x86_32.hashLong(item, 0); int h2 = Murmur3_x86_32.hashLong(item, h1); long bitSize = bits.bitSize(); boolean bitsChanged = false; - - // Integer.MAX_VALUE takes care of scrambling the higher four bytes of combinedHash - long combinedHash = (long) h1 * Integer.MAX_VALUE; - for (long i = 0; i < numHashFunctions; i++) { - combinedHash += h2; - + for (int i = 1; i <= numHashFunctions; i++) { + int combinedHash = h1 + (i * h2); // Flip all the bits if it's negative (guaranteed positive number) if (combinedHash < 0) { combinedHash = ~combinedHash; @@ -165,16 +147,12 @@ public boolean putLong(long item) { @Override public boolean mightContainLong(long item) { - int h1 = Murmur3_x86_32.hashLong(item, seed); + int h1 = Murmur3_x86_32.hashLong(item, 0); int h2 = Murmur3_x86_32.hashLong(item, h1); long bitSize = bits.bitSize(); - - // Integer.MAX_VALUE takes care of scrambling the higher four bytes of combinedHash - long combinedHash = (long) h1 * Integer.MAX_VALUE; - for (long i = 0; i < numHashFunctions; i++) { - combinedHash += h2; - + for (int i = 1; i <= numHashFunctions; i++) { + int combinedHash = h1 + (i * h2); // Flip all the bits if it's negative (guaranteed positive number) if (combinedHash < 0) { combinedHash = ~combinedHash; @@ -248,12 +226,6 @@ private BloomFilterImpl checkCompatibilityForMerge(BloomFilter other) throw new IncompatibleMergeException("Cannot merge bloom filters with different bit size"); } - if (this.seed != that.seed) { - throw new IncompatibleMergeException( - "Cannot merge bloom filters with different seeds" - ); - } - if (this.numHashFunctions != that.numHashFunctions) { throw new IncompatibleMergeException( "Cannot merge bloom filters with different number of hash functions" @@ -269,7 +241,6 @@ public void writeTo(OutputStream out) throws IOException { dos.writeInt(Version.V1.getVersionNumber()); dos.writeInt(numHashFunctions); bits.writeTo(dos); - dos.writeInt(seed); } private void readFrom0(InputStream in) throws IOException { @@ -282,13 +253,6 @@ private void readFrom0(InputStream in) throws IOException { this.numHashFunctions = dis.readInt(); this.bits = BitArray.readFrom(dis); - - // compatibility with "seedless" serialization streams. - try { - this.seed = dis.readInt(); - } catch (EOFException e) { - this.seed = DEFAULT_SEED; - } } public static BloomFilterImpl readFrom(InputStream in) throws IOException { @@ -297,10 +261,10 @@ public static BloomFilterImpl readFrom(InputStream in) throws IOException { return filter; } - public static BloomFilterImpl readFrom(byte[] bytes) throws IOException { - try (ByteArrayInputStream bis = new ByteArrayInputStream(bytes)) { - return readFrom(bis); - } + // no longer necessary, but can't remove without triggering MIMA violations + @Deprecated + public static BloomFilter readFrom(byte[] bytes) throws IOException { + return BloomFilter.readFrom(bytes); } private void writeObject(ObjectOutputStream out) throws IOException { diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java new file mode 100644 index 0000000000000..f5f1b2b0838ae --- /dev/null +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java @@ -0,0 +1,301 @@ +/* + * 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.spark.util.sketch; + +import java.io.*; + +class BloomFilterImplV2 extends BloomFilter implements Serializable { + + public static final int DEFAULT_SEED = 0; + + private int seed; + private int numHashFunctions; + + private BitArray bits; + + BloomFilterImplV2(int numHashFunctions, long numBits) { + this(numHashFunctions, numBits, DEFAULT_SEED); + } + + BloomFilterImplV2(int numHashFunctions, long numBits, int seed) { + this(new BitArray(numBits), numHashFunctions, seed); + } + + private BloomFilterImplV2(BitArray bits, int numHashFunctions, int seed) { + this.bits = bits; + this.numHashFunctions = numHashFunctions; + this.seed = seed; + } + + private BloomFilterImplV2() {} + + @Override + public boolean equals(Object other) { + if (other == this) { + return true; + } + + if (!(other instanceof BloomFilterImplV2 that)) { + return false; + } + + return this.numHashFunctions == that.numHashFunctions && this.bits.equals(that.bits); + } + + @Override + public int hashCode() { + return bits.hashCode() * 31 + numHashFunctions; + } + + @Override + public double expectedFpp() { + return Math.pow((double) bits.cardinality() / bits.bitSize(), numHashFunctions); + } + + @Override + public long bitSize() { + return bits.bitSize(); + } + + @Override + public boolean put(Object item) { + if (item instanceof String str) { + return putString(str); + } else if (item instanceof byte[] bytes) { + return putBinary(bytes); + } else { + return putLong(Utils.integralToLong(item)); + } + } + + @Override + public boolean putString(String item) { + return putBinary(Utils.getBytesFromUTF8String(item)); + } + + @Override + public boolean putBinary(byte[] item) { + int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, seed); + int h2 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, h1); + + long bitSize = bits.bitSize(); + boolean bitsChanged = false; + + // Integer.MAX_VALUE takes care of scrambling the higher four bytes of combinedHash + long combinedHash = (long) h1 * Integer.MAX_VALUE; + for (long i = 0; i < numHashFunctions; i++) { + combinedHash += h2; + // Flip all the bits if it's negative (guaranteed positive number) + if (combinedHash < 0) { + combinedHash = ~combinedHash; + } + bitsChanged |= bits.set(combinedHash % bitSize); + } + return bitsChanged; + } + + @Override + public boolean mightContainString(String item) { + return mightContainBinary(Utils.getBytesFromUTF8String(item)); + } + + @Override + public boolean mightContainBinary(byte[] item) { + int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, seed); + int h2 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, h1); + + long bitSize = bits.bitSize(); + + // Integer.MAX_VALUE takes care of scrambling the higher four bytes of combinedHash + long combinedHash = (long) h1 * Integer.MAX_VALUE; + for (long i = 0; i < numHashFunctions; i++) { + combinedHash += h2; + // Flip all the bits if it's negative (guaranteed positive number) + if (combinedHash < 0) { + combinedHash = ~combinedHash; + } + if (!bits.get(combinedHash % bitSize)) { + return false; + } + } + return true; + } + + @Override + public boolean putLong(long item) { + // Here we first hash the input long element into 2 int hash values, h1 and h2, then produce n + // hash values by `h1 + i * h2` with 1 <= i <= numHashFunctions. + // Note that `CountMinSketch` use a different strategy, it hash the input long element with + // every i to produce n hash values. + // TODO: the strategy of `CountMinSketch` looks more advanced, should we follow it here? + int h1 = Murmur3_x86_32.hashLong(item, seed); + int h2 = Murmur3_x86_32.hashLong(item, h1); + + long bitSize = bits.bitSize(); + boolean bitsChanged = false; + + // Integer.MAX_VALUE takes care of scrambling the higher four bytes of combinedHash + long combinedHash = (long) h1 * Integer.MAX_VALUE; + for (long i = 0; i < numHashFunctions; i++) { + combinedHash += h2; + + // Flip all the bits if it's negative (guaranteed positive number) + if (combinedHash < 0) { + combinedHash = ~combinedHash; + } + bitsChanged |= bits.set(combinedHash % bitSize); + } + return bitsChanged; + } + + @Override + public boolean mightContainLong(long item) { + int h1 = Murmur3_x86_32.hashLong(item, seed); + int h2 = Murmur3_x86_32.hashLong(item, h1); + + long bitSize = bits.bitSize(); + + // Integer.MAX_VALUE takes care of scrambling the higher four bytes of combinedHash + long combinedHash = (long) h1 * Integer.MAX_VALUE; + for (long i = 0; i < numHashFunctions; i++) { + combinedHash += h2; + + // Flip all the bits if it's negative (guaranteed positive number) + if (combinedHash < 0) { + combinedHash = ~combinedHash; + } + if (!bits.get(combinedHash % bitSize)) { + return false; + } + } + return true; + } + + @Override + public boolean mightContain(Object item) { + if (item instanceof String str) { + return mightContainString(str); + } else if (item instanceof byte[] bytes) { + return mightContainBinary(bytes); + } else { + return mightContainLong(Utils.integralToLong(item)); + } + } + + @Override + public boolean isCompatible(BloomFilter other) { + if (other == null) { + return false; + } + + if (!(other instanceof BloomFilterImplV2 that)) { + return false; + } + + return this.bitSize() == that.bitSize() && this.numHashFunctions == that.numHashFunctions; + } + + @Override + public BloomFilter mergeInPlace(BloomFilter other) throws IncompatibleMergeException { + BloomFilterImplV2 otherImplInstance = checkCompatibilityForMerge(other); + + this.bits.putAll(otherImplInstance.bits); + return this; + } + + @Override + public BloomFilter intersectInPlace(BloomFilter other) throws IncompatibleMergeException { + BloomFilterImplV2 otherImplInstance = checkCompatibilityForMerge(other); + + this.bits.and(otherImplInstance.bits); + return this; + } + + @Override + public long cardinality() { + return this.bits.cardinality(); + } + + private BloomFilterImplV2 checkCompatibilityForMerge(BloomFilter other) + throws IncompatibleMergeException { + // Duplicates the logic of `isCompatible` here to provide better error message. + if (other == null) { + throw new IncompatibleMergeException("Cannot merge null bloom filter"); + } + + if (!(other instanceof BloomFilterImplV2 that)) { + throw new IncompatibleMergeException( + "Cannot merge bloom filter of class " + other.getClass().getName() + ); + } + + if (this.bitSize() != that.bitSize()) { + throw new IncompatibleMergeException("Cannot merge bloom filters with different bit size"); + } + + if (this.seed != that.seed) { + throw new IncompatibleMergeException( + "Cannot merge bloom filters with different seeds" + ); + } + + if (this.numHashFunctions != that.numHashFunctions) { + throw new IncompatibleMergeException( + "Cannot merge bloom filters with different number of hash functions" + ); + } + return that; + } + + @Override + public void writeTo(OutputStream out) throws IOException { + DataOutputStream dos = new DataOutputStream(out); + + dos.writeInt(Version.V2.getVersionNumber()); + dos.writeInt(numHashFunctions); + dos.writeInt(seed); + bits.writeTo(dos); + } + + private void readFrom0(InputStream in) throws IOException { + DataInputStream dis = new DataInputStream(in); + + int version = dis.readInt(); + if (version != Version.V2.getVersionNumber()) { + throw new IOException("Unexpected Bloom filter version number (" + version + ")"); + } + + this.numHashFunctions = dis.readInt(); + this.seed = dis.readInt(); + this.bits = BitArray.readFrom(dis); + } + + public static BloomFilterImplV2 readFrom(InputStream in) throws IOException { + BloomFilterImplV2 filter = new BloomFilterImplV2(); + filter.readFrom0(in); + return filter; + } + + private void writeObject(ObjectOutputStream out) throws IOException { + writeTo(out); + } + + private void readObject(ObjectInputStream in) throws IOException { + readFrom0(in); + } +} diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java index f4309ed35978c..2dc2aba992276 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java @@ -115,7 +115,7 @@ public void testAccuracyEvenOdd( // with the default FPP and the default seed only. @Values(longs = {1_000_000_000L}) long numItems, @Values(doubles = {0.03}) double expectedFpp, - @Values(ints = {BloomFilterImpl.DEFAULT_SEED}) int deterministicSeed, + @Values(ints = {BloomFilterImplV2.DEFAULT_SEED}) int deterministicSeed, // preferable minimum parameter space for tests: // @Values(longs = {1_000_000L, 1_000_000_000L}) long numItems, // @Values(doubles = {0.05, 0.03, 0.01, 0.001}) double expectedFpp, @@ -138,7 +138,14 @@ public void testAccuracyEvenOdd( + " bits)" ); - BloomFilter bloomFilter = BloomFilter.create(numItems, optimalNumOfBits, deterministicSeed); + BloomFilter bloomFilter = + BloomFilter.create( + BloomFilter.Version.V2, + numItems, + optimalNumOfBits, + deterministicSeed + ); + testOut.printf( "allocated bitArray: %d (%d MB)\n", bloomFilter.bitSize(), @@ -236,7 +243,7 @@ public void testAccuracyRandom( // with the default FPP and the default seed only. @Values(longs = {1_000_000_000L}) long numItems, @Values(doubles = {0.03}) double expectedFpp, - @Values(ints = {BloomFilterImpl.DEFAULT_SEED}) int deterministicSeed, + @Values(ints = {BloomFilterImplV2.DEFAULT_SEED}) int deterministicSeed, // preferable minimum parameter space for tests: // @Values(longs = {1_000_000L, 1_000_000_000L}) long numItems, // @Values(doubles = {0.05, 0.03, 0.01, 0.001}) double expectedFpp, @@ -260,9 +267,20 @@ public void testAccuracyRandom( ); BloomFilter bloomFilterPrimary = - BloomFilter.create(numItems, optimalNumOfBits, deterministicSeed); + BloomFilter.create( + BloomFilter.Version.V2, + numItems, + optimalNumOfBits, + deterministicSeed + ); + BloomFilter bloomFilterSecondary = - BloomFilter.create(numItems, optimalNumOfBits, 0xCAFEBABE); + BloomFilter.create( + BloomFilter.Version.V2, + numItems, + optimalNumOfBits, + 0xCAFEBABE + ); testOut.printf( "allocated bitArray: %d (%d MB)\n", diff --git a/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala b/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala index d1e709e83a5a9..ba8f97a51aecf 100644 --- a/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala +++ b/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala @@ -160,5 +160,11 @@ class BloomFilterSuite extends AnyFunSuite { // scalastyle:ignore funsuite val filter2 = BloomFilter.create(2000, 6400) filter1.mergeInPlace(filter2) } + + intercept[IncompatibleMergeException] { + val filter1 = BloomFilter.create(BloomFilter.Version.V1, 1000L, 6400L, 0) + val filter2 = BloomFilter.create(BloomFilter.Version.V2, 1000L, 6400L, 0) + filter1.mergeInPlace(filter2) + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala index 7b20a7fcb9239..55e60ddefde70 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala @@ -360,7 +360,7 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { test("Test that might_contain can take a constant value input (seeded version)") { checkAnswer(spark.sql( """SELECT might_contain( - |X'00000001000000050000000343A2EC6EA8C117E2D3CDB767296B144FC5BFBCED9737F26700000000', + |X'0000000200000005000000000000000343A2EC6EA8C117E2D3CDB767296B144FC5BFBCED9737F267', |cast(201 as long))""".stripMargin), Row(false)) } From 6d44c1eedaaec6c9374d30bfced698bf2360ab10 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Sun, 6 Jul 2025 14:32:59 +0200 Subject: [PATCH 20/44] SPARK-47547 BloomFilter fpp degradation: scrambling test input without java.util.Random --- .../util/sketch/SparkBloomFilterSuite.java | 23 +++++++++++-------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java index 2dc2aba992276..aab1e992fe537 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java @@ -27,7 +27,6 @@ import java.time.Duration; import java.time.Instant; import java.util.Map; -import java.util.Random; import java.util.concurrent.ConcurrentHashMap; @@ -237,7 +236,7 @@ public void testAccuracyEvenOdd( * initialized with the constant seed of 0xCAFEBABE) */ @CartesianTest(name = "testAccuracyRandom_{index}.n{0}_fpp{1}_seed{2}") - public void testAccuracyRandom( + public void testAccuracyRandomDistribution( // temporary workaround: // to reduce running time to acceptable levels, we test only one case, // with the default FPP and the default seed only. @@ -288,17 +287,13 @@ public void testAccuracyRandom( bloomFilterPrimary.bitSize() / Byte.SIZE / 1024 / 1024 ); - - Random pseudoRandom = new Random(); long iterationCount = 2 * numItems; - - pseudoRandom.setSeed(deterministicSeed); for (long i = 0; i < iterationCount; i++) { if (verbose && i % 10_000_000 == 0) { System.err.printf("i: %d\n", i); } - long candidate = pseudoRandom.nextLong(); + long candidate = scramble(i); if (i % 2 == 0) { bloomFilterPrimary.putLong(candidate); bloomFilterSecondary.putLong(candidate); @@ -312,14 +307,12 @@ public void testAccuracyRandom( long mightContainEvenIndexed = 0; long mightContainOddIndexed = 0; long confirmedAsNotInserted = 0; - - pseudoRandom.setSeed(deterministicSeed); for (long i = 0; i < iterationCount; i++) { if (verbose && i % (iterationCount / 100) == 0) { System.err.printf("%s: %2d %%\n", testName, 100 * i / iterationCount); } - long candidate = pseudoRandom.nextLong(); + long candidate = scramble(i); if (i % 2 == 0) { // EVEN mightContainEvenIndexed++; @@ -379,4 +372,14 @@ public void testAccuracyRandom( ); } } + + // quick scrambling logic hacked out from java.util.Random + // its range is only 48bits (out of the 64bits of a Long value), + // but it should be enough for the purposes of this test. + private static final long multiplier = 0x5DEECE66DL; + private static final long addend = 0xBL; + private static final long mask = (1L << 48) - 1; + private static long scramble(long value) { + return (value * multiplier + addend) & mask; + } } From 925bf125339e1382ce7e4d484023428665a5a409 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Sun, 6 Jul 2025 15:21:22 +0200 Subject: [PATCH 21/44] SPARK-47547 BloomFilter fpp degradation: parallelizing BloomFilter reads in tests --- .../util/sketch/SparkBloomFilterSuite.java | 103 +++++++++--------- 1 file changed, 54 insertions(+), 49 deletions(-) diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java index aab1e992fe537..dcd9a84dd9745 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java @@ -28,6 +28,8 @@ import java.time.Instant; import java.util.Map; import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.atomic.LongAdder; +import java.util.stream.LongStream; public class SparkBloomFilterSuite { @@ -164,31 +166,30 @@ public void testAccuracyEvenOdd( (double) bloomFilter.cardinality() / bloomFilter.bitSize() ); - long mightContainEven = 0; - long mightContainOdd = 0; + LongAdder mightContainEven = new LongAdder(); + LongAdder mightContainOdd = new LongAdder(); - for (long i = 0; i < numItems; i++) { - if (verbose && i % (numItems / 100) == 0) { - System.err.printf("%s: %2d %%\n", testName, 100 * i / numItems); - } - - long even = 2 * i; - if (bloomFilter.mightContainLong(even)) { - mightContainEven++; - } + LongStream inputStream = LongStream.range(0, numItems).parallel(); + inputStream.forEach( + i -> { + long even = 2 * i; + if (bloomFilter.mightContainLong(even)) { + mightContainEven.increment(); + } - long odd = 2 * i + 1; - if (bloomFilter.mightContainLong(odd)) { - mightContainOdd++; + long odd = 2 * i + 1; + if (bloomFilter.mightContainLong(odd)) { + mightContainOdd.increment(); + } } - } + ); Assertions.assertEquals( - numItems, mightContainEven, + numItems, mightContainEven.longValue(), "mightContainLong must return true for all inserted numbers" ); - double actualFpp = (double) mightContainOdd / numItems; + double actualFpp = mightContainOdd.doubleValue() / numItems; double acceptableFpp = expectedFpp * (1 + FPP_EVEN_ODD_ERROR_FACTOR); testOut.printf("expectedFpp: %f %%\n", 100 * expectedFpp); @@ -288,6 +289,7 @@ public void testAccuracyRandomDistribution( ); long iterationCount = 2 * numItems; + for (long i = 0; i < iterationCount; i++) { if (verbose && i % 10_000_000 == 0) { System.err.printf("i: %d\n", i); @@ -304,49 +306,52 @@ public void testAccuracyRandomDistribution( (double) bloomFilterPrimary.cardinality() / bloomFilterPrimary.bitSize() ); - long mightContainEvenIndexed = 0; - long mightContainOddIndexed = 0; - long confirmedAsNotInserted = 0; - for (long i = 0; i < iterationCount; i++) { - if (verbose && i % (iterationCount / 100) == 0) { - System.err.printf("%s: %2d %%\n", testName, 100 * i / iterationCount); - } - - long candidate = scramble(i); + LongAdder mightContainEvenIndexed = new LongAdder(); + LongAdder mightContainOddIndexed = new LongAdder(); + LongAdder confirmedAsNotInserted = new LongAdder(); + LongStream inputStream = LongStream.range(0, iterationCount).parallel(); + inputStream.forEach( + i -> { + if (verbose && i % (iterationCount / 100) == 0) { + System.err.printf("%s: %2d %%\n", testName, 100 * i / iterationCount); + } - if (i % 2 == 0) { // EVEN - mightContainEvenIndexed++; - } else { // ODD - // for fpp estimation, only consider the odd indexes - // (to avoid querying the secondary with elements known to be inserted) - - // since here we avoided all the even indexes, - // most of these secondary queries will return false - if (!bloomFilterSecondary.mightContainLong(candidate)) { - // from the odd indexes, we consider only those items - // where the secondary confirms the non-insertion - - // anything on which the primary and the secondary - // disagrees here is a false positive - if (bloomFilterPrimary.mightContainLong(candidate)) { - mightContainOddIndexed++; + long candidate = scramble(i); + + if (i % 2 == 0) { // EVEN + mightContainEvenIndexed.increment(); + } else { // ODD + // for fpp estimation, only consider the odd indexes + // (to avoid querying the secondary with elements known to be inserted) + + // since here we avoided all the even indexes, + // most of these secondary queries will return false + if (!bloomFilterSecondary.mightContainLong(candidate)) { + // from the odd indexes, we consider only those items + // where the secondary confirms the non-insertion + + // anything on which the primary and the secondary + // disagrees here is a false positive + if (bloomFilterPrimary.mightContainLong(candidate)) { + mightContainOddIndexed.increment(); + } + // count the total number of considered items for a baseline + confirmedAsNotInserted.increment(); } - // count the total number of considered items for a baseline - confirmedAsNotInserted++; } } - } + ); Assertions.assertEquals( - numItems, mightContainEvenIndexed, + numItems, mightContainEvenIndexed.longValue(), "mightContainLong must return true for all inserted numbers" ); - double actualFpp = (double) mightContainOddIndexed / confirmedAsNotInserted; + double actualFpp = mightContainOddIndexed.doubleValue() / confirmedAsNotInserted.doubleValue(); double acceptableFpp = expectedFpp * (1 + FPP_RANDOM_ERROR_FACTOR); - testOut.printf("mightContainOddIndexed: %10d\n", mightContainOddIndexed); - testOut.printf("confirmedAsNotInserted: %10d\n", confirmedAsNotInserted); + testOut.printf("mightContainOddIndexed: %10d\n", mightContainOddIndexed.longValue()); + testOut.printf("confirmedAsNotInserted: %10d\n", confirmedAsNotInserted.longValue()); testOut.printf("numItems: %10d\n", numItems); testOut.printf("expectedFpp: %f %%\n", 100 * expectedFpp); testOut.printf("acceptableFpp: %f %%\n", 100 * acceptableFpp); From 6f288827f955243ec1b2ffb31b60c2ec026a29a0 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Mon, 7 Jul 2025 09:42:00 +0200 Subject: [PATCH 22/44] SPARK-47547 BloomFilter fpp degradation: add seed to equals/hashCode --- .../org/apache/spark/util/sketch/BloomFilterImplV2.java | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java index f5f1b2b0838ae..748d6bc897cd3 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java @@ -18,6 +18,7 @@ package org.apache.spark.util.sketch; import java.io.*; +import java.util.Objects; class BloomFilterImplV2 extends BloomFilter implements Serializable { @@ -54,12 +55,14 @@ public boolean equals(Object other) { return false; } - return this.numHashFunctions == that.numHashFunctions && this.bits.equals(that.bits); + return this.numHashFunctions == that.numHashFunctions + && this.seed == that.seed + && Objects.equals(this.bits, that.bits); } @Override public int hashCode() { - return bits.hashCode() * 31 + numHashFunctions; + return Objects.hash(numHashFunctions, seed, bits); } @Override From ed6caac645028c9af7af2f8c0324ae79d08d1a62 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Mon, 7 Jul 2025 09:54:09 +0200 Subject: [PATCH 23/44] SPARK-47547 BloomFilter fpp degradation: checkstyle fix --- .../org/apache/spark/util/sketch/SparkBloomFilterSuite.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java index dcd9a84dd9745..eccfaa3657bc2 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java @@ -347,7 +347,8 @@ public void testAccuracyRandomDistribution( "mightContainLong must return true for all inserted numbers" ); - double actualFpp = mightContainOddIndexed.doubleValue() / confirmedAsNotInserted.doubleValue(); + double actualFpp = + mightContainOddIndexed.doubleValue() / confirmedAsNotInserted.doubleValue(); double acceptableFpp = expectedFpp * (1 + FPP_RANDOM_ERROR_FACTOR); testOut.printf("mightContainOddIndexed: %10d\n", mightContainOddIndexed.longValue()); From 7d4ef74bf3409d85b880bcfb36a62081dea7a983 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Mon, 7 Jul 2025 15:20:39 +0200 Subject: [PATCH 24/44] SPARK-47547 BloomFilter fpp degradation: remove dependency between loops to aid later parallelization --- .../spark/util/sketch/BloomFilterImplV2.java | 30 +++++++++---------- 1 file changed, 14 insertions(+), 16 deletions(-) diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java index 748d6bc897cd3..7cd1eca5176bf 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java @@ -103,11 +103,11 @@ public boolean putBinary(byte[] item) { long combinedHash = (long) h1 * Integer.MAX_VALUE; for (long i = 0; i < numHashFunctions; i++) { combinedHash += h2; + // Flip all the bits if it's negative (guaranteed positive number) - if (combinedHash < 0) { - combinedHash = ~combinedHash; - } - bitsChanged |= bits.set(combinedHash % bitSize); + long combinedIndex = combinedHash < 0 ? ~combinedHash : combinedHash; + + bitsChanged |= bits.set(combinedIndex % bitSize); } return bitsChanged; } @@ -128,11 +128,11 @@ public boolean mightContainBinary(byte[] item) { long combinedHash = (long) h1 * Integer.MAX_VALUE; for (long i = 0; i < numHashFunctions; i++) { combinedHash += h2; + // Flip all the bits if it's negative (guaranteed positive number) - if (combinedHash < 0) { - combinedHash = ~combinedHash; - } - if (!bits.get(combinedHash % bitSize)) { + long combinedIndex = combinedHash < 0 ? ~combinedHash : combinedHash; + + if (!bits.get(combinedIndex % bitSize)) { return false; } } @@ -158,10 +158,9 @@ public boolean putLong(long item) { combinedHash += h2; // Flip all the bits if it's negative (guaranteed positive number) - if (combinedHash < 0) { - combinedHash = ~combinedHash; - } - bitsChanged |= bits.set(combinedHash % bitSize); + long combinedIndex = combinedHash < 0 ? ~combinedHash : combinedHash; + + bitsChanged |= bits.set(combinedIndex % bitSize); } return bitsChanged; } @@ -179,10 +178,9 @@ public boolean mightContainLong(long item) { combinedHash += h2; // Flip all the bits if it's negative (guaranteed positive number) - if (combinedHash < 0) { - combinedHash = ~combinedHash; - } - if (!bits.get(combinedHash % bitSize)) { + long combinedIndex = combinedHash < 0 ? ~combinedHash : combinedHash; + + if (!bits.get(combinedIndex % bitSize)) { return false; } } From 0ab8276ed6263226c8622d91a3d756943c487b37 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Mon, 7 Jul 2025 19:42:28 +0200 Subject: [PATCH 25/44] SPARK-47547 BloomFilter fpp degradation: running /dev/scalafmt --- .../spark/util/sketch/BloomFilterSuite.scala | 8 +- .../sql/BloomFilterAggregateQuerySuite.scala | 192 +++++++----------- 2 files changed, 83 insertions(+), 117 deletions(-) diff --git a/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala b/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala index ba8f97a51aecf..26963dbb291e9 100644 --- a/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala +++ b/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala @@ -72,7 +72,9 @@ class BloomFilterSuite extends AnyFunSuite { // scalastyle:ignore funsuite } } - def testMergeInPlace[T: ClassTag](typeName: String, numItems: Int)(itemGen: Random => T): Unit = { + def testMergeInPlace[T: ClassTag]( + typeName: String, + numItems: Int)(itemGen: Random => T): Unit = { test(s"mergeInPlace - $typeName") { // use a fixed seed to make the test predictable. val r = new Random(37) @@ -99,8 +101,8 @@ class BloomFilterSuite extends AnyFunSuite { // scalastyle:ignore funsuite } } - def testIntersectInPlace[T: ClassTag] - (typeName: String, numItems: Int)(itemGen: Random => T): Unit = { + def testIntersectInPlace[T: ClassTag](typeName: String, numItems: Int)( + itemGen: Random => T): Unit = { test(s"intersectInPlace - $typeName") { // use a fixed seed to make the test predictable. val r = new Random(37) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala index 55e60ddefde70..fb279b1db6fc9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/BloomFilterAggregateQuerySuite.scala @@ -41,16 +41,19 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { override def beforeAll(): Unit = { super.beforeAll() // Register 'bloom_filter_agg' to builtin. - spark.sessionState.functionRegistry.registerFunction(funcId_bloom_filter_agg, + spark.sessionState.functionRegistry.registerFunction( + funcId_bloom_filter_agg, new ExpressionInfo(classOf[BloomFilterAggregate].getName, "bloom_filter_agg"), - (children: Seq[Expression]) => children.size match { - case 1 => new BloomFilterAggregate(children.head) - case 2 => new BloomFilterAggregate(children.head, children(1)) - case 3 => new BloomFilterAggregate(children.head, children(1), children(2)) - }) + (children: Seq[Expression]) => + children.size match { + case 1 => new BloomFilterAggregate(children.head) + case 2 => new BloomFilterAggregate(children.head, children(1)) + case 3 => new BloomFilterAggregate(children.head, children(1), children(2)) + }) // Register 'might_contain' to builtin. - spark.sessionState.functionRegistry.registerFunction(funcId_might_contain, + spark.sessionState.functionRegistry.registerFunction( + funcId_might_contain, new ExpressionInfo(classOf[BloomFilterMightContain].getName, "might_contain"), (children: Seq[Expression]) => BloomFilterMightContain(children.head, children(1))) } @@ -64,10 +67,22 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { test("Test bloom_filter_agg and might_contain") { val conf = SQLConf.get val table = "bloom_filter_test" - for (numEstimatedItems <- Seq(Long.MinValue, -10L, 0L, 4096L, 4194304L, Long.MaxValue, - conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS))) { - for ((numBits, index) <- Seq(Long.MinValue, -10L, 0L, 4096L, 4194304L, Long.MaxValue, - conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_BITS)).zipWithIndex) { + for (numEstimatedItems <- Seq( + Long.MinValue, + -10L, + 0L, + 4096L, + 4194304L, + Long.MaxValue, + conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS))) { + for ((numBits, index) <- Seq( + Long.MinValue, + -10L, + 0L, + 4096L, + 4194304L, + Long.MaxValue, + conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_BITS)).zipWithIndex) { val sqlString = s""" |SELECT every(might_contain( | (SELECT bloom_filter_agg(col, @@ -85,7 +100,8 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { """.stripMargin withTempView(table) { (Seq(Long.MinValue, 0, Long.MaxValue) ++ (1L to 10000L)) - .toDF("col").createOrReplaceTempView(table) + .toDF("col") + .createOrReplaceTempView(table) // Validate error messages as well as answers when there's no error. if (numEstimatedItems <= 0) { val exception = intercept[AnalysisException] { @@ -104,16 +120,13 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { "valueRange" -> "[0, positive]", "currentValue" -> toSQLValue(numEstimatedItems, LongType), "sqlExpr" -> (s""""bloom_filter_agg(col, CAST($numEstimatedItems AS BIGINT), """ + - s"""CAST($numBits AS BIGINT))"""") - ), + s"""CAST($numBits AS BIGINT))"""")), context = ExpectedContext( fragment = "bloom_filter_agg(col,\n" + s" cast($numEstimatedItems as long),\n" + s" cast($numBits as long))", start = 49, - stop = stop(index) - ) - ) + stop = stop(index))) } else if (numBits <= 0) { val exception = intercept[AnalysisException] { spark.sql(sqlString) @@ -132,16 +145,13 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { "valueRange" -> "[0, positive]", "currentValue" -> toSQLValue(numBits, LongType), "sqlExpr" -> (s""""bloom_filter_agg(col, CAST($numEstimatedItems AS BIGINT), """ + - s"""CAST($numBits AS BIGINT))"""") - ), + s"""CAST($numBits AS BIGINT))"""")), context = ExpectedContext( fragment = "bloom_filter_agg(col,\n" + s" cast($numEstimatedItems as long),\n" + s" cast($numBits as long))", start = 49, - stop = stop(index) - ) - ) + stop = stop(index))) } else { checkAnswer(spark.sql(sqlString), Row(true, false)) } @@ -154,8 +164,7 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { val exception1 = intercept[AnalysisException] { spark.sql(""" |SELECT bloom_filter_agg(a) - |FROM values (1.2), (2.5) as t(a)""" - .stripMargin) + |FROM values (1.2), (2.5) as t(a)""".stripMargin) } checkError( exception = exception1, @@ -165,20 +174,13 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { "sqlExpr" -> "\"bloom_filter_agg(a, 1000000, 8388608)\"", "expectedLeft" -> "\"BINARY\"", "expectedRight" -> "\"BIGINT\"", - "actual" -> "\"DECIMAL(2,1)\", \"BIGINT\", \"BIGINT\"" - ), - context = ExpectedContext( - fragment = "bloom_filter_agg(a)", - start = 8, - stop = 26 - ) - ) + "actual" -> "\"DECIMAL(2,1)\", \"BIGINT\", \"BIGINT\""), + context = ExpectedContext(fragment = "bloom_filter_agg(a)", start = 8, stop = 26)) val exception2 = intercept[AnalysisException] { spark.sql(""" |SELECT bloom_filter_agg(a, 2) - |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)""" - .stripMargin) + |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)""".stripMargin) } checkError( exception = exception2, @@ -188,20 +190,13 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { "sqlExpr" -> "\"bloom_filter_agg(a, 2, (2 * 8))\"", "expectedLeft" -> "\"BINARY\"", "expectedRight" -> "\"BIGINT\"", - "actual" -> "\"BIGINT\", \"INT\", \"BIGINT\"" - ), - context = ExpectedContext( - fragment = "bloom_filter_agg(a, 2)", - start = 8, - stop = 29 - ) - ) + "actual" -> "\"BIGINT\", \"INT\", \"BIGINT\""), + context = ExpectedContext(fragment = "bloom_filter_agg(a, 2)", start = 8, stop = 29)) val exception3 = intercept[AnalysisException] { spark.sql(""" |SELECT bloom_filter_agg(a, cast(2 as long), 5) - |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)""" - .stripMargin) + |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)""".stripMargin) } checkError( exception = exception3, @@ -211,60 +206,42 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { "sqlExpr" -> "\"bloom_filter_agg(a, CAST(2 AS BIGINT), 5)\"", "expectedLeft" -> "\"BINARY\"", "expectedRight" -> "\"BIGINT\"", - "actual" -> "\"BIGINT\", \"BIGINT\", \"INT\"" - ), + "actual" -> "\"BIGINT\", \"BIGINT\", \"INT\""), context = ExpectedContext( fragment = "bloom_filter_agg(a, cast(2 as long), 5)", start = 8, - stop = 46 - ) - ) + stop = 46)) val exception4 = intercept[AnalysisException] { spark.sql(""" |SELECT bloom_filter_agg(a, null, 5) - |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)""" - .stripMargin) + |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)""".stripMargin) } checkError( exception = exception4, condition = "DATATYPE_MISMATCH.UNEXPECTED_NULL", parameters = Map( "exprName" -> "estimatedNumItems or numBits", - "sqlExpr" -> "\"bloom_filter_agg(a, NULL, 5)\"" - ), - context = ExpectedContext( - fragment = "bloom_filter_agg(a, null, 5)", - start = 8, - stop = 35 - ) - ) + "sqlExpr" -> "\"bloom_filter_agg(a, NULL, 5)\""), + context = ExpectedContext(fragment = "bloom_filter_agg(a, null, 5)", start = 8, stop = 35)) val exception5 = intercept[AnalysisException] { spark.sql(""" |SELECT bloom_filter_agg(a, 5, null) - |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)""" - .stripMargin) + |FROM values (cast(1 as long)), (cast(2 as long)) as t(a)""".stripMargin) } checkError( exception = exception5, condition = "DATATYPE_MISMATCH.UNEXPECTED_NULL", parameters = Map( "exprName" -> "estimatedNumItems or numBits", - "sqlExpr" -> "\"bloom_filter_agg(a, 5, NULL)\"" - ), - context = ExpectedContext( - fragment = "bloom_filter_agg(a, 5, null)", - start = 8, - stop = 35 - ) - ) + "sqlExpr" -> "\"bloom_filter_agg(a, 5, NULL)\""), + context = ExpectedContext(fragment = "bloom_filter_agg(a, 5, null)", start = 8, stop = 35)) } test("Test that might_contain errors out disallowed input value types") { val exception1 = intercept[AnalysisException] { - spark.sql("""|SELECT might_contain(1.0, 1L)""" - .stripMargin) + spark.sql("""|SELECT might_contain(1.0, 1L)""".stripMargin) } checkError( exception = exception1, @@ -274,18 +251,11 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { "functionName" -> "`might_contain`", "expectedLeft" -> "\"BINARY\"", "expectedRight" -> "\"BIGINT\"", - "actual" -> "\"DECIMAL(2,1)\", \"BIGINT\"" - ), - context = ExpectedContext( - fragment = "might_contain(1.0, 1L)", - start = 7, - stop = 28 - ) - ) + "actual" -> "\"DECIMAL(2,1)\", \"BIGINT\""), + context = ExpectedContext(fragment = "might_contain(1.0, 1L)", start = 7, stop = 28)) val exception2 = intercept[AnalysisException] { - spark.sql("""|SELECT might_contain(NULL, 0.1)""" - .stripMargin) + spark.sql("""|SELECT might_contain(NULL, 0.1)""".stripMargin) } checkError( exception = exception2, @@ -295,22 +265,15 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { "functionName" -> "`might_contain`", "expectedLeft" -> "\"BINARY\"", "expectedRight" -> "\"BIGINT\"", - "actual" -> "\"VOID\", \"DECIMAL(1,1)\"" - ), - context = ExpectedContext( - fragment = "might_contain(NULL, 0.1)", - start = 7, - stop = 30 - ) - ) + "actual" -> "\"VOID\", \"DECIMAL(1,1)\""), + context = ExpectedContext(fragment = "might_contain(NULL, 0.1)", start = 7, stop = 30)) } test("Test that might_contain errors out non-constant Bloom filter") { val exception1 = intercept[AnalysisException] { spark.sql(""" |SELECT might_contain(cast(a as binary), cast(5 as long)) - |FROM values (cast(1 as string)), (cast(2 as string)) as t(a)""" - .stripMargin) + |FROM values (cast(1 as string)), (cast(2 as string)) as t(a)""".stripMargin) } checkError( exception = exception1, @@ -318,20 +281,16 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { parameters = Map( "sqlExpr" -> "\"might_contain(CAST(a AS BINARY), CAST(5 AS BIGINT))\"", "functionName" -> "`might_contain`", - "actual" -> "\"CAST(a AS BINARY)\"" - ), + "actual" -> "\"CAST(a AS BINARY)\""), context = ExpectedContext( fragment = "might_contain(cast(a as binary), cast(5 as long))", start = 8, - stop = 56 - ) - ) + stop = 56)) val exception2 = intercept[AnalysisException] { spark.sql(""" |SELECT might_contain((select cast(a as binary)), cast(5 as long)) - |FROM values (cast(1 as string)), (cast(2 as string)) as t(a)""" - .stripMargin) + |FROM values (cast(1 as string)), (cast(2 as string)) as t(a)""".stripMargin) } checkError( exception = exception2, @@ -339,40 +298,38 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { parameters = Map( "sqlExpr" -> "\"might_contain(scalarsubquery(a), CAST(5 AS BIGINT))\"", "functionName" -> "`might_contain`", - "actual" -> "\"scalarsubquery(a)\"" - ), + "actual" -> "\"scalarsubquery(a)\""), context = ExpectedContext( fragment = "might_contain((select cast(a as binary)), cast(5 as long))", start = 8, - stop = 65 - ) - ) + stop = 65)) } test("Test that might_contain can take a constant value input (seedless version)") { - checkAnswer(spark.sql( - """SELECT might_contain( + checkAnswer( + spark.sql("""SELECT might_contain( |X'00000001000000050000000343A2EC6EA8C117E2D3CDB767296B144FC5BFBCED9737F267', |cast(201 as long))""".stripMargin), Row(false)) } test("Test that might_contain can take a constant value input (seeded version)") { - checkAnswer(spark.sql( - """SELECT might_contain( + checkAnswer( + spark.sql("""SELECT might_contain( |X'0000000200000005000000000000000343A2EC6EA8C117E2D3CDB767296B144FC5BFBCED9737F267', |cast(201 as long))""".stripMargin), Row(false)) } test("Test that bloom_filter_agg produces a NULL with empty input") { - checkAnswer(spark.sql("""SELECT bloom_filter_agg(cast(id as long)) from range(1, 1)"""), + checkAnswer( + spark.sql("""SELECT bloom_filter_agg(cast(id as long)) from range(1, 1)"""), Row(null)) } test("Test NULL inputs for might_contain") { - checkAnswer(spark.sql( - s""" + checkAnswer( + spark.sql(s""" |SELECT might_contain(null, null) both_null, | might_contain(null, 1L) null_bf, | might_contain((SELECT bloom_filter_agg(cast(id as long)) from range(1, 10000)), @@ -382,9 +339,15 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { } test("Test that a query with bloom_filter_agg has partial aggregates") { - assert(spark.sql("""SELECT bloom_filter_agg(cast(id as long)) from range(1, 1000000)""") - .queryExecution.executedPlan.asInstanceOf[AdaptiveSparkPlanExec].inputPlan - .collect({case agg: BaseAggregateExec => agg}).size == 2) + assert( + spark + .sql("""SELECT bloom_filter_agg(cast(id as long)) from range(1, 1000000)""") + .queryExecution + .executedPlan + .asInstanceOf[AdaptiveSparkPlanExec] + .inputPlan + .collect({ case agg: BaseAggregateExec => agg }) + .size == 2) } test("Test numBitsExpression") { @@ -393,7 +356,8 @@ class BloomFilterAggregateQuerySuite extends QueryTest with SharedSparkSession { assert(agg.numBitsExpression === Literal(numBits)) } - checkNumBits(conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS) * 100, + checkNumBits( + conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS) * 100, conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_BITS)) checkNumBits(conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS) + 10, 29193836) checkNumBits(conf.getConf(SQLConf.RUNTIME_BLOOM_FILTER_MAX_NUM_ITEMS), 29193763) From d2477bfc43b8d65769f7d5d88c650da274125ce0 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Mon, 7 Jul 2025 19:49:29 +0200 Subject: [PATCH 26/44] SPARK-47547 BloomFilter fpp degradation: javadoc comment for the V2 enum --- .../org/apache/spark/util/sketch/BloomFilter.java | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java index ca09155c1ced3..f5530c282f0a0 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java @@ -45,6 +45,7 @@ public abstract class BloomFilter { public enum Version { + /** * {@code BloomFilter} binary format version 1. All values written in big-endian order: *
      @@ -55,6 +56,20 @@ public enum Version { *
    */ V1(1), + + /** + * {@code BloomFilter} binary format version 2. + * Fixes the int32 truncation issue with V1 indexes, but by changing the bit pattern, + * it will become incompatible with V1 serializations. + * All values written in big-endian order: + *
      + *
    • Version number, always 2 (32 bit)
    • + *
    • Number of hash functions (32 bit)
    • + *
    • Integer seed to initialize hash functions (32 bit)
    • + *
    • Total number of words of the underlying bit array (32 bit)
    • + *
    • The words/longs (numWords * 64 bit)
    • + *
    + */ V2(2); private final int versionNumber; From 413c4fe7485bfa6ef601e940eb5ccb45015d310d Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Mon, 7 Jul 2025 20:04:31 +0200 Subject: [PATCH 27/44] SPARK-47547 BloomFilter fpp degradation: reindent with 2 spaces --- .../apache/spark/util/sketch/BloomFilter.java | 46 +- .../util/sketch/SparkBloomFilterSuite.java | 678 +++++++++--------- 2 files changed, 362 insertions(+), 362 deletions(-) diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java index f5530c282f0a0..a5cb031e49e75 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java @@ -194,25 +194,25 @@ public long cardinality() { * the stream. */ public static BloomFilter readFrom(InputStream in) throws IOException { - // peek into the inputstream so we can determine the version - BufferedInputStream bin = new BufferedInputStream(in); - bin.mark(4); - int version = ByteBuffer.wrap(bin.readNBytes(4)).getInt(); - bin.reset(); - - BloomFilter result; - switch (version) { - case 1: - result = BloomFilterImpl.readFrom(bin); - break; - case 2: - result = BloomFilterImplV2.readFrom(bin); - break; - default: - throw new IllegalArgumentException("Unknown BloomFilter version: " + version); - } - - return result; + // peek into the inputstream so we can determine the version + BufferedInputStream bin = new BufferedInputStream(in); + bin.mark(4); + int version = ByteBuffer.wrap(bin.readNBytes(4)).getInt(); + bin.reset(); + + BloomFilter result; + switch (version) { + case 1: + result = BloomFilterImpl.readFrom(bin); + break; + case 2: + result = BloomFilterImplV2.readFrom(bin); + break; + default: + throw new IllegalArgumentException("Unknown BloomFilter version: " + version); + } + + return result; } /** @@ -302,10 +302,10 @@ public static BloomFilter create(long expectedNumItems, long numBits, int seed) } public static BloomFilter create( - Version version, - long expectedNumItems, - long numBits, - int seed + Version version, + long expectedNumItems, + long numBits, + int seed ) { if (expectedNumItems <= 0) { throw new IllegalArgumentException("Expected insertions must be positive"); diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java index eccfaa3657bc2..35a7482f797a5 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java @@ -34,358 +34,358 @@ public class SparkBloomFilterSuite { - // the implemented fpp limit is only approximating the hard boundary, - // so we'll need an error threshold for the assertion - final double FPP_EVEN_ODD_ERROR_FACTOR = 0.10; - final double FPP_RANDOM_ERROR_FACTOR = 0.10; - - final long ONE_GB = 1024L * 1024L * 1024L; - final long REQUIRED_HEAP_UPPER_BOUND_IN_BYTES = 4 * ONE_GB; - - private static Instant START; - private static boolean strict; - private static boolean verbose; - - private Instant start; - private final Map testOutMap = new ConcurrentHashMap<>(); - - @BeforeAll - public static void beforeAll() { - START = Instant.now(); - String testClassName = SparkBloomFilterSuite.class.getName(); - strict = Boolean.parseBoolean(System.getProperty(testClassName+ ".strict", "true")); - verbose = Boolean.parseBoolean(System.getProperty(testClassName+ ".verbose", "false")); + // the implemented fpp limit is only approximating the hard boundary, + // so we'll need an error threshold for the assertion + final double FPP_EVEN_ODD_ERROR_FACTOR = 0.10; + final double FPP_RANDOM_ERROR_FACTOR = 0.10; + + final long ONE_GB = 1024L * 1024L * 1024L; + final long REQUIRED_HEAP_UPPER_BOUND_IN_BYTES = 4 * ONE_GB; + + private static Instant START; + private static boolean strict; + private static boolean verbose; + + private Instant start; + private final Map testOutMap = new ConcurrentHashMap<>(); + + @BeforeAll + public static void beforeAll() { + START = Instant.now(); + String testClassName = SparkBloomFilterSuite.class.getName(); + strict = Boolean.parseBoolean(System.getProperty(testClassName+ ".strict", "true")); + verbose = Boolean.parseBoolean(System.getProperty(testClassName+ ".verbose", "false")); + } + + @AfterAll + public static void afterAll() { + Duration duration = Duration.between(START, Instant.now()); + if (verbose) { + System.err.println(duration + " TOTAL"); } - - @AfterAll - public static void afterAll() { - Duration duration = Duration.between(START, Instant.now()); - if (verbose) { - System.err.println(duration + " TOTAL"); - } + } + + @BeforeEach + public void beforeEach( + TestInfo testInfo + ) throws Exception { + start = Instant.now(); + + String testName = testInfo.getDisplayName(); + + String testClassName = SparkBloomFilterSuite.class.getName(); + String logDir = System.getProperty(testClassName+ ".logDir", "./target/tmp"); + Path logDirPath = Path.of(logDir); + Files.createDirectories(logDirPath); + Path testLogPath = Path.of(logDir,testName + ".log"); + Files.deleteIfExists(testLogPath); + + PrintStream testOut = new PrintStream(Files.newOutputStream(testLogPath)); + testOutMap.put(testName, testOut); + + testOut.println("testName: " + testName); + } + + @AfterEach + public void afterEach(TestInfo testInfo) { + Duration duration = Duration.between(start, Instant.now()); + + String testName = testInfo.getDisplayName(); + PrintStream testOut = testOutMap.get(testName); + + testOut.println("duration: " + duration ); + testOut.close(); + } + + /** + * This test, in N number of iterations, inserts N even numbers (2*i) int, + * and leaves out N odd numbers (2*i+1) from the tested BloomFilter instance. + * + * It checks the 100% accuracy of mightContain=true on all of the even items, + * and measures the mightContain=true (false positive) rate on the not-inserted odd numbers. + * + * @param numItems the number of items to be inserted + * @param expectedFpp the expected fpp rate of the tested BloomFilter instance + * @param deterministicSeed the deterministic seed to use to initialize + * the primary BloomFilter instance. + */ + @CartesianTest(name = "testAccuracyEvenOdd_{index}.n{0}_fpp{1}_seed{2}") + public void testAccuracyEvenOdd( + // temporary workaround: + // to reduce running time to acceptable levels, we test only one case, + // with the default FPP and the default seed only. + @Values(longs = {1_000_000_000L}) long numItems, + @Values(doubles = {0.03}) double expectedFpp, + @Values(ints = {BloomFilterImplV2.DEFAULT_SEED}) int deterministicSeed, + // preferable minimum parameter space for tests: + // @Values(longs = {1_000_000L, 1_000_000_000L}) long numItems, + // @Values(doubles = {0.05, 0.03, 0.01, 0.001}) double expectedFpp, + // @Values(ints = {BloomFilterImpl.DEFAULT_SEED, 1, 127}) int deterministicSeed, + TestInfo testInfo + ) { + String testName = testInfo.getDisplayName(); + PrintStream testOut = testOutMap.get(testName); + + long optimalNumOfBits = BloomFilter.optimalNumOfBits(numItems, expectedFpp); + testOut.printf( + "optimal bitArray: %d (%d MB)\n", + optimalNumOfBits, + optimalNumOfBits / Byte.SIZE / 1024 / 1024 + ); + Assumptions.assumeTrue( + optimalNumOfBits / Byte.SIZE < REQUIRED_HEAP_UPPER_BOUND_IN_BYTES, + "this testcase would require allocating more than 4GB of heap mem (" + + optimalNumOfBits + + " bits)" + ); + + BloomFilter bloomFilter = + BloomFilter.create( + BloomFilter.Version.V2, + numItems, + optimalNumOfBits, + deterministicSeed + ); + + testOut.printf( + "allocated bitArray: %d (%d MB)\n", + bloomFilter.bitSize(), + bloomFilter.bitSize() / Byte.SIZE / 1024 / 1024 + ); + + for (long i = 0; i < numItems; i++) { + if (verbose && i % 10_000_000 == 0) { + System.err.printf("i: %d\n", i); + } + + bloomFilter.putLong(2 * i); } - @BeforeEach - public void beforeEach( - TestInfo testInfo - ) throws Exception { - start = Instant.now(); - - String testName = testInfo.getDisplayName(); + testOut.printf("bitCount: %d\nsaturation: %f\n", + bloomFilter.cardinality(), + (double) bloomFilter.cardinality() / bloomFilter.bitSize() + ); - String testClassName = SparkBloomFilterSuite.class.getName(); - String logDir = System.getProperty(testClassName+ ".logDir", "./target/tmp"); - Path logDirPath = Path.of(logDir); - Files.createDirectories(logDirPath); - Path testLogPath = Path.of(logDir,testName + ".log"); - Files.deleteIfExists(testLogPath); + LongAdder mightContainEven = new LongAdder(); + LongAdder mightContainOdd = new LongAdder(); - PrintStream testOut = new PrintStream(Files.newOutputStream(testLogPath)); - testOutMap.put(testName, testOut); + LongStream inputStream = LongStream.range(0, numItems).parallel(); + inputStream.forEach( + i -> { + long even = 2 * i; + if (bloomFilter.mightContainLong(even)) { + mightContainEven.increment(); + } - testOut.println("testName: " + testName); + long odd = 2 * i + 1; + if (bloomFilter.mightContainLong(odd)) { + mightContainOdd.increment(); + } + } + ); + + Assertions.assertEquals( + numItems, mightContainEven.longValue(), + "mightContainLong must return true for all inserted numbers" + ); + + double actualFpp = mightContainOdd.doubleValue() / numItems; + double acceptableFpp = expectedFpp * (1 + FPP_EVEN_ODD_ERROR_FACTOR); + + testOut.printf("expectedFpp: %f %%\n", 100 * expectedFpp); + testOut.printf("acceptableFpp: %f %%\n", 100 * acceptableFpp); + testOut.printf("actualFpp: %f %%\n", 100 * actualFpp); + + if (!strict) { + Assumptions.assumeTrue( + actualFpp <= acceptableFpp, + String.format( + "acceptableFpp(%f %%) < actualFpp (%f %%)", + 100 * acceptableFpp, + 100 * actualFpp + ) + ); + } else { + Assertions.assertTrue( + actualFpp <= acceptableFpp, + String.format( + "acceptableFpp(%f %%) < actualFpp (%f %%)", + 100 * acceptableFpp, + 100 * actualFpp + ) + ); } - - @AfterEach - public void afterEach(TestInfo testInfo) { - Duration duration = Duration.between(start, Instant.now()); - - String testName = testInfo.getDisplayName(); - PrintStream testOut = testOutMap.get(testName); - - testOut.println("duration: " + duration ); - testOut.close(); + } + + /** + * This test inserts N pseudorandomly generated numbers in 2N number of iterations in two + * differently seeded (theoretically independent) BloomFilter instances. All the random + * numbers generated in an even-iteration will be inserted into both filters, all the + * random numbers generated in an odd-iteration will be left out from both. + * + * The test checks the 100% accuracy of 'mightContain=true' for all the items inserted + * in an even-loop. It counts the false positives as the number of odd-loop items for + * which the primary filter reports 'mightContain=true', but secondary reports + * 'mightContain=false'. Since we inserted the same elements into both instances, + * and the secondary reports non-insertion, the 'mightContain=true' from the primary + * can only be a false positive. + * + * @param numItems the number of items to be inserted + * @param expectedFpp the expected fpp rate of the tested BloomFilter instance + * @param deterministicSeed the deterministic seed to use to initialize + * the primary BloomFilter instance. (The secondary will be + * initialized with the constant seed of 0xCAFEBABE) + */ + @CartesianTest(name = "testAccuracyRandom_{index}.n{0}_fpp{1}_seed{2}") + public void testAccuracyRandomDistribution( + // temporary workaround: + // to reduce running time to acceptable levels, we test only one case, + // with the default FPP and the default seed only. + @Values(longs = {1_000_000_000L}) long numItems, + @Values(doubles = {0.03}) double expectedFpp, + @Values(ints = {BloomFilterImplV2.DEFAULT_SEED}) int deterministicSeed, + // preferable minimum parameter space for tests: + // @Values(longs = {1_000_000L, 1_000_000_000L}) long numItems, + // @Values(doubles = {0.05, 0.03, 0.01, 0.001}) double expectedFpp, + // @Values(ints = {BloomFilterImpl.DEFAULT_SEED, 1, 127}) int deterministicSeed, + TestInfo testInfo + ) { + String testName = testInfo.getDisplayName(); + PrintStream testOut = testOutMap.get(testName); + + long optimalNumOfBits = BloomFilter.optimalNumOfBits(numItems, expectedFpp); + testOut.printf( + "optimal bitArray: %d (%d MB)\n", + optimalNumOfBits, + optimalNumOfBits / Byte.SIZE / 1024 / 1024 + ); + Assumptions.assumeTrue( + 2 * optimalNumOfBits / Byte.SIZE < REQUIRED_HEAP_UPPER_BOUND_IN_BYTES, + "this testcase would require allocating more than 4GB of heap mem (2x " + + optimalNumOfBits + + " bits)" + ); + + BloomFilter bloomFilterPrimary = + BloomFilter.create( + BloomFilter.Version.V2, + numItems, + optimalNumOfBits, + deterministicSeed + ); + + BloomFilter bloomFilterSecondary = + BloomFilter.create( + BloomFilter.Version.V2, + numItems, + optimalNumOfBits, + 0xCAFEBABE + ); + + testOut.printf( + "allocated bitArray: %d (%d MB)\n", + bloomFilterPrimary.bitSize(), + bloomFilterPrimary.bitSize() / Byte.SIZE / 1024 / 1024 + ); + + long iterationCount = 2 * numItems; + + for (long i = 0; i < iterationCount; i++) { + if (verbose && i % 10_000_000 == 0) { + System.err.printf("i: %d\n", i); + } + + long candidate = scramble(i); + if (i % 2 == 0) { + bloomFilterPrimary.putLong(candidate); + bloomFilterSecondary.putLong(candidate); + } } - - /** - * This test, in N number of iterations, inserts N even numbers (2*i) int, - * and leaves out N odd numbers (2*i+1) from the tested BloomFilter instance. - * - * It checks the 100% accuracy of mightContain=true on all of the even items, - * and measures the mightContain=true (false positive) rate on the not-inserted odd numbers. - * - * @param numItems the number of items to be inserted - * @param expectedFpp the expected fpp rate of the tested BloomFilter instance - * @param deterministicSeed the deterministic seed to use to initialize - * the primary BloomFilter instance. - */ - @CartesianTest(name = "testAccuracyEvenOdd_{index}.n{0}_fpp{1}_seed{2}") - public void testAccuracyEvenOdd( - // temporary workaround: - // to reduce running time to acceptable levels, we test only one case, - // with the default FPP and the default seed only. - @Values(longs = {1_000_000_000L}) long numItems, - @Values(doubles = {0.03}) double expectedFpp, - @Values(ints = {BloomFilterImplV2.DEFAULT_SEED}) int deterministicSeed, - // preferable minimum parameter space for tests: - // @Values(longs = {1_000_000L, 1_000_000_000L}) long numItems, - // @Values(doubles = {0.05, 0.03, 0.01, 0.001}) double expectedFpp, - // @Values(ints = {BloomFilterImpl.DEFAULT_SEED, 1, 127}) int deterministicSeed, - TestInfo testInfo - ) { - String testName = testInfo.getDisplayName(); - PrintStream testOut = testOutMap.get(testName); - - long optimalNumOfBits = BloomFilter.optimalNumOfBits(numItems, expectedFpp); - testOut.printf( - "optimal bitArray: %d (%d MB)\n", - optimalNumOfBits, - optimalNumOfBits / Byte.SIZE / 1024 / 1024 - ); - Assumptions.assumeTrue( - optimalNumOfBits / Byte.SIZE < REQUIRED_HEAP_UPPER_BOUND_IN_BYTES, - "this testcase would require allocating more than 4GB of heap mem (" - + optimalNumOfBits - + " bits)" - ); - - BloomFilter bloomFilter = - BloomFilter.create( - BloomFilter.Version.V2, - numItems, - optimalNumOfBits, - deterministicSeed - ); - - testOut.printf( - "allocated bitArray: %d (%d MB)\n", - bloomFilter.bitSize(), - bloomFilter.bitSize() / Byte.SIZE / 1024 / 1024 - ); - - for (long i = 0; i < numItems; i++) { - if (verbose && i % 10_000_000 == 0) { - System.err.printf("i: %d\n", i); - } - - bloomFilter.putLong(2 * i); + testOut.printf("bitCount: %d\nsaturation: %f\n", + bloomFilterPrimary.cardinality(), + (double) bloomFilterPrimary.cardinality() / bloomFilterPrimary.bitSize() + ); + + LongAdder mightContainEvenIndexed = new LongAdder(); + LongAdder mightContainOddIndexed = new LongAdder(); + LongAdder confirmedAsNotInserted = new LongAdder(); + LongStream inputStream = LongStream.range(0, iterationCount).parallel(); + inputStream.forEach( + i -> { + if (verbose && i % (iterationCount / 100) == 0) { + System.err.printf("%s: %2d %%\n", testName, 100 * i / iterationCount); } - testOut.printf("bitCount: %d\nsaturation: %f\n", - bloomFilter.cardinality(), - (double) bloomFilter.cardinality() / bloomFilter.bitSize() - ); - - LongAdder mightContainEven = new LongAdder(); - LongAdder mightContainOdd = new LongAdder(); - - LongStream inputStream = LongStream.range(0, numItems).parallel(); - inputStream.forEach( - i -> { - long even = 2 * i; - if (bloomFilter.mightContainLong(even)) { - mightContainEven.increment(); - } - - long odd = 2 * i + 1; - if (bloomFilter.mightContainLong(odd)) { - mightContainOdd.increment(); - } - } - ); - - Assertions.assertEquals( - numItems, mightContainEven.longValue(), - "mightContainLong must return true for all inserted numbers" - ); - - double actualFpp = mightContainOdd.doubleValue() / numItems; - double acceptableFpp = expectedFpp * (1 + FPP_EVEN_ODD_ERROR_FACTOR); - - testOut.printf("expectedFpp: %f %%\n", 100 * expectedFpp); - testOut.printf("acceptableFpp: %f %%\n", 100 * acceptableFpp); - testOut.printf("actualFpp: %f %%\n", 100 * actualFpp); - - if (!strict) { - Assumptions.assumeTrue( - actualFpp <= acceptableFpp, - String.format( - "acceptableFpp(%f %%) < actualFpp (%f %%)", - 100 * acceptableFpp, - 100 * actualFpp - ) - ); - } else { - Assertions.assertTrue( - actualFpp <= acceptableFpp, - String.format( - "acceptableFpp(%f %%) < actualFpp (%f %%)", - 100 * acceptableFpp, - 100 * actualFpp - ) - ); - } - } + long candidate = scramble(i); - /** - * This test inserts N pseudorandomly generated numbers in 2N number of iterations in two - * differently seeded (theoretically independent) BloomFilter instances. All the random - * numbers generated in an even-iteration will be inserted into both filters, all the - * random numbers generated in an odd-iteration will be left out from both. - * - * The test checks the 100% accuracy of 'mightContain=true' for all the items inserted - * in an even-loop. It counts the false positives as the number of odd-loop items for - * which the primary filter reports 'mightContain=true', but secondary reports - * 'mightContain=false'. Since we inserted the same elements into both instances, - * and the secondary reports non-insertion, the 'mightContain=true' from the primary - * can only be a false positive. - * - * @param numItems the number of items to be inserted - * @param expectedFpp the expected fpp rate of the tested BloomFilter instance - * @param deterministicSeed the deterministic seed to use to initialize - * the primary BloomFilter instance. (The secondary will be - * initialized with the constant seed of 0xCAFEBABE) - */ - @CartesianTest(name = "testAccuracyRandom_{index}.n{0}_fpp{1}_seed{2}") - public void testAccuracyRandomDistribution( - // temporary workaround: - // to reduce running time to acceptable levels, we test only one case, - // with the default FPP and the default seed only. - @Values(longs = {1_000_000_000L}) long numItems, - @Values(doubles = {0.03}) double expectedFpp, - @Values(ints = {BloomFilterImplV2.DEFAULT_SEED}) int deterministicSeed, - // preferable minimum parameter space for tests: - // @Values(longs = {1_000_000L, 1_000_000_000L}) long numItems, - // @Values(doubles = {0.05, 0.03, 0.01, 0.001}) double expectedFpp, - // @Values(ints = {BloomFilterImpl.DEFAULT_SEED, 1, 127}) int deterministicSeed, - TestInfo testInfo - ) { - String testName = testInfo.getDisplayName(); - PrintStream testOut = testOutMap.get(testName); - - long optimalNumOfBits = BloomFilter.optimalNumOfBits(numItems, expectedFpp); - testOut.printf( - "optimal bitArray: %d (%d MB)\n", - optimalNumOfBits, - optimalNumOfBits / Byte.SIZE / 1024 / 1024 - ); - Assumptions.assumeTrue( - 2 * optimalNumOfBits / Byte.SIZE < REQUIRED_HEAP_UPPER_BOUND_IN_BYTES, - "this testcase would require allocating more than 4GB of heap mem (2x " - + optimalNumOfBits - + " bits)" - ); - - BloomFilter bloomFilterPrimary = - BloomFilter.create( - BloomFilter.Version.V2, - numItems, - optimalNumOfBits, - deterministicSeed - ); - - BloomFilter bloomFilterSecondary = - BloomFilter.create( - BloomFilter.Version.V2, - numItems, - optimalNumOfBits, - 0xCAFEBABE - ); - - testOut.printf( - "allocated bitArray: %d (%d MB)\n", - bloomFilterPrimary.bitSize(), - bloomFilterPrimary.bitSize() / Byte.SIZE / 1024 / 1024 - ); - - long iterationCount = 2 * numItems; - - for (long i = 0; i < iterationCount; i++) { - if (verbose && i % 10_000_000 == 0) { - System.err.printf("i: %d\n", i); - } + if (i % 2 == 0) { // EVEN + mightContainEvenIndexed.increment(); + } else { // ODD + // for fpp estimation, only consider the odd indexes + // (to avoid querying the secondary with elements known to be inserted) - long candidate = scramble(i); - if (i % 2 == 0) { - bloomFilterPrimary.putLong(candidate); - bloomFilterSecondary.putLong(candidate); - } - } - testOut.printf("bitCount: %d\nsaturation: %f\n", - bloomFilterPrimary.cardinality(), - (double) bloomFilterPrimary.cardinality() / bloomFilterPrimary.bitSize() - ); - - LongAdder mightContainEvenIndexed = new LongAdder(); - LongAdder mightContainOddIndexed = new LongAdder(); - LongAdder confirmedAsNotInserted = new LongAdder(); - LongStream inputStream = LongStream.range(0, iterationCount).parallel(); - inputStream.forEach( - i -> { - if (verbose && i % (iterationCount / 100) == 0) { - System.err.printf("%s: %2d %%\n", testName, 100 * i / iterationCount); - } - - long candidate = scramble(i); - - if (i % 2 == 0) { // EVEN - mightContainEvenIndexed.increment(); - } else { // ODD - // for fpp estimation, only consider the odd indexes - // (to avoid querying the secondary with elements known to be inserted) - - // since here we avoided all the even indexes, - // most of these secondary queries will return false - if (!bloomFilterSecondary.mightContainLong(candidate)) { - // from the odd indexes, we consider only those items - // where the secondary confirms the non-insertion - - // anything on which the primary and the secondary - // disagrees here is a false positive - if (bloomFilterPrimary.mightContainLong(candidate)) { - mightContainOddIndexed.increment(); - } - // count the total number of considered items for a baseline - confirmedAsNotInserted.increment(); - } - } + // since here we avoided all the even indexes, + // most of these secondary queries will return false + if (!bloomFilterSecondary.mightContainLong(candidate)) { + // from the odd indexes, we consider only those items + // where the secondary confirms the non-insertion + + // anything on which the primary and the secondary + // disagrees here is a false positive + if (bloomFilterPrimary.mightContainLong(candidate)) { + mightContainOddIndexed.increment(); } - ); - - Assertions.assertEquals( - numItems, mightContainEvenIndexed.longValue(), - "mightContainLong must return true for all inserted numbers" - ); - - double actualFpp = - mightContainOddIndexed.doubleValue() / confirmedAsNotInserted.doubleValue(); - double acceptableFpp = expectedFpp * (1 + FPP_RANDOM_ERROR_FACTOR); - - testOut.printf("mightContainOddIndexed: %10d\n", mightContainOddIndexed.longValue()); - testOut.printf("confirmedAsNotInserted: %10d\n", confirmedAsNotInserted.longValue()); - testOut.printf("numItems: %10d\n", numItems); - testOut.printf("expectedFpp: %f %%\n", 100 * expectedFpp); - testOut.printf("acceptableFpp: %f %%\n", 100 * acceptableFpp); - testOut.printf("actualFpp: %f %%\n", 100 * actualFpp); - - if (!strict) { - Assumptions.assumeTrue( - actualFpp <= acceptableFpp, - String.format( - "acceptableFpp(%f %%) < actualFpp (%f %%)", - 100 * acceptableFpp, - 100 * actualFpp - ) - ); - } else { - Assertions.assertTrue( - actualFpp <= acceptableFpp, - String.format( - "acceptableFpp(%f %%) < actualFpp (%f %%)", - 100 * acceptableFpp, - 100 * actualFpp - ) - ); + // count the total number of considered items for a baseline + confirmedAsNotInserted.increment(); + } } + } + ); + + Assertions.assertEquals( + numItems, mightContainEvenIndexed.longValue(), + "mightContainLong must return true for all inserted numbers" + ); + + double actualFpp = + mightContainOddIndexed.doubleValue() / confirmedAsNotInserted.doubleValue(); + double acceptableFpp = expectedFpp * (1 + FPP_RANDOM_ERROR_FACTOR); + + testOut.printf("mightContainOddIndexed: %10d\n", mightContainOddIndexed.longValue()); + testOut.printf("confirmedAsNotInserted: %10d\n", confirmedAsNotInserted.longValue()); + testOut.printf("numItems: %10d\n", numItems); + testOut.printf("expectedFpp: %f %%\n", 100 * expectedFpp); + testOut.printf("acceptableFpp: %f %%\n", 100 * acceptableFpp); + testOut.printf("actualFpp: %f %%\n", 100 * actualFpp); + + if (!strict) { + Assumptions.assumeTrue( + actualFpp <= acceptableFpp, + String.format( + "acceptableFpp(%f %%) < actualFpp (%f %%)", + 100 * acceptableFpp, + 100 * actualFpp + ) + ); + } else { + Assertions.assertTrue( + actualFpp <= acceptableFpp, + String.format( + "acceptableFpp(%f %%) < actualFpp (%f %%)", + 100 * acceptableFpp, + 100 * actualFpp + ) + ); } - - // quick scrambling logic hacked out from java.util.Random - // its range is only 48bits (out of the 64bits of a Long value), - // but it should be enough for the purposes of this test. - private static final long multiplier = 0x5DEECE66DL; - private static final long addend = 0xBL; - private static final long mask = (1L << 48) - 1; - private static long scramble(long value) { - return (value * multiplier + addend) & mask; - } + } + + // quick scrambling logic hacked out from java.util.Random +// its range is only 48bits (out of the 64bits of a Long value), +// but it should be enough for the purposes of this test. + private static final long multiplier = 0x5DEECE66DL; + private static final long addend = 0xBL; + private static final long mask = (1L << 48) - 1; + private static long scramble(long value) { + return (value * multiplier + addend) & mask; + } } From 4599fcb5596417d1eb645cfef90ef1b66e533c4b Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Mon, 7 Jul 2025 20:11:59 +0200 Subject: [PATCH 28/44] SPARK-47547 BloomFilter fpp degradation: (recover empty line in BloomFilter.java) --- .../src/main/java/org/apache/spark/util/sketch/BloomFilter.java | 1 + 1 file changed, 1 insertion(+) diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java index a5cb031e49e75..2443d78fa4093 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java @@ -289,6 +289,7 @@ public static BloomFilter create(long expectedNumItems, double fpp) { return create(expectedNumItems, optimalNumOfBits(expectedNumItems, fpp)); } + /** * Creates a {@link BloomFilter} with given {@code expectedNumItems} and {@code numBits}, it will * pick an optimal {@code numHashFunctions} which can minimize {@code fpp} for the bloom filter. From 1ee2e137607365017f3cf806411156b7fc05549a Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Mon, 7 Jul 2025 20:17:24 +0200 Subject: [PATCH 29/44] SPARK-47547 BloomFilter fpp degradation: JEP-361 style switches --- .../apache/spark/util/sketch/BloomFilter.java | 35 ++++++------------- 1 file changed, 10 insertions(+), 25 deletions(-) diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java index 2443d78fa4093..2c956ef51f8a0 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilter.java @@ -200,19 +200,11 @@ public static BloomFilter readFrom(InputStream in) throws IOException { int version = ByteBuffer.wrap(bin.readNBytes(4)).getInt(); bin.reset(); - BloomFilter result; - switch (version) { - case 1: - result = BloomFilterImpl.readFrom(bin); - break; - case 2: - result = BloomFilterImplV2.readFrom(bin); - break; - default: - throw new IllegalArgumentException("Unknown BloomFilter version: " + version); - } - - return result; + return switch (version) { + case 1 -> BloomFilterImpl.readFrom(bin); + case 2 -> BloomFilterImplV2.readFrom(bin); + default -> throw new IllegalArgumentException("Unknown BloomFilter version: " + version); + }; } /** @@ -318,17 +310,10 @@ public static BloomFilter create( int numHashFunctions = optimalNumOfHashFunctions(expectedNumItems, numBits); - BloomFilter result; - switch (version) { - case V1: - result = new BloomFilterImpl(numHashFunctions, numBits); - break; - case V2: - result = new BloomFilterImplV2(numHashFunctions, numBits, seed); - break; - default: - throw new IllegalArgumentException("Unknown BloomFilter version: " + version); - } - return result; + return switch (version) { + case V1 -> new BloomFilterImpl(numHashFunctions, numBits); + case V2 -> new BloomFilterImplV2(numHashFunctions, numBits, seed); + default -> throw new IllegalArgumentException("Unknown BloomFilter version: " + version); + }; } } From c501b2a2bca602854eb0847052b1a596e22c0a0b Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Mon, 7 Jul 2025 20:23:58 +0200 Subject: [PATCH 30/44] SPARK-47547 BloomFilter fpp degradation: removing Objects::equals --- .../org/apache/spark/util/sketch/BloomFilterImplV2.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java index 7cd1eca5176bf..f4b0d61671300 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java @@ -55,9 +55,10 @@ public boolean equals(Object other) { return false; } - return this.numHashFunctions == that.numHashFunctions - && this.seed == that.seed - && Objects.equals(this.bits, that.bits); + return + this.numHashFunctions == that.numHashFunctions + && this.seed == that.seed + && this.bits.equals(that.bits); } @Override From 1f5cfb607f74957a32b9b41de7ce10b1c144a380 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Mon, 7 Jul 2025 20:25:53 +0200 Subject: [PATCH 31/44] SPARK-47547 BloomFilter fpp degradation: add missing seed comparison in V2 compat check --- .../java/org/apache/spark/util/sketch/BloomFilterImplV2.java | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java index f4b0d61671300..d637df862c31f 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java @@ -209,7 +209,10 @@ public boolean isCompatible(BloomFilter other) { return false; } - return this.bitSize() == that.bitSize() && this.numHashFunctions == that.numHashFunctions; + return + this.bitSize() == that.bitSize() + && this.numHashFunctions == that.numHashFunctions + && this.seed == that.seed; } @Override From f60d55f0731141445ca95d1c044676dd7b7334a1 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Tue, 8 Jul 2025 10:31:56 +0200 Subject: [PATCH 32/44] SPARK-47547 BloomFilter fpp degradation: checkstyle --- .../org/apache/spark/util/sketch/SparkBloomFilterSuite.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java index 35a7482f797a5..e07e9d02932ae 100644 --- a/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java +++ b/common/sketch/src/test/java/org/apache/spark/util/sketch/SparkBloomFilterSuite.java @@ -380,8 +380,8 @@ public void testAccuracyRandomDistribution( } // quick scrambling logic hacked out from java.util.Random -// its range is only 48bits (out of the 64bits of a Long value), -// but it should be enough for the purposes of this test. + // its range is only 48bits (out of the 64bits of a Long value), + // but it should be enough for the purposes of this test. private static final long multiplier = 0x5DEECE66DL; private static final long addend = 0xBL; private static final long mask = (1L << 48) - 1; From 0314963e704434ed67707c627e7092d9c8fb2f99 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Fri, 11 Jul 2025 16:14:21 +0200 Subject: [PATCH 33/44] SPARK-47547 BloomFilter fpp degradation: BloomFilterBase abstract parent for BloomFilterImpl{,V2} --- .../spark/util/sketch/BloomFilterBase.java | 165 ++++++++++++++++++ .../spark/util/sketch/BloomFilterImpl.java | 113 +----------- .../spark/util/sketch/BloomFilterImplV2.java | 106 +---------- 3 files changed, 177 insertions(+), 207 deletions(-) create mode 100644 common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterBase.java diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterBase.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterBase.java new file mode 100644 index 0000000000000..fd12666d48997 --- /dev/null +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterBase.java @@ -0,0 +1,165 @@ +/* + * 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.spark.util.sketch; + +import java.util.Objects; + +abstract class BloomFilterBase extends BloomFilter { + + public static final int DEFAULT_SEED = 0; + + protected int seed; + protected int numHashFunctions; + protected BitArray bits; + + protected BloomFilterBase(int numHashFunctions, long numBits) { + this(numHashFunctions, numBits, DEFAULT_SEED); + } + + protected BloomFilterBase(int numHashFunctions, long numBits, int seed) { + this(new BitArray(numBits), numHashFunctions, seed); + } + + protected BloomFilterBase(BitArray bits, int numHashFunctions, int seed) { + this.bits = bits; + this.numHashFunctions = numHashFunctions; + this.seed = seed; + } + + protected BloomFilterBase() {} + + @Override + public boolean equals(Object other) { + if (other == this) { + return true; + } + + if (!(other instanceof BloomFilterBase that)) { + return false; + } + + return + this.getClass() == that.getClass() + && this.numHashFunctions == that.numHashFunctions + && this.seed == that.seed + // TODO: this.bits can be null temporarily, during deserialization, + // should we worry about this? + && this.bits.equals(that.bits); + } + + @Override + public int hashCode() { + return Objects.hash(numHashFunctions, seed, bits); + } + + @Override + public double expectedFpp() { + return Math.pow((double) bits.cardinality() / bits.bitSize(), numHashFunctions); + } + + @Override + public long bitSize() { + return bits.bitSize(); + } + + @Override + public boolean put(Object item) { + if (item instanceof String str) { + return putString(str); + } else if (item instanceof byte[] bytes) { + return putBinary(bytes); + } else { + return putLong(Utils.integralToLong(item)); + } + } + + @Override + public boolean putString(String item) { + return putBinary(Utils.getBytesFromUTF8String(item)); + } + + @Override + public abstract boolean putBinary(byte[] item); + + @Override + public boolean mightContainString(String item) { + return mightContainBinary(Utils.getBytesFromUTF8String(item)); + } + + @Override + public abstract boolean mightContainBinary(byte[] item) ; + + @Override + public abstract boolean putLong(long item); + + @Override + public abstract boolean mightContainLong(long item); + + @Override + public boolean mightContain(Object item) { + if (item instanceof String str) { + return mightContainString(str); + } else if (item instanceof byte[] bytes) { + return mightContainBinary(bytes); + } else { + return mightContainLong(Utils.integralToLong(item)); + } + } + + @Override + public boolean isCompatible(BloomFilter other) { + if (other == null) { + return false; + } + + if (!(other instanceof BloomFilterBase that)) { + return false; + } + + return + this.getClass() == that.getClass() + && this.bitSize() == that.bitSize() + && this.numHashFunctions == that.numHashFunctions + && this.seed == that.seed; + } + + @Override + public BloomFilter mergeInPlace(BloomFilter other) throws IncompatibleMergeException { + BloomFilterBase otherImplInstance = checkCompatibilityForMerge(other); + + this.bits.putAll(otherImplInstance.bits); + return this; + } + + @Override + public BloomFilter intersectInPlace(BloomFilter other) throws IncompatibleMergeException { + BloomFilterBase otherImplInstance = checkCompatibilityForMerge(other); + + this.bits.and(otherImplInstance.bits); + return this; + } + + @Override + public long cardinality() { + return this.bits.cardinality(); + } + + protected abstract BloomFilterBase checkCompatibilityForMerge(BloomFilter other) + throws IncompatibleMergeException; + +} diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java index ba569994cdf87..cc5e677df231b 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java @@ -19,67 +19,14 @@ import java.io.*; -class BloomFilterImpl extends BloomFilter implements Serializable { - - private int numHashFunctions; - - private BitArray bits; +class BloomFilterImpl extends BloomFilterBase implements Serializable { BloomFilterImpl(int numHashFunctions, long numBits) { - this(new BitArray(numBits), numHashFunctions); - } - - private BloomFilterImpl(BitArray bits, int numHashFunctions) { - this.bits = bits; - this.numHashFunctions = numHashFunctions; + super(numHashFunctions, numBits); } private BloomFilterImpl() {} - @Override - public boolean equals(Object other) { - if (other == this) { - return true; - } - - if (!(other instanceof BloomFilterImpl that)) { - return false; - } - - return this.numHashFunctions == that.numHashFunctions && this.bits.equals(that.bits); - } - - @Override - public int hashCode() { - return bits.hashCode() * 31 + numHashFunctions; - } - - @Override - public double expectedFpp() { - return Math.pow((double) bits.cardinality() / bits.bitSize(), numHashFunctions); - } - - @Override - public long bitSize() { - return bits.bitSize(); - } - - @Override - public boolean put(Object item) { - if (item instanceof String str) { - return putString(str); - } else if (item instanceof byte[] bytes) { - return putBinary(bytes); - } else { - return putLong(Utils.integralToLong(item)); - } - } - - @Override - public boolean putString(String item) { - return putBinary(Utils.getBytesFromUTF8String(item)); - } - @Override public boolean putBinary(byte[] item) { int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, 0); @@ -98,11 +45,6 @@ public boolean putBinary(byte[] item) { return bitsChanged; } - @Override - public boolean mightContainString(String item) { - return mightContainBinary(Utils.getBytesFromUTF8String(item)); - } - @Override public boolean mightContainBinary(byte[] item) { int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, 0); @@ -164,52 +106,7 @@ public boolean mightContainLong(long item) { return true; } - @Override - public boolean mightContain(Object item) { - if (item instanceof String str) { - return mightContainString(str); - } else if (item instanceof byte[] bytes) { - return mightContainBinary(bytes); - } else { - return mightContainLong(Utils.integralToLong(item)); - } - } - - @Override - public boolean isCompatible(BloomFilter other) { - if (other == null) { - return false; - } - - if (!(other instanceof BloomFilterImpl that)) { - return false; - } - - return this.bitSize() == that.bitSize() && this.numHashFunctions == that.numHashFunctions; - } - - @Override - public BloomFilter mergeInPlace(BloomFilter other) throws IncompatibleMergeException { - BloomFilterImpl otherImplInstance = checkCompatibilityForMerge(other); - - this.bits.putAll(otherImplInstance.bits); - return this; - } - - @Override - public BloomFilter intersectInPlace(BloomFilter other) throws IncompatibleMergeException { - BloomFilterImpl otherImplInstance = checkCompatibilityForMerge(other); - - this.bits.and(otherImplInstance.bits); - return this; - } - - @Override - public long cardinality() { - return this.bits.cardinality(); - } - - private BloomFilterImpl checkCompatibilityForMerge(BloomFilter other) + protected BloomFilterImpl checkCompatibilityForMerge(BloomFilter other) throws IncompatibleMergeException { // Duplicates the logic of `isCompatible` here to provide better error message. if (other == null) { @@ -240,6 +137,7 @@ public void writeTo(OutputStream out) throws IOException { dos.writeInt(Version.V1.getVersionNumber()); dos.writeInt(numHashFunctions); + // ignore seed bits.writeTo(dos); } @@ -252,6 +150,7 @@ private void readFrom0(InputStream in) throws IOException { } this.numHashFunctions = dis.readInt(); + this.seed = DEFAULT_SEED; this.bits = BitArray.readFrom(dis); } @@ -267,10 +166,12 @@ public static BloomFilter readFrom(byte[] bytes) throws IOException { return BloomFilter.readFrom(bytes); } + @Serial private void writeObject(ObjectOutputStream out) throws IOException { writeTo(out); } + @Serial private void readObject(ObjectInputStream in) throws IOException { readFrom0(in); } diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java index d637df862c31f..cceae87ab418a 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java @@ -18,29 +18,15 @@ package org.apache.spark.util.sketch; import java.io.*; -import java.util.Objects; -class BloomFilterImplV2 extends BloomFilter implements Serializable { - - public static final int DEFAULT_SEED = 0; - - private int seed; - private int numHashFunctions; - - private BitArray bits; - - BloomFilterImplV2(int numHashFunctions, long numBits) { - this(numHashFunctions, numBits, DEFAULT_SEED); - } +class BloomFilterImplV2 extends BloomFilterBase implements Serializable { BloomFilterImplV2(int numHashFunctions, long numBits, int seed) { this(new BitArray(numBits), numHashFunctions, seed); } private BloomFilterImplV2(BitArray bits, int numHashFunctions, int seed) { - this.bits = bits; - this.numHashFunctions = numHashFunctions; - this.seed = seed; + super(bits, numHashFunctions, seed); } private BloomFilterImplV2() {} @@ -61,37 +47,6 @@ public boolean equals(Object other) { && this.bits.equals(that.bits); } - @Override - public int hashCode() { - return Objects.hash(numHashFunctions, seed, bits); - } - - @Override - public double expectedFpp() { - return Math.pow((double) bits.cardinality() / bits.bitSize(), numHashFunctions); - } - - @Override - public long bitSize() { - return bits.bitSize(); - } - - @Override - public boolean put(Object item) { - if (item instanceof String str) { - return putString(str); - } else if (item instanceof byte[] bytes) { - return putBinary(bytes); - } else { - return putLong(Utils.integralToLong(item)); - } - } - - @Override - public boolean putString(String item) { - return putBinary(Utils.getBytesFromUTF8String(item)); - } - @Override public boolean putBinary(byte[] item) { int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, seed); @@ -113,11 +68,6 @@ public boolean putBinary(byte[] item) { return bitsChanged; } - @Override - public boolean mightContainString(String item) { - return mightContainBinary(Utils.getBytesFromUTF8String(item)); - } - @Override public boolean mightContainBinary(byte[] item) { int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, seed); @@ -188,55 +138,7 @@ public boolean mightContainLong(long item) { return true; } - @Override - public boolean mightContain(Object item) { - if (item instanceof String str) { - return mightContainString(str); - } else if (item instanceof byte[] bytes) { - return mightContainBinary(bytes); - } else { - return mightContainLong(Utils.integralToLong(item)); - } - } - - @Override - public boolean isCompatible(BloomFilter other) { - if (other == null) { - return false; - } - - if (!(other instanceof BloomFilterImplV2 that)) { - return false; - } - - return - this.bitSize() == that.bitSize() - && this.numHashFunctions == that.numHashFunctions - && this.seed == that.seed; - } - - @Override - public BloomFilter mergeInPlace(BloomFilter other) throws IncompatibleMergeException { - BloomFilterImplV2 otherImplInstance = checkCompatibilityForMerge(other); - - this.bits.putAll(otherImplInstance.bits); - return this; - } - - @Override - public BloomFilter intersectInPlace(BloomFilter other) throws IncompatibleMergeException { - BloomFilterImplV2 otherImplInstance = checkCompatibilityForMerge(other); - - this.bits.and(otherImplInstance.bits); - return this; - } - - @Override - public long cardinality() { - return this.bits.cardinality(); - } - - private BloomFilterImplV2 checkCompatibilityForMerge(BloomFilter other) + protected BloomFilterImplV2 checkCompatibilityForMerge(BloomFilter other) throws IncompatibleMergeException { // Duplicates the logic of `isCompatible` here to provide better error message. if (other == null) { @@ -296,10 +198,12 @@ public static BloomFilterImplV2 readFrom(InputStream in) throws IOException { return filter; } + @Serial private void writeObject(ObjectOutputStream out) throws IOException { writeTo(out); } + @Serial private void readObject(ObjectInputStream in) throws IOException { readFrom0(in); } From f2df338faa77e9d1ddbd6bbd04f12c22bb72ceeb Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Fri, 11 Jul 2025 16:39:10 +0200 Subject: [PATCH 34/44] SPARK-47547 BloomFilter fpp degradation: pull up long and byte hashing to abstract base to make them extensible --- .../spark/util/sketch/BloomFilterBase.java | 19 ++++++++++++++ .../spark/util/sketch/BloomFilterImpl.java | 25 +++++++++---------- .../spark/util/sketch/BloomFilterImplV2.java | 25 +++++++++---------- 3 files changed, 43 insertions(+), 26 deletions(-) diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterBase.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterBase.java index fd12666d48997..6e9804be9ec79 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterBase.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterBase.java @@ -162,4 +162,23 @@ public long cardinality() { protected abstract BloomFilterBase checkCompatibilityForMerge(BloomFilter other) throws IncompatibleMergeException; + public record HiLoHash (int hi, int lo) {} + + protected HiLoHash hashLongToIntPair(long item, int seed) { + // Here we first hash the input long element into 2 int hash values, h1 and h2, then produce n + // hash values by `h1 + i * h2` with 1 <= i <= numHashFunctions. + // Note that `CountMinSketch` use a different strategy, it hash the input long element with + // every i to produce n hash values. + // TODO: the strategy of `CountMinSketch` looks more advanced, should we follow it here? + int h1 = Murmur3_x86_32.hashLong(item, seed); + int h2 = Murmur3_x86_32.hashLong(item, h1); + return new HiLoHash(h1, h2); + } + + protected HiLoHash hashBytesToIntPair(byte[] item, int seed) { + int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, seed); + int h2 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, h1); + return new HiLoHash(h1, h2); + } + } diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java index cc5e677df231b..dea810f9e9279 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImpl.java @@ -29,8 +29,9 @@ private BloomFilterImpl() {} @Override public boolean putBinary(byte[] item) { - int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, 0); - int h2 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, h1); + HiLoHash hiLoHash = hashBytesToIntPair(item, seed); + int h1 = hiLoHash.hi(); + int h2 = hiLoHash.lo(); long bitSize = bits.bitSize(); boolean bitsChanged = false; @@ -47,8 +48,9 @@ public boolean putBinary(byte[] item) { @Override public boolean mightContainBinary(byte[] item) { - int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, 0); - int h2 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, h1); + HiLoHash hiLoHash = hashBytesToIntPair(item, seed); + int h1 = hiLoHash.hi(); + int h2 = hiLoHash.lo(); long bitSize = bits.bitSize(); for (int i = 1; i <= numHashFunctions; i++) { @@ -66,13 +68,9 @@ public boolean mightContainBinary(byte[] item) { @Override public boolean putLong(long item) { - // Here we first hash the input long element into 2 int hash values, h1 and h2, then produce n - // hash values by `h1 + i * h2` with 1 <= i <= numHashFunctions. - // Note that `CountMinSketch` use a different strategy, it hash the input long element with - // every i to produce n hash values. - // TODO: the strategy of `CountMinSketch` looks more advanced, should we follow it here? - int h1 = Murmur3_x86_32.hashLong(item, 0); - int h2 = Murmur3_x86_32.hashLong(item, h1); + HiLoHash hiLoHash = hashLongToIntPair(item, seed); + int h1 = hiLoHash.hi(); + int h2 = hiLoHash.lo(); long bitSize = bits.bitSize(); boolean bitsChanged = false; @@ -89,8 +87,9 @@ public boolean putLong(long item) { @Override public boolean mightContainLong(long item) { - int h1 = Murmur3_x86_32.hashLong(item, 0); - int h2 = Murmur3_x86_32.hashLong(item, h1); + HiLoHash hiLoHash = hashLongToIntPair(item, seed); + int h1 = hiLoHash.hi(); + int h2 = hiLoHash.lo(); long bitSize = bits.bitSize(); for (int i = 1; i <= numHashFunctions; i++) { diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java index cceae87ab418a..6dcd3310d42b7 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterImplV2.java @@ -49,8 +49,9 @@ public boolean equals(Object other) { @Override public boolean putBinary(byte[] item) { - int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, seed); - int h2 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, h1); + HiLoHash hiLoHash = hashBytesToIntPair(item, seed); + int h1 = hiLoHash.hi(); + int h2 = hiLoHash.lo(); long bitSize = bits.bitSize(); boolean bitsChanged = false; @@ -70,8 +71,9 @@ public boolean putBinary(byte[] item) { @Override public boolean mightContainBinary(byte[] item) { - int h1 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, seed); - int h2 = Murmur3_x86_32.hashUnsafeBytes(item, Platform.BYTE_ARRAY_OFFSET, item.length, h1); + HiLoHash hiLoHash = hashBytesToIntPair(item, seed); + int h1 = hiLoHash.hi(); + int h2 = hiLoHash.lo(); long bitSize = bits.bitSize(); @@ -92,13 +94,9 @@ public boolean mightContainBinary(byte[] item) { @Override public boolean putLong(long item) { - // Here we first hash the input long element into 2 int hash values, h1 and h2, then produce n - // hash values by `h1 + i * h2` with 1 <= i <= numHashFunctions. - // Note that `CountMinSketch` use a different strategy, it hash the input long element with - // every i to produce n hash values. - // TODO: the strategy of `CountMinSketch` looks more advanced, should we follow it here? - int h1 = Murmur3_x86_32.hashLong(item, seed); - int h2 = Murmur3_x86_32.hashLong(item, h1); + HiLoHash hiLoHash = hashLongToIntPair(item, seed); + int h1 = hiLoHash.hi(); + int h2 = hiLoHash.lo(); long bitSize = bits.bitSize(); boolean bitsChanged = false; @@ -118,8 +116,9 @@ public boolean putLong(long item) { @Override public boolean mightContainLong(long item) { - int h1 = Murmur3_x86_32.hashLong(item, seed); - int h2 = Murmur3_x86_32.hashLong(item, h1); + HiLoHash hiLoHash = hashLongToIntPair(item, seed); + int h1 = hiLoHash.hi(); + int h2 = hiLoHash.lo(); long bitSize = bits.bitSize(); From 4aaff8378dbc1ce9aff5530c972333d9396d8083 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Sun, 13 Jul 2025 10:20:23 +0200 Subject: [PATCH 35/44] SPARK-47547 BloomFilter fpp degradation: checkstyle --- .../main/java/org/apache/spark/util/sketch/BloomFilterBase.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterBase.java b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterBase.java index 6e9804be9ec79..b401d2d8f18e7 100644 --- a/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterBase.java +++ b/common/sketch/src/main/java/org/apache/spark/util/sketch/BloomFilterBase.java @@ -162,7 +162,7 @@ public long cardinality() { protected abstract BloomFilterBase checkCompatibilityForMerge(BloomFilter other) throws IncompatibleMergeException; - public record HiLoHash (int hi, int lo) {} + public record HiLoHash(int hi, int lo) {} protected HiLoHash hashLongToIntPair(long item, int seed) { // Here we first hash the input long element into 2 int hash values, h1 and h2, then produce n From e214bd700fe1e0792f8a4f9d3c3fb48115c5edeb Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Tue, 15 Jul 2025 12:54:42 +0200 Subject: [PATCH 36/44] SPARK-47547 BloomFilter fpp degradation: removing unnecessary line wrappings --- .../org/apache/spark/util/sketch/BloomFilterSuite.scala | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala b/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala index 26963dbb291e9..ba8f97a51aecf 100644 --- a/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala +++ b/common/sketch/src/test/scala/org/apache/spark/util/sketch/BloomFilterSuite.scala @@ -72,9 +72,7 @@ class BloomFilterSuite extends AnyFunSuite { // scalastyle:ignore funsuite } } - def testMergeInPlace[T: ClassTag]( - typeName: String, - numItems: Int)(itemGen: Random => T): Unit = { + def testMergeInPlace[T: ClassTag](typeName: String, numItems: Int)(itemGen: Random => T): Unit = { test(s"mergeInPlace - $typeName") { // use a fixed seed to make the test predictable. val r = new Random(37) @@ -101,8 +99,8 @@ class BloomFilterSuite extends AnyFunSuite { // scalastyle:ignore funsuite } } - def testIntersectInPlace[T: ClassTag](typeName: String, numItems: Int)( - itemGen: Random => T): Unit = { + def testIntersectInPlace[T: ClassTag] + (typeName: String, numItems: Int)(itemGen: Random => T): Unit = { test(s"intersectInPlace - $typeName") { // use a fixed seed to make the test predictable. val r = new Random(37) From 99f73438a3ebabf3b99bbffe55aed5cc4039a5c2 Mon Sep 17 00:00:00 2001 From: Ish Nagy Date: Tue, 15 Jul 2025 13:13:44 +0200 Subject: [PATCH 37/44] SPARK-47547 BloomFilter fpp degradation: moving junit-pioneer version under the main pom's dependencyManagement --- common/sketch/pom.xml | 1 - pom.xml | 7 +++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index fdc5a9720687d..6c1156f9600b0 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -54,7 +54,6 @@ org.junit-pioneer junit-pioneer - 2.3.0 test diff --git a/pom.xml b/pom.xml index 1037eaf2863e5..78cceef9d4596 100644 --- a/pom.xml +++ b/pom.xml @@ -220,6 +220,7 @@ 77.1 5.13.1 1.13.1 + 2.3.0