From 41a542f76221ad676dafe75413c309f49fcf8da7 Mon Sep 17 00:00:00 2001 From: summerleafs Date: Wed, 7 Feb 2018 00:47:25 +0800 Subject: [PATCH 1/5] introduce bloom filter state. --- .../PartitionedBloomFilterDescriptor.java | 164 ++++++++++++ .../bloomfilter/BloomFilterExample.java | 118 ++++++++ .../api/operators/AbstractStreamOperator.java | 18 ++ .../api/operators/LinkedBloomFilter.java | 172 ++++++++++++ .../api/operators/LinkedBloomFilterNode.java | 180 +++++++++++++ .../api/operators/PartitionedBloomFilter.java | 251 ++++++++++++++++++ .../PartitionedBloomFilterManager.java | 154 +++++++++++ .../operators/StreamOperatorStateContext.java | 2 + .../StreamTaskStateInitializerImpl.java | 46 ++++ .../operators/StreamingRuntimeContext.java | 6 + .../operators/LinkedBloomFilterNodeTest.java | 59 ++++ .../api/operators/LinkedBloomFilterTest.java | 131 +++++++++ .../PartitionedBloomFilterManagerTest.java | 146 ++++++++++ .../operators/PartitionedBloomFilterTest.java | 145 ++++++++++ .../runtime/tasks/StreamTaskTest.java | 6 + .../flink-test-utils/pom.xml | 6 + 16 files changed, 1604 insertions(+) create mode 100644 flink-core/src/main/java/org/apache/flink/api/common/state/PartitionedBloomFilterDescriptor.java create mode 100644 flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/bloomfilter/BloomFilterExample.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedBloomFilter.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNode.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilter.java create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterManager.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNodeTest.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterTest.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterManagerTest.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterTest.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/PartitionedBloomFilterDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/PartitionedBloomFilterDescriptor.java new file mode 100644 index 0000000000000..b40c2fa7348ed --- /dev/null +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/PartitionedBloomFilterDescriptor.java @@ -0,0 +1,164 @@ +package org.apache.flink.api.common.state; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.common.typeutils.TypeSerializer; + +import java.io.Serializable; + +/** + * Created by zsh on 06/02/2018. + */ +public class PartitionedBloomFilterDescriptor implements Serializable { + + private static final long serialVersionUID = 1L; + + private static final long DEFAULT_MAX_EXPECT_NUM = 128_000_000; + private static final long DEFAULT_MIN_EXPECT_NUMBER = 1000_000; + private static final double DEFAULT_GROW_RATE = 2.0; + + private String stateName; + private long capacity; + private long ttl; + + // false positive probability + private double fpp; + + private long miniExpectNum; + private long maxExpectNum; + private double growRate; + + /** The type information describing the value type. Only used to lazily create the serializer + * and dropped during serialization */ + private transient TypeInformation typeInfo; + + /** The serializer for the type. May be eagerly initialized in the constructor, + * or lazily once the type is serialized or an ExecutionConfig is provided. */ + protected TypeSerializer serializer; + + public PartitionedBloomFilterDescriptor(String stateName, TypeInformation typeInfo, long capacity, double fpp) { + this( + stateName, + typeInfo, + capacity, + fpp, + Integer.MAX_VALUE); + } + + public PartitionedBloomFilterDescriptor(String stateName, TypeInformation typeInfo, long capacity, double fpp, long ttl) { + this( + stateName, + typeInfo, + capacity, + fpp, + ttl, + DEFAULT_MIN_EXPECT_NUMBER, + DEFAULT_MAX_EXPECT_NUM, + DEFAULT_GROW_RATE); + } + + public PartitionedBloomFilterDescriptor(String stateName, TypeInformation typeInfo, long capacity, double fpp, long ttl, long minExpectNum, long maxExpectNum, double growRate) { + this.stateName = stateName; + this.typeInfo = typeInfo; + this.capacity = capacity; + this.fpp = fpp; + this.ttl = ttl; + this.miniExpectNum = minExpectNum; + this.maxExpectNum = maxExpectNum; + this.growRate = growRate; + } + + public PartitionedBloomFilterDescriptor(String stateName, TypeSerializer serializer, long capacity, double fpp, long ttl) { + this( + stateName, + serializer, + capacity, + fpp, + ttl, + DEFAULT_MIN_EXPECT_NUMBER, + DEFAULT_MAX_EXPECT_NUM, + DEFAULT_GROW_RATE); + } + + public PartitionedBloomFilterDescriptor(String stateName, TypeSerializer serializer, long capacity, double fpp, long ttl, long minExpectNum, long maxExpectNum, double growRate) { + this.stateName = stateName; + this.serializer = serializer; + this.capacity = capacity; + this.fpp = fpp; + this.ttl = ttl; + this.miniExpectNum = minExpectNum; + this.maxExpectNum = maxExpectNum; + this.growRate = growRate; + } + + public long getTtl() { + return ttl; + } + + public double getFpp() { + return fpp; + } + + public String getStateName() { + return stateName; + } + + public long getCapacity() { + return capacity; + } + + public long getMiniExpectNum() { + return miniExpectNum; + } + + public long getMaxExpectNum() { + return maxExpectNum; + } + + public double getGrowRate() { + return growRate; + } + +// public void snapshot(DataOutputViewStreamWrapper out) throws IOException { +// out.writeUTF(stateName); +// out.writeLong(capacity); +// out.writeLong(ttl); +// out.writeDouble(fpp); +// +// out.writeLong(miniExpectNum); +// out.writeLong(maxExpectNum); +// out.writeDouble(growRate); +// } +// +// public static BloomFilterStateDescriptor restore(DataInputViewStreamWrapper in) throws IOException { +// BloomFilterStateDescriptor desc = new BloomFilterStateDescriptor(); +// desc.stateName = in.readUTF(); +// desc.capacity = in.readLong(); +// desc.ttl = in.readLong(); +// desc.fpp = in.readDouble(); +// +// desc.miniExpectNum = in.readLong(); +// desc.maxExpectNum = in.readLong(); +// desc.growRate = in.readDouble(); +// return desc; +// } + + public void initializeSerializerUnlessSet(ExecutionConfig executionConfig) { + if (serializer == null) { + if (typeInfo != null) { + serializer = typeInfo.createSerializer(executionConfig); + } else { + throw new IllegalStateException( + "Cannot initialize serializer after TypeInformation was dropped during serialization"); + } + } + } + + public TypeSerializer getSerializer() { + if (serializer != null) { + return serializer.duplicate(); + } else { + throw new IllegalStateException("Serializer not yet initialized."); + } + } +} diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/bloomfilter/BloomFilterExample.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/bloomfilter/BloomFilterExample.java new file mode 100644 index 0000000000000..2a2cfb57fcf2d --- /dev/null +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/bloomfilter/BloomFilterExample.java @@ -0,0 +1,118 @@ +package org.apache.flink.streaming.examples.bloomfilter; + +import org.apache.flink.api.common.state.PartitionedBloomFilterDescriptor; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.api.java.functions.KeySelector; +import org.apache.flink.configuration.Configuration; +import org.apache.flink.runtime.state.memory.MemoryStateBackend; +import org.apache.flink.streaming.api.CheckpointingMode; +import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; +import org.apache.flink.streaming.api.functions.source.SourceFunction; +import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction; +import org.apache.flink.streaming.api.operators.PartitionedBloomFilter; +import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; +import org.apache.flink.streaming.api.windowing.time.Time; +import org.apache.flink.streaming.api.windowing.windows.TimeWindow; +import org.apache.flink.util.Collector; + +import java.util.Random; + +/** + * Created by zsh on 11/02/2018. + */ +public class BloomFilterExample { + + private static class SimpleSource implements SourceFunction { + private static final long serialVersionUID = 1L; + + private volatile boolean isRunning = true; + + private long ts = -1L; + + public SimpleSource() { + } + + @Override + public void run(SourceContext ctx) throws Exception { + if (ts == -1L) { + ts = System.currentTimeMillis(); + } + while (isRunning) { + int num = new Random().nextInt(1000); + ctx.collect(num); + Thread.sleep(1000L); + + if (System.currentTimeMillis() - ts >= 30000) { + throw new RuntimeException("trigger failover."); + } + } + } + + @Override + public void cancel() { + isRunning = false; + } + } + + public static void main(String[] args) throws Exception { + + // obtain execution environment + StreamExecutionEnvironment env = StreamExecutionEnvironment.getExecutionEnvironment(); + + env.setStateBackend(new MemoryStateBackend()); + env.enableCheckpointing(15000L, CheckpointingMode.EXACTLY_ONCE); + env.setParallelism(1); + env.setMaxParallelism(8); + + // create input stream of an single integer + env.addSource(new SimpleSource()) + .keyBy(new KeySelector() { + @Override + public Integer getKey(Integer value) throws Exception { + return value; + } + }) + .timeWindow(Time.milliseconds(1000)) + .apply(new RichWindowFunction() { + private PartitionedBloomFilter bf1; + private PartitionedBloomFilter bf2; + + @Override + public void open(Configuration parameters) throws Exception { + super.open(parameters); + StreamingRuntimeContext runtimeContext = (StreamingRuntimeContext) this.getRuntimeContext(); + bf1 = runtimeContext.getPartitionedBloomFilter(new PartitionedBloomFilterDescriptor( + "0.01", + TypeInformation.of(String.class), + 60000, + 0.01)); + + bf2 = runtimeContext.getPartitionedBloomFilter(new PartitionedBloomFilterDescriptor( + "0.02", + TypeInformation.of(String.class), + 60000, + 0.01)); + } + + @Override + public void apply(Integer integer, TimeWindow window, Iterable input, Collector out) throws Exception { + for (Integer ele : input) { + if (!bf1.contains(String.valueOf(ele))) { + System.out.println("write:" + ele); + bf2.add(String.valueOf(ele)); + out.collect(ele); + } + + if (!bf2.contains(String.valueOf(ele) + "a")) { + System.out.println("write:" + ele); + bf2.add(String.valueOf(ele) + "a"); + out.collect(ele); + } + } + } + }).print(); + + // execute the program + env.execute("BloomFilter Example"); + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java index 9915dd518168a..fd824771f574f 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java @@ -153,6 +153,11 @@ public abstract class AbstractStreamOperator protected transient InternalTimeServiceManager timeServiceManager; + // ---------------- bloom filter handler ------------------ + + /** Backend for bloomfilter. */ + private PartitionedBloomFilterManager bloomFilterManager; + // ---------------- two-input operator watermarks ------------------ // We keep track of watermarks from both inputs, the combined input is the minimum @@ -240,6 +245,8 @@ public final void initializeState() throws Exception { timeServiceManager = context.internalTimerServiceManager(); + bloomFilterManager = context.bloomFilterStateManager(); + CloseableIterable keyedStateInputs = context.rawKeyedStateInputs(); CloseableIterable operatorStateInputs = context.rawOperatorStateInputs(); @@ -341,6 +348,10 @@ public void dispose() throws Exception { exception = ExceptionUtils.firstOrSuppressed(e, exception); } + if (bloomFilterManager != null) { + bloomFilterManager.dispose(); + } + if (exception != null) { throw exception; } @@ -419,6 +430,9 @@ public void snapshotState(StateSnapshotContext context) throws Exception { timeServiceManager.snapshotStateForKeyGroup( new DataOutputViewStreamWrapper(out), keyGroupIdx); + + bloomFilterManager.snapshotStateForKeyGroup( + new DataOutputViewStreamWrapper(out), keyGroupIdx); } } catch (Exception exception) { throw new Exception("Could not write timer service of " + getOperatorName() + @@ -614,6 +628,10 @@ public KeyedStateStore getKeyedStateStore() { return keyedStateStore; } + protected PartitionedBloomFilterManager getBloomFilterStateManager() { + return bloomFilterManager; + } + // ------------------------------------------------------------------------ // Context and chaining properties // ------------------------------------------------------------------------ diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedBloomFilter.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedBloomFilter.java new file mode 100644 index 0000000000000..ac2fce7e0ada3 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedBloomFilter.java @@ -0,0 +1,172 @@ +/* + * 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.flink.streaming.api.operators; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Iterator; +import java.util.LinkedList; + +/** + * A list of {@link LinkedBloomFilterNode} to avoid data skewed between key ranges. The size of nodes on the list + * grow by a {@code growRate} to avoid the list to be too long. + */ +public class LinkedBloomFilter { + + private static final Logger LOG = LoggerFactory.getLogger(LinkedBloomFilter.class); + + private long currentSize; + + private long initSize; + private double growRate; + + private PartitionedBloomFilter partitionedBloomFilter; + + private LinkedList bloomFilterNodes = new LinkedList<>(); + + public LinkedBloomFilter(PartitionedBloomFilter partitionedBloomFilter, long initSize, double growRate) { + this.partitionedBloomFilter = partitionedBloomFilter; + this.currentSize = initSize; + this.initSize = initSize; + this.growRate = growRate; + } + + public void add(byte[] content) { + synchronized (bloomFilterNodes) { + LinkedBloomFilterNode node; + if (bloomFilterNodes.size() > 0) { + node = bloomFilterNodes.getLast(); + if (node.isFull()) { + LOG.info("allocate new node."); + currentSize = (long) (this.initSize * Math.pow(growRate, bloomFilterNodes.size())); + node = this.partitionedBloomFilter.allocateBloomFilterNode(currentSize); + if (node != null) { + LOG.info("allocate new node successfully."); + bloomFilterNodes.add(node); + } else { + LOG.warn("allocate new node failed (run out of configured capacity), reuse the last node."); + node = bloomFilterNodes.getLast(); + node.reSetTtl(); + } + } + } else { + LOG.info("init the first node."); + node = this.partitionedBloomFilter.allocateBloomFilterNode(currentSize, true); + bloomFilterNodes.add(node); + } + node.add(content); + } + } + + public boolean contains(byte[] content) { + synchronized (bloomFilterNodes) { + Iterator iter = bloomFilterNodes.descendingIterator(); + while (iter.hasNext()) { + LinkedBloomFilterNode node = iter.next(); + if (node.contains(content)) { + return true; + } + } + return false; + } + } + + // for checkpoint and recovery + public LinkedBloomFilter copy() { + synchronized (bloomFilterNodes) { + LinkedBloomFilter bloomFilter = new LinkedBloomFilter(partitionedBloomFilter, initSize, growRate); + for (LinkedBloomFilterNode node : bloomFilterNodes) { + bloomFilter.bloomFilterNodes.add(node.copy()); + } + return bloomFilter; + } + } + + @VisibleForTesting + long getCurrentSize() { + return currentSize; + } + + @VisibleForTesting + long getInitSize() { + return initSize; + } + + @VisibleForTesting + double getGrowRate() { + return growRate; + } + + @VisibleForTesting + LinkedList getBloomFilterNodes() { + return bloomFilterNodes; + } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder(); + synchronized (bloomFilterNodes) { + for (LinkedBloomFilterNode node : bloomFilterNodes) { + if (builder.length() > 0) { + builder.append(" -> "); + } + builder.append(node.toString()); + } + } + return builder.toString(); + } + + void snapshot(DataOutputView outputView) throws IOException { + long ts = System.currentTimeMillis(); + outputView.writeLong(currentSize); + outputView.writeLong(initSize); + outputView.writeDouble(growRate); + + bloomFilterNodes.removeIf(node -> { + if (node.getDeleteTS() <= ts) { + partitionedBloomFilter.takeBack(node); + return true; + } + return false; + }); + + outputView.writeInt(bloomFilterNodes.size()); + for (LinkedBloomFilterNode node : bloomFilterNodes) { + node.snapshot(outputView); + } + } + + void restore(DataInputView source) throws IOException { + currentSize = source.readLong(); + initSize = source.readLong(); + growRate = source.readDouble(); + int len = source.readInt(); + for (int i = 0; i < len; ++i) { + LinkedBloomFilterNode node = new LinkedBloomFilterNode(); + node.restore(source); + bloomFilterNodes.add(node); + } + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNode.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNode.java new file mode 100644 index 0000000000000..bf7a05b5870fc --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNode.java @@ -0,0 +1,180 @@ +/* + * 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.flink.streaming.api.operators; + +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import org.apache.flink.shaded.guava18.com.google.common.hash.BloomFilter; +import org.apache.flink.shaded.guava18.com.google.common.hash.Funnels; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; + +/** + * A bloom filter node linked each other in {@link LinkedBloomFilter} to avoid data skewed. + */ +public class LinkedBloomFilterNode { + + private static final Logger LOG = LoggerFactory.getLogger(LinkedBloomFilterNode.class); + + private long capacity; + private double fpp; + private long size; + private long rawTtl; + private long ttl; + private long deleteTS = Long.MAX_VALUE; + + private BloomFilter bloomFilter; + + LinkedBloomFilterNode() { + + } + + public LinkedBloomFilterNode(long capacity, double fpp, long ttl) { + this.capacity = capacity; + this.fpp = fpp; + this.rawTtl = ttl; + this.ttl = ttl; + this.size = 0; + + LOG.info("create bf capacity:{} fpp:{}", capacity, fpp); + bloomFilter = BloomFilter.create( + Funnels.byteArrayFunnel(), + (int) capacity, + this.fpp); + } + + public boolean isFull() { + return size >= capacity; + } + + public void add(byte[] content) { + bloomFilter.put(content); + + this.size++; + if (size >= capacity) { + if (deleteTS == Long.MAX_VALUE) { + long ts = System.currentTimeMillis(); + deleteTS = ts + ttl; + } + } + } + + public boolean contains(byte[] content) { + return bloomFilter.mightContain(content); + } + + public long getCapacity() { + return capacity; + } + + public double getFpp() { + return fpp; + } + + public long getSize() { + return size; + } + + public long getTtl() { + return ttl; + } + + public void reSetTtl() { + this.ttl = this.rawTtl; + this.deleteTS = Long.MAX_VALUE; + } + + public long getDeleteTS() { + return deleteTS; + } + + public BloomFilter getBloomFilter() { + return bloomFilter; + } + + public LinkedBloomFilterNode copy() { + LinkedBloomFilterNode node = new LinkedBloomFilterNode(capacity, fpp, ttl); + node.size = this.size; + node.bloomFilter = this.bloomFilter.copy(); + return node; + } + + @Override + public String toString() { + return String.format("{c:%d s:%d}", capacity, size); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof LinkedBloomFilterNode) { + LinkedBloomFilterNode other = (LinkedBloomFilterNode) obj; + if (other.capacity == this.capacity + && other.size == this.size + && other.rawTtl == this.rawTtl + && other.fpp == this.fpp + && other.bloomFilter.equals(this.bloomFilter)) { + return true; + } + return false; + } + return false; + } + + void snapshot(DataOutputView outputView) throws IOException { + outputView.writeLong(capacity); + outputView.writeLong(rawTtl); + if (deleteTS == Long.MAX_VALUE) { + outputView.writeLong(ttl); //rest ttl + } else { + outputView.writeLong(deleteTS - System.currentTimeMillis()); //rest ttl + } + outputView.writeLong(deleteTS); + outputView.writeLong(size); + outputView.writeDouble(fpp); + ByteArrayOutputStream out = new ByteArrayOutputStream(); + bloomFilter.writeTo(out); + byte[] bytes = out.toByteArray(); + outputView.writeInt(bytes.length); + outputView.write(bytes); + } + + void restore(DataInputView source) throws IOException { + capacity = source.readLong(); + rawTtl = source.readLong(); + ttl = source.readLong(); + deleteTS = source.readLong(); + if (rawTtl != ttl) { + deleteTS = System.currentTimeMillis() + ttl; + } + size = source.readLong(); + fpp = source.readDouble(); + + int byteLen = source.readInt(); + byte[] bytes = new byte[byteLen]; + source.read(bytes, 0, byteLen); + ByteArrayInputStream input = new ByteArrayInputStream(bytes); + bloomFilter = BloomFilter.readFrom(input, Funnels.byteArrayFunnel()); + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilter.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilter.java new file mode 100644 index 0000000000000..d67e43bcdbe30 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilter.java @@ -0,0 +1,251 @@ +/* + * 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.flink.streaming.api.operators; + +import org.apache.flink.annotation.VisibleForTesting; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.ByteArrayOutputStreamWithPos; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.runtime.operators.util.BloomFilter; +import org.apache.flink.runtime.state.KeyGroupRangeAssignment; +import org.apache.flink.runtime.state.KeyGroupsList; +import org.apache.flink.util.FlinkRuntimeException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +import static org.apache.flink.util.Preconditions.checkArgument; + +/** + * Bloom filter which supports + * - ability to handle data skew. + * - rescaling. + * - fail tolerant. + * - relax ttl. + * + * @param The type of keys. + * @param The type of values. + */ +public class PartitionedBloomFilter { + + private static final Logger LOG = LoggerFactory.getLogger(PartitionedBloomFilter.class); + + private final int totalKeyGroups; + private final KeyGroupsList localKeyGroupRange; + private final KeyContext keyContext; + private LinkedBloomFilter[] linkedBloomFilters; + private final int localKeyGroupRangeStartIdx; + + private long totalMemSize; + private long restMemSize; + + private long miniExpectNum; + private long maxExpectNum; + private double growRate; + + private long capacity; + private double fpp; + private long ttl; + + /** + * Serializer for the value. + */ + private final TypeSerializer keySerializer; + private final TypeSerializer valueSerializer; + private final ByteArrayOutputStreamWithPos valueSerializationStream; + private final DataOutputView valueSerializationDataOutputView; + + public PartitionedBloomFilter(TypeSerializer keySerializer, + TypeSerializer valueSerializer, + int totalKeyGroups, + KeyGroupsList localKeyGroupRange, + KeyContext keyContext, + long capacity, + double fpp, + long ttl, + long miniExpectNum, + long maxExpectNum, + double growRate) { + this.capacity = capacity; + this.fpp = fpp; + this.ttl = ttl; + this.totalMemSize = BloomFilter.optimalNumOfBits(capacity, fpp); + this.restMemSize = this.totalMemSize; + + this.miniExpectNum = miniExpectNum; + this.maxExpectNum = maxExpectNum; + this.growRate = growRate; + + this.keyContext = keyContext; + this.totalKeyGroups = totalKeyGroups; + this.localKeyGroupRange = localKeyGroupRange; + this.linkedBloomFilters = new LinkedBloomFilter[localKeyGroupRange.getNumberOfKeyGroups()]; + + // find the starting index of the local key-group range + int startIdx = Integer.MAX_VALUE; + for (Integer keyGroupIdx : localKeyGroupRange) { + startIdx = Math.min(keyGroupIdx, startIdx); + } + this.localKeyGroupRangeStartIdx = startIdx; + + this.keySerializer = keySerializer; + this.valueSerializer = valueSerializer; + this.valueSerializationStream = new ByteArrayOutputStreamWithPos(128); + this.valueSerializationDataOutputView = new DataOutputViewStreamWrapper(valueSerializationStream); + } + + public void add(V content) { + int keyGroupIndex = KeyGroupRangeAssignment.assignToKeyGroup(keyContext.getCurrentKey(), totalKeyGroups); + int index = getIndexForKeyGroup(keyGroupIndex); + + LinkedBloomFilter bloomFilter = linkedBloomFilters[index]; + if (bloomFilter == null) { + bloomFilter = new LinkedBloomFilter(this, miniExpectNum, growRate); + linkedBloomFilters[index] = bloomFilter; + } + bloomFilter.add(buildBloomFilterKey(content)); + } + + public boolean contains(V content) { + int keyGroupIndex = KeyGroupRangeAssignment.assignToKeyGroup(keyContext.getCurrentKey(), totalKeyGroups); + int index = getIndexForKeyGroup(keyGroupIndex); + + LinkedBloomFilter bloomFilter = linkedBloomFilters[index]; + if (bloomFilter == null) { + return false; + } + return bloomFilter.contains(buildBloomFilterKey(content)); + } + + private int getIndexForKeyGroup(int keyGroupIdx) { + checkArgument(localKeyGroupRange.contains(keyGroupIdx), + "Key Group " + keyGroupIdx + " does not belong to the local range."); + return keyGroupIdx - this.localKeyGroupRangeStartIdx; + } + + public void snapshotStateForKeyGroup(DataOutputViewStreamWrapper stream, int keyGroupIdx) throws IOException { + + LOG.info("snapshot state for group {} ", keyGroupIdx); + int index = getIndexForKeyGroup(keyGroupIdx); + LinkedBloomFilter bloomFilter = this.linkedBloomFilters[index]; + if (bloomFilter != null) { + stream.writeBoolean(true); + stream.writeLong(this.restMemSize); + bloomFilter.snapshot(stream); + } else { + stream.writeBoolean(false); + } + } + + public void restoreStateForKeyGroup( + DataInputViewStreamWrapper stream, + int keyGroupIdx) throws IOException, ClassNotFoundException { + + LOG.info("restore state for group {} ", keyGroupIdx); + int index = getIndexForKeyGroup(keyGroupIdx); + if (stream.readBoolean()) { + this.restMemSize = stream.readLong(); + LinkedBloomFilter linkedBloomFilter = new LinkedBloomFilter(this, miniExpectNum, growRate); + linkedBloomFilter.restore(stream); + this.linkedBloomFilters[index] = linkedBloomFilter; + LOG.info("group {} restored.", keyGroupIdx); + } else { + LOG.info("nothing to restore."); + } + } + + // --------------------- + + LinkedBloomFilterNode allocateBloomFilterNode(long expectNum) { + return allocateBloomFilterNode(expectNum, false); + } + + LinkedBloomFilterNode allocateBloomFilterNode(long expectNum, boolean force) { + long requestNum = expectNum; + if (!force) { + if (restMemSize > 0) { + requestNum = estimatePropExpectNum(expectNum, fpp); + } else { + return null; + } + } + + if (requestNum < miniExpectNum) { + requestNum = miniExpectNum; + } + + if (requestNum > maxExpectNum) { + requestNum = maxExpectNum; + } + + restMemSize -= BloomFilter.optimalNumOfBits(requestNum, fpp); + return new LinkedBloomFilterNode(requestNum, fpp, ttl); + } + + void takeBack(LinkedBloomFilterNode node) { + restMemSize += BloomFilter.optimalNumOfBits(node.getCapacity(), node.getFpp()); + } + + long estimatePropExpectNum(long expectNum, double fpp) { + if (restMemSize <= 0) { + expectNum = miniExpectNum; + } else { + while (BloomFilter.optimalNumOfBits(expectNum, fpp) > restMemSize) { + expectNum >>= 1; + } + } + return expectNum; + } + + byte[] buildBloomFilterKey(V record) { + try { + valueSerializationStream.reset(); + keySerializer.serialize(keyContext.getCurrentKey(), valueSerializationDataOutputView); + valueSerializer.serialize(record, valueSerializationDataOutputView); + return valueSerializationStream.toByteArray(); + } catch (IOException e) { + LOG.error("build bloom filter key failed {}", e); + throw new FlinkRuntimeException(e); + } + } + + @Override + public String toString() { + StringBuilder builder = new StringBuilder() + .append("\n------------------------------------------------>\n") + .append("total memory:").append(totalMemSize).append("\t").append("rest memory:").append(restMemSize).append("\n"); + + for (int i = 0; i < localKeyGroupRange.getNumberOfKeyGroups(); ++i) { + LinkedBloomFilter bloomFilter = this.linkedBloomFilters[i]; + if (bloomFilter != null) { + builder.append("group ").append(i + localKeyGroupRangeStartIdx).append(":").append(bloomFilter.toString()).append("\n"); + } + } + return builder.toString(); + } + + @VisibleForTesting + LinkedBloomFilter[] getLinkedBloomFilters() { + return this.linkedBloomFilters; + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterManager.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterManager.java new file mode 100644 index 0000000000000..0213492f72c41 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterManager.java @@ -0,0 +1,154 @@ +/* + * 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.flink.streaming.api.operators; + +import org.apache.flink.api.common.state.PartitionedBloomFilterDescriptor; +import org.apache.flink.api.common.typeutils.TypeSerializer; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.util.Preconditions; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.ObjectInputStream; +import java.io.ObjectOutputStream; +import java.util.HashMap; +import java.util.Map; + +/** + * Responsible for managing all {@link PartitionedBloomFilter}. + * + * @param The type of keys. + */ +public class PartitionedBloomFilterManager { + + private static final Logger LOG = LoggerFactory.getLogger(PartitionedBloomFilterManager.class); + + private Map bloomFilterStates = new HashMap<>(); + private Map bloomFilterStateDescriptors = new HashMap<>(); + private int numberOfKeyGroups; + private KeyGroupRange keyGroupRange; + private final KeyContext keyContext; + + /** Serializer for the key. */ + private final TypeSerializer keySerializer; + + public PartitionedBloomFilterManager( + KeyContext keyContext, + TypeSerializer keySerializer, + int numberOfKeyGroups, + KeyGroupRange keyGroupRange) { + + this.keyContext = Preconditions.checkNotNull(keyContext); + this.keySerializer = Preconditions.checkNotNull(keySerializer); + this.numberOfKeyGroups = Preconditions.checkNotNull(numberOfKeyGroups); + this.keyGroupRange = Preconditions.checkNotNull(keyGroupRange); + } + + // ---------------------------------------------------- + public PartitionedBloomFilter getOrCreateBloomFilterState(PartitionedBloomFilterDescriptor stateDescriptor) { + String stateName = stateDescriptor.getStateName(); + PartitionedBloomFilter state = bloomFilterStates.get(stateName); + if (state == null) { + state = new PartitionedBloomFilter(keySerializer, + stateDescriptor.getSerializer(), + numberOfKeyGroups, + keyGroupRange, + keyContext, + stateDescriptor.getCapacity(), + stateDescriptor.getFpp(), + stateDescriptor.getTtl(), + stateDescriptor.getMiniExpectNum(), + stateDescriptor.getMaxExpectNum(), + stateDescriptor.getGrowRate()); + bloomFilterStates.put(stateName, state); + bloomFilterStateDescriptors.put(stateName, stateDescriptor); + } + return state; + } + + public void dispose() { + try { + bloomFilterStates.clear(); + bloomFilterStateDescriptors.clear(); + } catch (Exception e) { + LOG.error("cancel registry close failed: {}", e); + throw new RuntimeException("cancel registry close failed: " + e); + } + } + + public void snapshotStateForKeyGroup(DataOutputViewStreamWrapper stream, int keyGroupIdx) { + try { + stream.writeInt(this.bloomFilterStates.size()); + for (Map.Entry entry : this.bloomFilterStates.entrySet()) { + PartitionedBloomFilterDescriptor desc = this.bloomFilterStateDescriptors.get(entry.getKey()); + + ObjectOutputStream outputStream = new ObjectOutputStream(stream); + outputStream.writeObject(desc); + + entry.getValue().snapshotStateForKeyGroup(stream, keyGroupIdx); + if (keyGroupIdx == 0) { + LOG.info("\n------------------------------------------------>\n" + + "Bloom filter state [{}] nodes map:" + + "{}", entry.getKey(), entry.getValue().toString()); + } + } + } catch (Exception e) { + throw new RuntimeException("Shapshot bloom filter state failed:" + e); + } + } + + public void restoreStateForKeyGroup( + DataInputViewStreamWrapper stream, + int keyGroupIdx) throws IOException, ClassNotFoundException { + try { + LOG.info("restoring state for key group {}", keyGroupIdx); + int len = stream.readInt(); + for (int i = 0; i < len; ++i) { + ObjectInputStream inputStream = new ObjectInputStream(stream); + PartitionedBloomFilterDescriptor desc = (PartitionedBloomFilterDescriptor) inputStream.readObject(); + + PartitionedBloomFilter state = bloomFilterStates.get(desc.getStateName()); + LOG.info("restoring state [{}] for key group {}", desc.getStateName(), keyGroupIdx); + if (state == null) { + LOG.info("c:{} f:{} t:{} mie:{} mae:{} g:{}", desc.getCapacity(), desc.getFpp(), desc.getTtl(), desc.getMiniExpectNum(), desc.getMaxExpectNum(), desc.getGrowRate()); + state = new PartitionedBloomFilter(keySerializer, + desc.getSerializer(), + numberOfKeyGroups, + keyGroupRange, + keyContext, + desc.getCapacity(), + desc.getFpp(), + desc.getTtl(), + desc.getMiniExpectNum(), + desc.getMaxExpectNum(), + desc.getGrowRate()); + bloomFilterStates.put(desc.getStateName(), state); + bloomFilterStateDescriptors.put(desc.getStateName(), desc); + } + state.restoreStateForKeyGroup(stream, keyGroupIdx); + } + } catch (Exception e) { + throw new RuntimeException("Restore bloom filter state failed:" + e); + } + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateContext.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateContext.java index 2eb777a8db6e3..e093d7c6b624e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateContext.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateContext.java @@ -51,6 +51,8 @@ public interface StreamOperatorStateContext { */ InternalTimeServiceManager internalTimerServiceManager(); + PartitionedBloomFilterManager bloomFilterStateManager(); + /** * Returns an iterable to obtain input streams for previously stored operator state partitions that are assigned to * this stream operator. diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java index 578302ba1e5ee..bcdac02134e21 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java @@ -23,6 +23,7 @@ import org.apache.flink.api.java.tuple.Tuple2; import org.apache.flink.core.fs.CloseableRegistry; import org.apache.flink.core.fs.FSDataInputStream; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.runtime.checkpoint.PrioritizedOperatorSubtaskState; import org.apache.flink.runtime.checkpoint.StateObjectCollection; import org.apache.flink.runtime.execution.Environment; @@ -125,6 +126,7 @@ public StreamOperatorStateContext streamOperatorStateContext( CloseableIterable rawKeyedStateInputs = null; CloseableIterable rawOperatorStateInputs = null; InternalTimeServiceManager timeServiceManager; + PartitionedBloomFilterManager bloomfilterStateManager; try { @@ -153,6 +155,9 @@ public StreamOperatorStateContext streamOperatorStateContext( // -------------- Internal Timer Service Manager -------------- timeServiceManager = internalTimeServiceManager(keyedStatedBackend, keyContext, rawKeyedStateInputs); + // -------------- BloomFilter Manager -------------- + bloomfilterStateManager = bloomfilterStateManager(keyedStatedBackend, keyContext, rawKeyedStateInputs); + // -------------- Preparing return value -------------- return new StreamOperatorStateContextImpl( @@ -160,6 +165,7 @@ public StreamOperatorStateContext streamOperatorStateContext( operatorStateBackend, keyedStatedBackend, timeServiceManager, + bloomfilterStateManager, rawOperatorStateInputs, rawKeyedStateInputs); } catch (Exception ex) { @@ -224,6 +230,38 @@ protected InternalTimeServiceManager internalTimeServiceManager( return timeServiceManager; } + protected PartitionedBloomFilterManager bloomfilterStateManager( + AbstractKeyedStateBackend keyedStatedBackend, + KeyContext keyContext, //the operator + Iterable rawKeyedStates) throws Exception { + + if (keyedStatedBackend == null) { + return null; + } + + final KeyGroupRange keyGroupRange = keyedStatedBackend.getKeyGroupRange(); + + PartitionedBloomFilterManager bloomFilterManager = new PartitionedBloomFilterManager( + keyContext, + keyedStatedBackend.getKeySerializer(), + keyedStatedBackend.getNumberOfKeyGroups(), + keyedStatedBackend.getKeyGroupRange()); + + // and then initialize the timer services + for (KeyGroupStatePartitionStreamProvider streamProvider : rawKeyedStates) { + int keyGroupIdx = streamProvider.getKeyGroupId(); + + Preconditions.checkArgument(keyGroupRange.contains(keyGroupIdx), + "Key Group " + keyGroupIdx + " does not belong to the local range."); + + bloomFilterManager.restoreStateForKeyGroup( + new DataInputViewStreamWrapper(streamProvider.getStream()), + keyGroupIdx); + } + + return bloomFilterManager; + } + protected OperatorStateBackend operatorStateBackend( String operatorIdentifierText, PrioritizedOperatorSubtaskState prioritizedOperatorSubtaskStates, @@ -545,6 +583,7 @@ private static class StreamOperatorStateContextImpl implements StreamOperatorSta private final OperatorStateBackend operatorStateBackend; private final AbstractKeyedStateBackend keyedStateBackend; private final InternalTimeServiceManager internalTimeServiceManager; + private final PartitionedBloomFilterManager bloomFilterStateManager; private final CloseableIterable rawOperatorStateInputs; private final CloseableIterable rawKeyedStateInputs; @@ -554,6 +593,7 @@ private static class StreamOperatorStateContextImpl implements StreamOperatorSta OperatorStateBackend operatorStateBackend, AbstractKeyedStateBackend keyedStateBackend, InternalTimeServiceManager internalTimeServiceManager, + PartitionedBloomFilterManager bloomFilterStateManager, CloseableIterable rawOperatorStateInputs, CloseableIterable rawKeyedStateInputs) { @@ -561,6 +601,7 @@ private static class StreamOperatorStateContextImpl implements StreamOperatorSta this.operatorStateBackend = operatorStateBackend; this.keyedStateBackend = keyedStateBackend; this.internalTimeServiceManager = internalTimeServiceManager; + this.bloomFilterStateManager = bloomFilterStateManager; this.rawOperatorStateInputs = rawOperatorStateInputs; this.rawKeyedStateInputs = rawKeyedStateInputs; } @@ -585,6 +626,11 @@ public InternalTimeServiceManager internalTimerServiceManager() { return internalTimeServiceManager; } + @Override + public PartitionedBloomFilterManager bloomFilterStateManager() { + return bloomFilterStateManager; + } + @Override public CloseableIterable rawOperatorStateInputs() { return rawOperatorStateInputs; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java index 89c038fbad574..0b7e0297a03d1 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java @@ -31,6 +31,7 @@ import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.MapState; import org.apache.flink.api.common.state.MapStateDescriptor; +import org.apache.flink.api.common.state.PartitionedBloomFilterDescriptor; import org.apache.flink.api.common.state.ReducingState; import org.apache.flink.api.common.state.ReducingStateDescriptor; import org.apache.flink.api.common.state.StateDescriptor; @@ -170,6 +171,11 @@ public MapState getMapState(MapStateDescriptor statePro return keyedStateStore.getMapState(stateProperties); } + public PartitionedBloomFilter getPartitionedBloomFilter(PartitionedBloomFilterDescriptor descriptor) { + descriptor.initializeSerializerUnlessSet(getExecutionConfig()); + return operator.getBloomFilterStateManager().getOrCreateBloomFilterState(descriptor); + } + private KeyedStateStore checkPreconditionsAndGetKeyedStateStore(StateDescriptor stateDescriptor) { Preconditions.checkNotNull(stateDescriptor, "The state properties must not be null"); KeyedStateStore keyedStateStore = operator.getKeyedStateStore(); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNodeTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNodeTest.java new file mode 100644 index 0000000000000..8a190d112cb49 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNodeTest.java @@ -0,0 +1,59 @@ +package org.apache.flink.streaming.api.operators; + +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; + +import org.junit.Assert; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +/** + * {@link LinkedBloomFilterNode} unit tests. + */ +public class LinkedBloomFilterNodeTest { + + @Test + public void basicTest() throws InterruptedException { + + LinkedBloomFilterNode node = new LinkedBloomFilterNode(100, 0.01, 1000); + + Assert.assertEquals(Long.MAX_VALUE, node.getDeleteTS()); + + for (int i = 0; i < 100; ++i) { + node.add(String.valueOf(i).getBytes()); + Assert.assertTrue(node.contains(String.valueOf(i).getBytes())); + } + Assert.assertTrue(node.isFull()); + TimeUnit.MILLISECONDS.sleep(1000); + Assert.assertNotEquals(Long.MAX_VALUE, node.getDeleteTS()); + } + + @Test + public void testSnapshotAndRestore() throws InterruptedException, IOException { + + LinkedBloomFilterNode node1 = new LinkedBloomFilterNode(100, 0.01, 1000); + for (int i = 0; i < 100; ++i) { + node1.add(String.valueOf(i).getBytes()); + Assert.assertTrue(node1.contains(String.valueOf(i).getBytes())); + } + + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper outputViewStreamWrapper = new DataOutputViewStreamWrapper(outputStream); + + node1.snapshot(outputViewStreamWrapper); + + byte[] outputBytes = outputStream.toByteArray(); + + LinkedBloomFilterNode node2 = new LinkedBloomFilterNode(); + node2.restore(new DataInputViewStreamWrapper(new ByteArrayInputStream(outputBytes))); + + Assert.assertEquals(node1.getCapacity(), node2.getCapacity()); + Assert.assertEquals(String.valueOf(node1.getFpp()), String.valueOf(node2.getFpp())); + Assert.assertEquals(node1.getSize(), node2.getSize()); + Assert.assertEquals(node1.getBloomFilter(), node2.getBloomFilter()); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterTest.java new file mode 100644 index 0000000000000..82fd2c34f8ca3 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterTest.java @@ -0,0 +1,131 @@ +package org.apache.flink.streaming.api.operators; + +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.runtime.state.KeyGroupRange; + +import org.junit.Assert; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.List; +import java.util.concurrent.TimeUnit; + +/** + * {@link LinkedBloomFilter} unit tests. + */ +public class LinkedBloomFilterTest { + + @Test + public void basicTest() throws InterruptedException { + + StreamOperator dumyKeyContext = new AbstractStreamOperator() { + private static final long serialVersionUID = 1L; + }; + + PartitionedBloomFilter partitionedBloomFilter = new PartitionedBloomFilter( + new StringSerializer(), + new IntSerializer(), + 10, + new KeyGroupRange(0, 9), + dumyKeyContext, + 10000, + 0.01, + 1000, + 1000, + 10000, + 2.0 + ); + + LinkedBloomFilter linkedBloomFilter = new LinkedBloomFilter(partitionedBloomFilter, 1000, 2); + + List nodes = linkedBloomFilter.getBloomFilterNodes(); + + Assert.assertEquals(0, nodes.size()); + for (int i = 0; i < 1000; ++i) { + linkedBloomFilter.add(String.valueOf(i).getBytes()); + Assert.assertTrue(linkedBloomFilter.contains(String.valueOf(i).getBytes())); + } + Assert.assertEquals(1, nodes.size()); + + linkedBloomFilter.add("1001".getBytes()); + Assert.assertEquals(2, nodes.size()); + } + + @Test + public void testSnapshotAndRestore() throws InterruptedException, IOException { + + StreamOperator dumyKeyContext = new AbstractStreamOperator() { + private static final long serialVersionUID = 1L; + }; + + PartitionedBloomFilter partitionedBloomFilter = new PartitionedBloomFilter( + new StringSerializer(), + new IntSerializer(), + 10, + new KeyGroupRange(0, 9), + dumyKeyContext, + 10000, + 0.01, + 1000, + 1000, + 10000, + 2.0 + ); + + LinkedBloomFilter linkedBloomFilter = new LinkedBloomFilter(partitionedBloomFilter, 1000, 2); + + List nodes = linkedBloomFilter.getBloomFilterNodes(); + + Assert.assertEquals(0, nodes.size()); + for (int i = 0; i < 1000; ++i) { + linkedBloomFilter.add(String.valueOf(i).getBytes()); + Assert.assertTrue(linkedBloomFilter.contains(String.valueOf(i).getBytes())); + } + Assert.assertEquals(1, nodes.size()); + + linkedBloomFilter.add(String.valueOf("1001").getBytes()); + Assert.assertEquals(2, nodes.size()); + + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + + DataOutputViewStreamWrapper outputViewStreamWrapper = new DataOutputViewStreamWrapper(outputStream); + linkedBloomFilter.snapshot(outputViewStreamWrapper); + + byte[] outputBytes = outputStream.toByteArray(); + + LinkedBloomFilter linkedBloomFilter2 = new LinkedBloomFilter(partitionedBloomFilter, 1000, 2); + + linkedBloomFilter2.restore(new DataInputViewStreamWrapper(new ByteArrayInputStream(outputBytes))); + + Assert.assertEquals(linkedBloomFilter.getCurrentSize(), linkedBloomFilter2.getCurrentSize()); + Assert.assertEquals(linkedBloomFilter.getInitSize(), linkedBloomFilter2.getInitSize()); + Assert.assertEquals(String.valueOf(linkedBloomFilter.getGrowRate()), String.valueOf(linkedBloomFilter2.getGrowRate())); + + List nodes2 = linkedBloomFilter2.getBloomFilterNodes(); + + Assert.assertEquals(nodes.size(), nodes2.size()); + for (int i = 0; i < nodes.size(); ++i) { + Assert.assertEquals(nodes.get(i), nodes2.get(i)); + } + + // ---- test ttl + TimeUnit.SECONDS.sleep(2); + outputStream.reset(); + outputViewStreamWrapper = new DataOutputViewStreamWrapper(outputStream); + linkedBloomFilter.snapshot(outputViewStreamWrapper); + outputBytes = outputStream.toByteArray(); + + LinkedBloomFilter linkedBloomFilter3 = new LinkedBloomFilter(partitionedBloomFilter, 1000, 2); + linkedBloomFilter3.restore(new DataInputViewStreamWrapper(new ByteArrayInputStream(outputBytes))); + List nodes3 = linkedBloomFilter3.getBloomFilterNodes(); + + Assert.assertEquals(1, nodes.size()); + Assert.assertEquals(1, nodes3.size()); + Assert.assertEquals(nodes.get(0), nodes3.get(0)); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterManagerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterManagerTest.java new file mode 100644 index 0000000000000..a4a77d5d53e39 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterManagerTest.java @@ -0,0 +1,146 @@ +package org.apache.flink.streaming.api.operators; + +import org.apache.flink.api.common.ExecutionConfig; +import org.apache.flink.api.common.state.PartitionedBloomFilterDescriptor; +import org.apache.flink.api.common.typeinfo.TypeInformation; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.runtime.state.KeyGroupRange; + +import org.junit.Assert; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.util.Random; + +/** + * {@link PartitionedBloomFilterManager} unit tests. + */ +public class PartitionedBloomFilterManagerTest { + + @Test + public void basicTest() throws Exception { + + KeyContext keyContext = new KeyContext() { + private Object key; + + @Override + public void setCurrentKey(Object key) { + this.key = key; + } + + @Override + public Object getCurrentKey() { + return this.key; + } + }; + + PartitionedBloomFilterManager partitionedBloomFilterManager = new PartitionedBloomFilterManager( + keyContext, + TypeInformation.of(String.class).createSerializer(new ExecutionConfig()), + 10, + new KeyGroupRange(0, 9)); + + PartitionedBloomFilter partitionedBloomFilter = partitionedBloomFilterManager.getOrCreateBloomFilterState( + new PartitionedBloomFilterDescriptor( + "test-bf", + TypeInformation.of(Integer.class).createSerializer(new ExecutionConfig()), + 100, 0.01, 10000)); + + keyContext.setCurrentKey("hello"); + for (int i = 0; i < 100; ++i) { + partitionedBloomFilter.add(i); + Assert.assertTrue(partitionedBloomFilter.contains(i)); + } + } + + @Test + public void testSerializerAndDeserializer() throws Exception { + + KeyContext keyContext = new KeyContext() { + private Object key; + + @Override + public void setCurrentKey(Object key) { + this.key = key; + } + + @Override + public Object getCurrentKey() { + return this.key; + } + }; + + PartitionedBloomFilterManager partitionedBloomFilterManager = new PartitionedBloomFilterManager( + keyContext, + TypeInformation.of(String.class).createSerializer(new ExecutionConfig()), + 10, + new KeyGroupRange(0, 9)); + + PartitionedBloomFilterDescriptor desc1 = new PartitionedBloomFilterDescriptor( + "test-bf-1", + TypeInformation.of(Integer.class).createSerializer(new ExecutionConfig()), + 10_000, 0.01, 60000); + + PartitionedBloomFilterDescriptor desc2 = new PartitionedBloomFilterDescriptor( + "test-bf-2", + TypeInformation.of(Integer.class).createSerializer(new ExecutionConfig()), + 10_000, 0.01, 60000); + + PartitionedBloomFilter partitionedBloomFilter1 = partitionedBloomFilterManager.getOrCreateBloomFilterState(desc1); + PartitionedBloomFilter partitionedBloomFilter2 = partitionedBloomFilterManager.getOrCreateBloomFilterState(desc2); + + String[] keys1 = new String[10_000]; + for (int i = 0; i < 10_000; ++i) { + String key = String.valueOf(new Random().nextInt(1000)); + keyContext.setCurrentKey(key); + partitionedBloomFilter1.add(i); + keys1[i] = key; + } + + String[] keys2 = new String[10_000]; + for (int i = 0; i < 10_000; ++i) { + String key = String.valueOf(new Random().nextInt(1000)); + keyContext.setCurrentKey(key); + partitionedBloomFilter2.add(i); + keys2[i] = key; + } + + // snapshot + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper outputViewStreamWrapper = new DataOutputViewStreamWrapper(outputStream); + for (int i = 0; i < 10; ++i) { + partitionedBloomFilterManager.snapshotStateForKeyGroup(outputViewStreamWrapper, i); + } + + // restore + ByteArrayInputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray()); + DataInputViewStreamWrapper inputViewStreamWrapper = new DataInputViewStreamWrapper(inputStream); + PartitionedBloomFilterManager partitionedBloomFilterManager2 = new PartitionedBloomFilterManager( + keyContext, + TypeInformation.of(String.class).createSerializer(new ExecutionConfig()), + 10, + new KeyGroupRange(0, 9)); + + for (int i = 0; i < 10; ++i) { + partitionedBloomFilterManager2.restoreStateForKeyGroup(inputViewStreamWrapper, i); + } + + // valid + PartitionedBloomFilter partitionedBloomFilter3 = partitionedBloomFilterManager2.getOrCreateBloomFilterState(desc1); + PartitionedBloomFilter partitionedBloomFilter4 = partitionedBloomFilterManager2.getOrCreateBloomFilterState(desc2); + + for (int i = 0; i < 10_000; ++i) { + String key = keys1[i]; + keyContext.setCurrentKey(key); + Assert.assertTrue(partitionedBloomFilter3.contains(i)); + } + + for (int i = 0; i < 10_000; ++i) { + String key = keys2[i]; + keyContext.setCurrentKey(key); + Assert.assertTrue(partitionedBloomFilter4.contains(i)); + } + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterTest.java new file mode 100644 index 0000000000000..548d6c7112b73 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterTest.java @@ -0,0 +1,145 @@ +package org.apache.flink.streaming.api.operators; + +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.runtime.state.KeyGroupRange; +import org.apache.flink.runtime.state.KeyGroupRangeAssignment; + +import org.junit.Assert; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.util.Random; + +/** + * {@link PartitionedBloomFilter} unit tests. + */ +public class PartitionedBloomFilterTest { + + @Test + public void basicTest() throws InterruptedException { + + KeyContext keyContext = new KeyContext() { + private Object key; + + @Override + public void setCurrentKey(Object key) { + this.key = key; + } + + @Override + public Object getCurrentKey() { + return this.key; + } + }; + + PartitionedBloomFilter partitionedBloomFilter = new PartitionedBloomFilter( + new StringSerializer(), + new IntSerializer(), + 10, + new KeyGroupRange(0, 9), + keyContext, + 10000, + 0.01, + 1000, + 1000, + 10000, + 2.0 + ); + + LinkedBloomFilter[] linkedBloomFilters = partitionedBloomFilter.getLinkedBloomFilters(); + Assert.assertEquals(10, linkedBloomFilters.length); + for (int i = 0; i < 10; ++i) { + Assert.assertNull(linkedBloomFilters[i]); + } + + String currentKey = "hello"; + + keyContext.setCurrentKey(currentKey); + for (int i = 0; i < 1000; ++i) { + partitionedBloomFilter.add(i); + Assert.assertTrue(partitionedBloomFilter.contains(i)); + } + + int currentGroup = KeyGroupRangeAssignment.assignToKeyGroup(currentKey, 10); + Assert.assertNotNull(linkedBloomFilters[currentGroup]); + } + + @Test + public void testSnapshotAndRestore() throws Exception { + + KeyContext keyContext = new KeyContext() { + private Object key; + + @Override + public void setCurrentKey(Object key) { + this.key = key; + } + + @Override + public Object getCurrentKey() { + return this.key; + } + }; + + PartitionedBloomFilter partitionedBloomFilter = new PartitionedBloomFilter( + new StringSerializer(), + new IntSerializer(), + 10, + new KeyGroupRange(0, 9), + keyContext, + 10000, + 0.01, + 1000, + 1000, + 10000, + 2.0 + ); + + String[] keys = new String[10_000]; + for (int i = 0; i < 10_000; ++i) { + String key = String.valueOf(new Random().nextInt(1000)); + keyContext.setCurrentKey(key); + partitionedBloomFilter.add(i); + keys[i] = key; + } + + // snapshot one by one + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + DataOutputViewStreamWrapper outputViewStreamWrapper = new DataOutputViewStreamWrapper(outputStream); + + for (int i = 0; i < 10; ++i) { + partitionedBloomFilter.snapshotStateForKeyGroup(outputViewStreamWrapper, i); + } + + PartitionedBloomFilter partitionedBloomFilter2 = new PartitionedBloomFilter( + new StringSerializer(), + new IntSerializer(), + 10, + new KeyGroupRange(0, 9), + keyContext, + 10000, + 0.01, + 1000, + 1000, + 10000, + 2.0 + ); + + ByteArrayInputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray()); + DataInputViewStreamWrapper inputViewStreamWrapper = new DataInputViewStreamWrapper(inputStream); + // restore one by one + for (int i = 0; i < 10; ++i) { + partitionedBloomFilter2.restoreStateForKeyGroup(inputViewStreamWrapper, i); + } + + // valid + for (int i = 0; i < 10_000; ++i) { + keyContext.setCurrentKey(keys[i]); + Assert.assertTrue(partitionedBloomFilter2.contains(i)); + } + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index 1a4ce315d276e..fe85b9602abd2 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -101,6 +101,7 @@ import org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer; import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; import org.apache.flink.streaming.api.operators.Output; +import org.apache.flink.streaming.api.operators.PartitionedBloomFilterManager; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperatorStateContext; import org.apache.flink.streaming.api.operators.StreamSource; @@ -1113,6 +1114,11 @@ public InternalTimeServiceManager internalTimerServiceManager() { return timeServiceManager != null ? spy(timeServiceManager) : null; } + @Override + public PartitionedBloomFilterManager bloomFilterStateManager() { + return context.bloomFilterStateManager(); + } + @Override public CloseableIterable rawOperatorStateInputs() { return replaceWithSpy(context.rawOperatorStateInputs()); diff --git a/flink-test-utils-parent/flink-test-utils/pom.xml b/flink-test-utils-parent/flink-test-utils/pom.xml index 2e92cfb3c3206..d921dc27cbdbf 100644 --- a/flink-test-utils-parent/flink-test-utils/pom.xml +++ b/flink-test-utils-parent/flink-test-utils/pom.xml @@ -83,6 +83,12 @@ under the License. curator-test ${curator.version} compile + + + com.google.guava + guava + + From d433a5290c9971b1feb4f0915e66e06cddbaecf6 Mon Sep 17 00:00:00 2001 From: sihuazhou Date: Wed, 25 Apr 2018 00:06:29 +0800 Subject: [PATCH 2/5] Change LinkedBloomFilterNode to ShrinkableBloomFilterNode which is shrinkable. --- .../bloomfilter/BloomFilterExample.java | 6 +- .../api/operators/AbstractStreamOperator.java | 4 +- ...oomFilter.java => ElasticBloomFilter.java} | 52 +-- ...er.java => ElasticBloomFilterManager.java} | 22 +- .../api/operators/LinkedBloomFilterNode.java | 180 ---------- ....java => LinkedShrinkableBloomFilter.java} | 50 +-- .../operators/ShrinkableBloomFilterNode.java | 309 ++++++++++++++++++ .../operators/StreamOperatorStateContext.java | 2 +- .../StreamTaskStateInitializerImpl.java | 12 +- .../operators/StreamingRuntimeContext.java | 2 +- .../operators/LinkedBloomFilterNodeTest.java | 8 +- .../api/operators/LinkedBloomFilterTest.java | 22 +- .../PartitionedBloomFilterManagerTest.java | 18 +- .../operators/PartitionedBloomFilterTest.java | 10 +- .../runtime/tasks/StreamTaskTest.java | 4 +- 15 files changed, 415 insertions(+), 286 deletions(-) rename flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/{PartitionedBloomFilter.java => ElasticBloomFilter.java} (82%) rename flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/{PartitionedBloomFilterManager.java => ElasticBloomFilterManager.java} (86%) delete mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNode.java rename flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/{LinkedBloomFilter.java => LinkedShrinkableBloomFilter.java} (74%) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNode.java diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/bloomfilter/BloomFilterExample.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/bloomfilter/BloomFilterExample.java index 2a2cfb57fcf2d..05b82e9a3e4da 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/bloomfilter/BloomFilterExample.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/bloomfilter/BloomFilterExample.java @@ -9,7 +9,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction; -import org.apache.flink.streaming.api.operators.PartitionedBloomFilter; +import org.apache.flink.streaming.api.operators.ElasticBloomFilter; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; @@ -74,8 +74,8 @@ public Integer getKey(Integer value) throws Exception { }) .timeWindow(Time.milliseconds(1000)) .apply(new RichWindowFunction() { - private PartitionedBloomFilter bf1; - private PartitionedBloomFilter bf2; + private ElasticBloomFilter bf1; + private ElasticBloomFilter bf2; @Override public void open(Configuration parameters) throws Exception { diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java index fd824771f574f..e1e2862d7e7ee 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java @@ -156,7 +156,7 @@ public abstract class AbstractStreamOperator // ---------------- bloom filter handler ------------------ /** Backend for bloomfilter. */ - private PartitionedBloomFilterManager bloomFilterManager; + private ElasticBloomFilterManager bloomFilterManager; // ---------------- two-input operator watermarks ------------------ @@ -628,7 +628,7 @@ public KeyedStateStore getKeyedStateStore() { return keyedStateStore; } - protected PartitionedBloomFilterManager getBloomFilterStateManager() { + protected ElasticBloomFilterManager getBloomFilterStateManager() { return bloomFilterManager; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilter.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticBloomFilter.java similarity index 82% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilter.java rename to flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticBloomFilter.java index d67e43bcdbe30..b17ba7e597b2e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilter.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticBloomFilter.java @@ -46,14 +46,14 @@ * @param The type of keys. * @param The type of values. */ -public class PartitionedBloomFilter { +public class ElasticBloomFilter { - private static final Logger LOG = LoggerFactory.getLogger(PartitionedBloomFilter.class); + private static final Logger LOG = LoggerFactory.getLogger(ElasticBloomFilter.class); private final int totalKeyGroups; private final KeyGroupsList localKeyGroupRange; private final KeyContext keyContext; - private LinkedBloomFilter[] linkedBloomFilters; + private LinkedShrinkableBloomFilter[] linkedBloomFilters; private final int localKeyGroupRangeStartIdx; private long totalMemSize; @@ -75,17 +75,17 @@ public class PartitionedBloomFilter { private final ByteArrayOutputStreamWithPos valueSerializationStream; private final DataOutputView valueSerializationDataOutputView; - public PartitionedBloomFilter(TypeSerializer keySerializer, - TypeSerializer valueSerializer, - int totalKeyGroups, - KeyGroupsList localKeyGroupRange, - KeyContext keyContext, - long capacity, - double fpp, - long ttl, - long miniExpectNum, - long maxExpectNum, - double growRate) { + public ElasticBloomFilter(TypeSerializer keySerializer, + TypeSerializer valueSerializer, + int totalKeyGroups, + KeyGroupsList localKeyGroupRange, + KeyContext keyContext, + long capacity, + double fpp, + long ttl, + long miniExpectNum, + long maxExpectNum, + double growRate) { this.capacity = capacity; this.fpp = fpp; this.ttl = ttl; @@ -99,7 +99,7 @@ public PartitionedBloomFilter(TypeSerializer keySerializer, this.keyContext = keyContext; this.totalKeyGroups = totalKeyGroups; this.localKeyGroupRange = localKeyGroupRange; - this.linkedBloomFilters = new LinkedBloomFilter[localKeyGroupRange.getNumberOfKeyGroups()]; + this.linkedBloomFilters = new LinkedShrinkableBloomFilter[localKeyGroupRange.getNumberOfKeyGroups()]; // find the starting index of the local key-group range int startIdx = Integer.MAX_VALUE; @@ -118,9 +118,9 @@ public void add(V content) { int keyGroupIndex = KeyGroupRangeAssignment.assignToKeyGroup(keyContext.getCurrentKey(), totalKeyGroups); int index = getIndexForKeyGroup(keyGroupIndex); - LinkedBloomFilter bloomFilter = linkedBloomFilters[index]; + LinkedShrinkableBloomFilter bloomFilter = linkedBloomFilters[index]; if (bloomFilter == null) { - bloomFilter = new LinkedBloomFilter(this, miniExpectNum, growRate); + bloomFilter = new LinkedShrinkableBloomFilter(this, miniExpectNum, growRate); linkedBloomFilters[index] = bloomFilter; } bloomFilter.add(buildBloomFilterKey(content)); @@ -130,7 +130,7 @@ public boolean contains(V content) { int keyGroupIndex = KeyGroupRangeAssignment.assignToKeyGroup(keyContext.getCurrentKey(), totalKeyGroups); int index = getIndexForKeyGroup(keyGroupIndex); - LinkedBloomFilter bloomFilter = linkedBloomFilters[index]; + LinkedShrinkableBloomFilter bloomFilter = linkedBloomFilters[index]; if (bloomFilter == null) { return false; } @@ -147,7 +147,7 @@ public void snapshotStateForKeyGroup(DataOutputViewStreamWrapper stream, int key LOG.info("snapshot state for group {} ", keyGroupIdx); int index = getIndexForKeyGroup(keyGroupIdx); - LinkedBloomFilter bloomFilter = this.linkedBloomFilters[index]; + LinkedShrinkableBloomFilter bloomFilter = this.linkedBloomFilters[index]; if (bloomFilter != null) { stream.writeBoolean(true); stream.writeLong(this.restMemSize); @@ -165,7 +165,7 @@ public void restoreStateForKeyGroup( int index = getIndexForKeyGroup(keyGroupIdx); if (stream.readBoolean()) { this.restMemSize = stream.readLong(); - LinkedBloomFilter linkedBloomFilter = new LinkedBloomFilter(this, miniExpectNum, growRate); + LinkedShrinkableBloomFilter linkedBloomFilter = new LinkedShrinkableBloomFilter(this, miniExpectNum, growRate); linkedBloomFilter.restore(stream); this.linkedBloomFilters[index] = linkedBloomFilter; LOG.info("group {} restored.", keyGroupIdx); @@ -176,11 +176,11 @@ public void restoreStateForKeyGroup( // --------------------- - LinkedBloomFilterNode allocateBloomFilterNode(long expectNum) { + ShrinkableBloomFilterNode allocateBloomFilterNode(long expectNum) { return allocateBloomFilterNode(expectNum, false); } - LinkedBloomFilterNode allocateBloomFilterNode(long expectNum, boolean force) { + ShrinkableBloomFilterNode allocateBloomFilterNode(long expectNum, boolean force) { long requestNum = expectNum; if (!force) { if (restMemSize > 0) { @@ -199,10 +199,10 @@ LinkedBloomFilterNode allocateBloomFilterNode(long expectNum, boolean force) { } restMemSize -= BloomFilter.optimalNumOfBits(requestNum, fpp); - return new LinkedBloomFilterNode(requestNum, fpp, ttl); + return new ShrinkableBloomFilterNode((int) requestNum, fpp, ttl); } - void takeBack(LinkedBloomFilterNode node) { + void takeBack(ShrinkableBloomFilterNode node) { restMemSize += BloomFilter.optimalNumOfBits(node.getCapacity(), node.getFpp()); } @@ -236,7 +236,7 @@ public String toString() { .append("total memory:").append(totalMemSize).append("\t").append("rest memory:").append(restMemSize).append("\n"); for (int i = 0; i < localKeyGroupRange.getNumberOfKeyGroups(); ++i) { - LinkedBloomFilter bloomFilter = this.linkedBloomFilters[i]; + LinkedShrinkableBloomFilter bloomFilter = this.linkedBloomFilters[i]; if (bloomFilter != null) { builder.append("group ").append(i + localKeyGroupRangeStartIdx).append(":").append(bloomFilter.toString()).append("\n"); } @@ -245,7 +245,7 @@ public String toString() { } @VisibleForTesting - LinkedBloomFilter[] getLinkedBloomFilters() { + LinkedShrinkableBloomFilter[] getLinkedBloomFilters() { return this.linkedBloomFilters; } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterManager.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticBloomFilterManager.java similarity index 86% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterManager.java rename to flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticBloomFilterManager.java index 0213492f72c41..d0b556ecdb578 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterManager.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticBloomFilterManager.java @@ -35,15 +35,15 @@ import java.util.Map; /** - * Responsible for managing all {@link PartitionedBloomFilter}. + * Responsible for managing all {@link ElasticBloomFilter}. * * @param The type of keys. */ -public class PartitionedBloomFilterManager { +public class ElasticBloomFilterManager { - private static final Logger LOG = LoggerFactory.getLogger(PartitionedBloomFilterManager.class); + private static final Logger LOG = LoggerFactory.getLogger(ElasticBloomFilterManager.class); - private Map bloomFilterStates = new HashMap<>(); + private Map bloomFilterStates = new HashMap<>(); private Map bloomFilterStateDescriptors = new HashMap<>(); private int numberOfKeyGroups; private KeyGroupRange keyGroupRange; @@ -52,7 +52,7 @@ public class PartitionedBloomFilterManager { /** Serializer for the key. */ private final TypeSerializer keySerializer; - public PartitionedBloomFilterManager( + public ElasticBloomFilterManager( KeyContext keyContext, TypeSerializer keySerializer, int numberOfKeyGroups, @@ -65,11 +65,11 @@ public PartitionedBloomFilterManager( } // ---------------------------------------------------- - public PartitionedBloomFilter getOrCreateBloomFilterState(PartitionedBloomFilterDescriptor stateDescriptor) { + public ElasticBloomFilter getOrCreateBloomFilterState(PartitionedBloomFilterDescriptor stateDescriptor) { String stateName = stateDescriptor.getStateName(); - PartitionedBloomFilter state = bloomFilterStates.get(stateName); + ElasticBloomFilter state = bloomFilterStates.get(stateName); if (state == null) { - state = new PartitionedBloomFilter(keySerializer, + state = new ElasticBloomFilter(keySerializer, stateDescriptor.getSerializer(), numberOfKeyGroups, keyGroupRange, @@ -99,7 +99,7 @@ public void dispose() { public void snapshotStateForKeyGroup(DataOutputViewStreamWrapper stream, int keyGroupIdx) { try { stream.writeInt(this.bloomFilterStates.size()); - for (Map.Entry entry : this.bloomFilterStates.entrySet()) { + for (Map.Entry entry : this.bloomFilterStates.entrySet()) { PartitionedBloomFilterDescriptor desc = this.bloomFilterStateDescriptors.get(entry.getKey()); ObjectOutputStream outputStream = new ObjectOutputStream(stream); @@ -127,11 +127,11 @@ public void restoreStateForKeyGroup( ObjectInputStream inputStream = new ObjectInputStream(stream); PartitionedBloomFilterDescriptor desc = (PartitionedBloomFilterDescriptor) inputStream.readObject(); - PartitionedBloomFilter state = bloomFilterStates.get(desc.getStateName()); + ElasticBloomFilter state = bloomFilterStates.get(desc.getStateName()); LOG.info("restoring state [{}] for key group {}", desc.getStateName(), keyGroupIdx); if (state == null) { LOG.info("c:{} f:{} t:{} mie:{} mae:{} g:{}", desc.getCapacity(), desc.getFpp(), desc.getTtl(), desc.getMiniExpectNum(), desc.getMaxExpectNum(), desc.getGrowRate()); - state = new PartitionedBloomFilter(keySerializer, + state = new ElasticBloomFilter(keySerializer, desc.getSerializer(), numberOfKeyGroups, keyGroupRange, diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNode.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNode.java deleted file mode 100644 index bf7a05b5870fc..0000000000000 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNode.java +++ /dev/null @@ -1,180 +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.flink.streaming.api.operators; - -import org.apache.flink.core.memory.DataInputView; -import org.apache.flink.core.memory.DataOutputView; - -import org.apache.flink.shaded.guava18.com.google.common.hash.BloomFilter; -import org.apache.flink.shaded.guava18.com.google.common.hash.Funnels; - -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; - -/** - * A bloom filter node linked each other in {@link LinkedBloomFilter} to avoid data skewed. - */ -public class LinkedBloomFilterNode { - - private static final Logger LOG = LoggerFactory.getLogger(LinkedBloomFilterNode.class); - - private long capacity; - private double fpp; - private long size; - private long rawTtl; - private long ttl; - private long deleteTS = Long.MAX_VALUE; - - private BloomFilter bloomFilter; - - LinkedBloomFilterNode() { - - } - - public LinkedBloomFilterNode(long capacity, double fpp, long ttl) { - this.capacity = capacity; - this.fpp = fpp; - this.rawTtl = ttl; - this.ttl = ttl; - this.size = 0; - - LOG.info("create bf capacity:{} fpp:{}", capacity, fpp); - bloomFilter = BloomFilter.create( - Funnels.byteArrayFunnel(), - (int) capacity, - this.fpp); - } - - public boolean isFull() { - return size >= capacity; - } - - public void add(byte[] content) { - bloomFilter.put(content); - - this.size++; - if (size >= capacity) { - if (deleteTS == Long.MAX_VALUE) { - long ts = System.currentTimeMillis(); - deleteTS = ts + ttl; - } - } - } - - public boolean contains(byte[] content) { - return bloomFilter.mightContain(content); - } - - public long getCapacity() { - return capacity; - } - - public double getFpp() { - return fpp; - } - - public long getSize() { - return size; - } - - public long getTtl() { - return ttl; - } - - public void reSetTtl() { - this.ttl = this.rawTtl; - this.deleteTS = Long.MAX_VALUE; - } - - public long getDeleteTS() { - return deleteTS; - } - - public BloomFilter getBloomFilter() { - return bloomFilter; - } - - public LinkedBloomFilterNode copy() { - LinkedBloomFilterNode node = new LinkedBloomFilterNode(capacity, fpp, ttl); - node.size = this.size; - node.bloomFilter = this.bloomFilter.copy(); - return node; - } - - @Override - public String toString() { - return String.format("{c:%d s:%d}", capacity, size); - } - - @Override - public boolean equals(Object obj) { - if (obj instanceof LinkedBloomFilterNode) { - LinkedBloomFilterNode other = (LinkedBloomFilterNode) obj; - if (other.capacity == this.capacity - && other.size == this.size - && other.rawTtl == this.rawTtl - && other.fpp == this.fpp - && other.bloomFilter.equals(this.bloomFilter)) { - return true; - } - return false; - } - return false; - } - - void snapshot(DataOutputView outputView) throws IOException { - outputView.writeLong(capacity); - outputView.writeLong(rawTtl); - if (deleteTS == Long.MAX_VALUE) { - outputView.writeLong(ttl); //rest ttl - } else { - outputView.writeLong(deleteTS - System.currentTimeMillis()); //rest ttl - } - outputView.writeLong(deleteTS); - outputView.writeLong(size); - outputView.writeDouble(fpp); - ByteArrayOutputStream out = new ByteArrayOutputStream(); - bloomFilter.writeTo(out); - byte[] bytes = out.toByteArray(); - outputView.writeInt(bytes.length); - outputView.write(bytes); - } - - void restore(DataInputView source) throws IOException { - capacity = source.readLong(); - rawTtl = source.readLong(); - ttl = source.readLong(); - deleteTS = source.readLong(); - if (rawTtl != ttl) { - deleteTS = System.currentTimeMillis() + ttl; - } - size = source.readLong(); - fpp = source.readDouble(); - - int byteLen = source.readInt(); - byte[] bytes = new byte[byteLen]; - source.read(bytes, 0, byteLen); - ByteArrayInputStream input = new ByteArrayInputStream(bytes); - bloomFilter = BloomFilter.readFrom(input, Funnels.byteArrayFunnel()); - } -} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedBloomFilter.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedShrinkableBloomFilter.java similarity index 74% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedBloomFilter.java rename to flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedShrinkableBloomFilter.java index ac2fce7e0ada3..6e63b89cc88f7 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedBloomFilter.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedShrinkableBloomFilter.java @@ -30,23 +30,23 @@ import java.util.LinkedList; /** - * A list of {@link LinkedBloomFilterNode} to avoid data skewed between key ranges. The size of nodes on the list + * A list of {@link ShrinkableBloomFilterNode} to avoid data skewed between key ranges. The size of nodes on the list * grow by a {@code growRate} to avoid the list to be too long. */ -public class LinkedBloomFilter { +public class LinkedShrinkableBloomFilter { - private static final Logger LOG = LoggerFactory.getLogger(LinkedBloomFilter.class); + private static final Logger LOG = LoggerFactory.getLogger(LinkedShrinkableBloomFilter.class); private long currentSize; private long initSize; private double growRate; - private PartitionedBloomFilter partitionedBloomFilter; + private ElasticBloomFilter partitionedBloomFilter; - private LinkedList bloomFilterNodes = new LinkedList<>(); + private LinkedList bloomFilterNodes = new LinkedList<>(); - public LinkedBloomFilter(PartitionedBloomFilter partitionedBloomFilter, long initSize, double growRate) { + public LinkedShrinkableBloomFilter(ElasticBloomFilter partitionedBloomFilter, long initSize, double growRate) { this.partitionedBloomFilter = partitionedBloomFilter; this.currentSize = initSize; this.initSize = initSize; @@ -55,7 +55,7 @@ public LinkedBloomFilter(PartitionedBloomFilter partitionedBloomFilter, long ini public void add(byte[] content) { synchronized (bloomFilterNodes) { - LinkedBloomFilterNode node; + ShrinkableBloomFilterNode node; if (bloomFilterNodes.size() > 0) { node = bloomFilterNodes.getLast(); if (node.isFull()) { @@ -82,9 +82,9 @@ public void add(byte[] content) { public boolean contains(byte[] content) { synchronized (bloomFilterNodes) { - Iterator iter = bloomFilterNodes.descendingIterator(); + Iterator iter = bloomFilterNodes.descendingIterator(); while (iter.hasNext()) { - LinkedBloomFilterNode node = iter.next(); + ShrinkableBloomFilterNode node = iter.next(); if (node.contains(content)) { return true; } @@ -92,17 +92,17 @@ public boolean contains(byte[] content) { return false; } } - - // for checkpoint and recovery - public LinkedBloomFilter copy() { - synchronized (bloomFilterNodes) { - LinkedBloomFilter bloomFilter = new LinkedBloomFilter(partitionedBloomFilter, initSize, growRate); - for (LinkedBloomFilterNode node : bloomFilterNodes) { - bloomFilter.bloomFilterNodes.add(node.copy()); - } - return bloomFilter; - } - } +// +// // for checkpoint and recovery +// public LinkedBloomFilter copy() { +// synchronized (bloomFilterNodes) { +// LinkedBloomFilter bloomFilter = new LinkedBloomFilter(partitionedBloomFilter, initSize, growRate); +// for (ShrinkableBloomFilterNode node : bloomFilterNodes) { +// bloomFilter.bloomFilterNodes.add(node.copy()); +// } +// return bloomFilter; +// } +// } @VisibleForTesting long getCurrentSize() { @@ -120,7 +120,7 @@ long getInitSize() { } @VisibleForTesting - LinkedList getBloomFilterNodes() { + LinkedList getBloomFilterNodes() { return bloomFilterNodes; } @@ -128,7 +128,7 @@ LinkedList getBloomFilterNodes() { public String toString() { StringBuilder builder = new StringBuilder(); synchronized (bloomFilterNodes) { - for (LinkedBloomFilterNode node : bloomFilterNodes) { + for (ShrinkableBloomFilterNode node : bloomFilterNodes) { if (builder.length() > 0) { builder.append(" -> "); } @@ -145,7 +145,7 @@ void snapshot(DataOutputView outputView) throws IOException { outputView.writeDouble(growRate); bloomFilterNodes.removeIf(node -> { - if (node.getDeleteTS() <= ts) { + if (node.getDealine() <= ts) { partitionedBloomFilter.takeBack(node); return true; } @@ -153,7 +153,7 @@ void snapshot(DataOutputView outputView) throws IOException { }); outputView.writeInt(bloomFilterNodes.size()); - for (LinkedBloomFilterNode node : bloomFilterNodes) { + for (ShrinkableBloomFilterNode node : bloomFilterNodes) { node.snapshot(outputView); } } @@ -164,7 +164,7 @@ void restore(DataInputView source) throws IOException { growRate = source.readDouble(); int len = source.readInt(); for (int i = 0; i < len; ++i) { - LinkedBloomFilterNode node = new LinkedBloomFilterNode(); + ShrinkableBloomFilterNode node = new ShrinkableBloomFilterNode(1, 1, 1); node.restore(source); bloomFilterNodes.add(node); } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNode.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNode.java new file mode 100644 index 0000000000000..8aad7f2a3c6b5 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNode.java @@ -0,0 +1,309 @@ +/* + * 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.flink.streaming.api.operators; + +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import org.apache.flink.shaded.guava18.com.google.common.hash.BloomFilter; +import org.apache.flink.shaded.guava18.com.google.common.hash.Funnels; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Objects; + +/** + * A shrinkable bloom filter node linked each other in {@link LinkedShrinkableBloomFilter} to avoid data skewed. + */ +public class ShrinkableBloomFilterNode { + + private static final Logger LOG = LoggerFactory.getLogger(ShrinkableBloomFilterNode.class); + + private static final int DEFAULT_UNITS_NUM = 8; + + private int capacity; + private double fpp; + private long size; + private long rawTtl; + private long ttl; + private long deleteTS = Long.MAX_VALUE; + + public BloomFilterUnit[] bloomFilterUnits; + + public ShrinkableBloomFilterNode(int capacity, double fpp, long ttl) { + this.capacity = capacity; + this.fpp = fpp; + this.rawTtl = ttl; + this.ttl = ttl; + this.size = 0; + + bloomFilterUnits = new BloomFilterUnit[DEFAULT_UNITS_NUM]; + + final int unitCapacity = computeUnitCapacity(capacity, bloomFilterUnits.length); + + for (int i = 0; i < bloomFilterUnits.length; ++i) { + bloomFilterUnits[i] = new BloomFilterUnit(unitCapacity, fpp); + } + } + + public boolean isFull() { + return size >= capacity; + } + + public long getCapacity() { + int capacity = 0; + for (BloomFilterUnit bloomFilterUnit : bloomFilterUnits) { + capacity += bloomFilterUnit.size(); + } + return capacity; + } + + public long getSize() { + int size = 0; + for (BloomFilterUnit bloomFilterUnit : bloomFilterUnits) { + size += bloomFilterUnit.size(); + } + return size; + } + + public long getTTL() { + return ttl; + } + + public double getFpp() { + return fpp; + } + + public void reSetTtl() { + this.ttl = this.rawTtl; + this.deleteTS = Long.MAX_VALUE; + } + + public long getDealine() { + return deleteTS; + } + + public void add(byte[] content) { + + int unitIndex = content.hashCode() % bloomFilterUnits.length; + + BloomFilterUnit bloomFilterUnit = bloomFilterUnits[unitIndex]; + + bloomFilterUnit.add(content); + } + + public boolean contains(byte[] content) { + + int unitIndex = content.hashCode() % bloomFilterUnits.length; + + BloomFilterUnit bloomFilterUnit = bloomFilterUnits[unitIndex]; + + return bloomFilterUnit.contains(content); + } + + /** + * Shrinks the BF set to reduce memory consumed. + */ + public void shrink() { + if (shrinkable()) { + int left = 0; + int right = bloomFilterUnits.length - 1; + + BloomFilterUnit[] newBloomFilterUnits = new BloomFilterUnit[bloomFilterUnits.length >> 1]; + int index = 0; + while(left < right) { + BloomFilterUnit leftUnit = bloomFilterUnits[left]; + BloomFilterUnit rightUnit = bloomFilterUnits[right]; + + // merging + leftUnit.merge(rightUnit); + + newBloomFilterUnits[index] = leftUnit; + + ++left; + --right; + ++index; + } + + bloomFilterUnits = newBloomFilterUnits; + } + } + + /** + * Check whether this Node is shrinkable, + */ + private boolean shrinkable() { + + // we can't shrink it when it has only one unit. + if (bloomFilterUnits.length <= 1) { + return false; + } + + for (BloomFilterUnit bloomFilterUnit : bloomFilterUnits) { + int capacity = bloomFilterUnit.capacity(); + int size = bloomFilterUnit.size(); + + if (size > (capacity >> 1)) { + return false; + } + } + return true; + } + + private int computeUnitCapacity(int capacity, int units) { + return 0; + } + + @Override + public String toString() { + return String.format("{c:%d s:%d}", capacity, size); + } + + @Override + public boolean equals(Object obj) { + if (obj instanceof ShrinkableBloomFilterNode) { + ShrinkableBloomFilterNode other = (ShrinkableBloomFilterNode) obj; + if (other.capacity == this.capacity + && other.size == this.size + && other.rawTtl == this.rawTtl + && other.fpp == this.fpp + && other.bloomFilterUnits.length == bloomFilterUnits.length) { + + for (int i = 0; i < bloomFilterUnits.length; ++i) { + if(!Objects.equals(bloomFilterUnits[i], other.bloomFilterUnits[i])) { + return false; + } + } + return true; + } + } + return false; + } + + void snapshot(DataOutputView outputView) throws IOException { + outputView.writeLong(capacity); + outputView.writeLong(rawTtl); + if (deleteTS == Long.MAX_VALUE) { + outputView.writeLong(ttl); //rest ttl + } else { + outputView.writeLong(deleteTS - System.currentTimeMillis()); //rest ttl + } + outputView.writeLong(deleteTS); + outputView.writeLong(size); + outputView.writeDouble(fpp); + + int unitNum = bloomFilterUnits.length; + outputView.writeInt(unitNum); + + for (int i = 0; i < unitNum; ++i) { + BloomFilterUnit bloomFilterUnit = bloomFilterUnits[i]; + bloomFilterUnit.snapshot(outputView); + } + } + + void restore(DataInputView source) throws IOException { + capacity = source.readInt(); + rawTtl = source.readLong(); + ttl = source.readLong(); + deleteTS = source.readLong(); + if (rawTtl != ttl) { + deleteTS = System.currentTimeMillis() + ttl; + } + size = source.readLong(); + fpp = source.readDouble(); + + int unitNum = source.readInt(); + bloomFilterUnits = new BloomFilterUnit[unitNum]; + + for (int i = 0; i < unitNum; ++i) { + BloomFilterUnit bloomFilterUnit = new BloomFilterUnit(capacity, fpp); + bloomFilterUnit.restore(source); + bloomFilterUnits[i] = bloomFilterUnit; + } + } + + /** + * + */ + private class BloomFilterUnit { + + private BloomFilter bloomFilter; + + private int capacity; + + private int size; + + public BloomFilterUnit(int capacity, double fpp) { + + bloomFilter = BloomFilter.create( + Funnels.byteArrayFunnel(), + capacity, + fpp); + + this.capacity = capacity; + size = 0; + } + + public void add(byte[] content) { + if (bloomFilter.put(content)) { + ++size; + } + } + + public boolean contains(byte[] content) { + return bloomFilter.mightContain(content); + } + + public void merge(BloomFilterUnit bloomFilterUnit) { + bloomFilter.putAll(bloomFilterUnit.bloomFilter); + size += bloomFilterUnit.size; + } + + public boolean full() { + return size >= capacity; + } + + public int capacity() { + return capacity; + } + + public int size() { + return size; + } + + public void snapshot(DataOutputView outputView) { +// ByteArrayOutputStream out = new ByteArrayOutputStream(); +// bloomFilter.writeTo(out); +// byte[] bytes = out.toByteArray(); +// outputView.writeInt(bytes.length); +// outputView.write(bytes); + } + + public void restore(DataInputView source) { +// int byteLen = source.readInt(); +// byte[] bytes = new byte[byteLen]; +// source.read(bytes, 0, byteLen); +// ByteArrayInputStream input = new ByteArrayInputStream(bytes); + +// bloomFilter = BloomFilter.readFrom(input, Funnels.byteArrayFunnel()); + } + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateContext.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateContext.java index e093d7c6b624e..d9f6d6f08b96b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateContext.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateContext.java @@ -51,7 +51,7 @@ public interface StreamOperatorStateContext { */ InternalTimeServiceManager internalTimerServiceManager(); - PartitionedBloomFilterManager bloomFilterStateManager(); + ElasticBloomFilterManager bloomFilterStateManager(); /** * Returns an iterable to obtain input streams for previously stored operator state partitions that are assigned to diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java index bcdac02134e21..b43acd8eee79d 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java @@ -126,7 +126,7 @@ public StreamOperatorStateContext streamOperatorStateContext( CloseableIterable rawKeyedStateInputs = null; CloseableIterable rawOperatorStateInputs = null; InternalTimeServiceManager timeServiceManager; - PartitionedBloomFilterManager bloomfilterStateManager; + ElasticBloomFilterManager bloomfilterStateManager; try { @@ -230,7 +230,7 @@ protected InternalTimeServiceManager internalTimeServiceManager( return timeServiceManager; } - protected PartitionedBloomFilterManager bloomfilterStateManager( + protected ElasticBloomFilterManager bloomfilterStateManager( AbstractKeyedStateBackend keyedStatedBackend, KeyContext keyContext, //the operator Iterable rawKeyedStates) throws Exception { @@ -241,7 +241,7 @@ protected PartitionedBloomFilterManager bloomfilterStateManager( final KeyGroupRange keyGroupRange = keyedStatedBackend.getKeyGroupRange(); - PartitionedBloomFilterManager bloomFilterManager = new PartitionedBloomFilterManager( + ElasticBloomFilterManager bloomFilterManager = new ElasticBloomFilterManager( keyContext, keyedStatedBackend.getKeySerializer(), keyedStatedBackend.getNumberOfKeyGroups(), @@ -583,7 +583,7 @@ private static class StreamOperatorStateContextImpl implements StreamOperatorSta private final OperatorStateBackend operatorStateBackend; private final AbstractKeyedStateBackend keyedStateBackend; private final InternalTimeServiceManager internalTimeServiceManager; - private final PartitionedBloomFilterManager bloomFilterStateManager; + private final ElasticBloomFilterManager bloomFilterStateManager; private final CloseableIterable rawOperatorStateInputs; private final CloseableIterable rawKeyedStateInputs; @@ -593,7 +593,7 @@ private static class StreamOperatorStateContextImpl implements StreamOperatorSta OperatorStateBackend operatorStateBackend, AbstractKeyedStateBackend keyedStateBackend, InternalTimeServiceManager internalTimeServiceManager, - PartitionedBloomFilterManager bloomFilterStateManager, + ElasticBloomFilterManager bloomFilterStateManager, CloseableIterable rawOperatorStateInputs, CloseableIterable rawKeyedStateInputs) { @@ -627,7 +627,7 @@ public InternalTimeServiceManager internalTimerServiceManager() { } @Override - public PartitionedBloomFilterManager bloomFilterStateManager() { + public ElasticBloomFilterManager bloomFilterStateManager() { return bloomFilterStateManager; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java index 0b7e0297a03d1..ed2bd5655d96c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java @@ -171,7 +171,7 @@ public MapState getMapState(MapStateDescriptor statePro return keyedStateStore.getMapState(stateProperties); } - public PartitionedBloomFilter getPartitionedBloomFilter(PartitionedBloomFilterDescriptor descriptor) { + public ElasticBloomFilter getPartitionedBloomFilter(PartitionedBloomFilterDescriptor descriptor) { descriptor.initializeSerializerUnlessSet(getExecutionConfig()); return operator.getBloomFilterStateManager().getOrCreateBloomFilterState(descriptor); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNodeTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNodeTest.java index 8a190d112cb49..174b14b466b16 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNodeTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNodeTest.java @@ -12,14 +12,14 @@ import java.util.concurrent.TimeUnit; /** - * {@link LinkedBloomFilterNode} unit tests. + * {@link ElasticBloomFilterNode} unit tests. */ public class LinkedBloomFilterNodeTest { @Test public void basicTest() throws InterruptedException { - LinkedBloomFilterNode node = new LinkedBloomFilterNode(100, 0.01, 1000); + ElasticBloomFilterNode node = new ElasticBloomFilterNode(100, 0.01, 1000); Assert.assertEquals(Long.MAX_VALUE, node.getDeleteTS()); @@ -35,7 +35,7 @@ public void basicTest() throws InterruptedException { @Test public void testSnapshotAndRestore() throws InterruptedException, IOException { - LinkedBloomFilterNode node1 = new LinkedBloomFilterNode(100, 0.01, 1000); + ElasticBloomFilterNode node1 = new ElasticBloomFilterNode(100, 0.01, 1000); for (int i = 0; i < 100; ++i) { node1.add(String.valueOf(i).getBytes()); Assert.assertTrue(node1.contains(String.valueOf(i).getBytes())); @@ -48,7 +48,7 @@ public void testSnapshotAndRestore() throws InterruptedException, IOException { byte[] outputBytes = outputStream.toByteArray(); - LinkedBloomFilterNode node2 = new LinkedBloomFilterNode(); + ElasticBloomFilterNode node2 = new ElasticBloomFilterNode(); node2.restore(new DataInputViewStreamWrapper(new ByteArrayInputStream(outputBytes))); Assert.assertEquals(node1.getCapacity(), node2.getCapacity()); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterTest.java index 82fd2c34f8ca3..3462d4fee7a94 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterTest.java @@ -16,7 +16,7 @@ import java.util.concurrent.TimeUnit; /** - * {@link LinkedBloomFilter} unit tests. + * {@link LinkedShrinkableBloomFilter} unit tests. */ public class LinkedBloomFilterTest { @@ -27,7 +27,7 @@ public void basicTest() throws InterruptedException { private static final long serialVersionUID = 1L; }; - PartitionedBloomFilter partitionedBloomFilter = new PartitionedBloomFilter( + ElasticBloomFilter partitionedBloomFilter = new ElasticBloomFilter( new StringSerializer(), new IntSerializer(), 10, @@ -41,9 +41,9 @@ public void basicTest() throws InterruptedException { 2.0 ); - LinkedBloomFilter linkedBloomFilter = new LinkedBloomFilter(partitionedBloomFilter, 1000, 2); + LinkedShrinkableBloomFilter linkedBloomFilter = new LinkedShrinkableBloomFilter(partitionedBloomFilter, 1000, 2); - List nodes = linkedBloomFilter.getBloomFilterNodes(); + List nodes = linkedBloomFilter.getBloomFilterNodes(); Assert.assertEquals(0, nodes.size()); for (int i = 0; i < 1000; ++i) { @@ -63,7 +63,7 @@ public void testSnapshotAndRestore() throws InterruptedException, IOException { private static final long serialVersionUID = 1L; }; - PartitionedBloomFilter partitionedBloomFilter = new PartitionedBloomFilter( + ElasticBloomFilter partitionedBloomFilter = new ElasticBloomFilter( new StringSerializer(), new IntSerializer(), 10, @@ -77,9 +77,9 @@ public void testSnapshotAndRestore() throws InterruptedException, IOException { 2.0 ); - LinkedBloomFilter linkedBloomFilter = new LinkedBloomFilter(partitionedBloomFilter, 1000, 2); + LinkedShrinkableBloomFilter linkedBloomFilter = new LinkedShrinkableBloomFilter(partitionedBloomFilter, 1000, 2); - List nodes = linkedBloomFilter.getBloomFilterNodes(); + List nodes = linkedBloomFilter.getBloomFilterNodes(); Assert.assertEquals(0, nodes.size()); for (int i = 0; i < 1000; ++i) { @@ -98,7 +98,7 @@ public void testSnapshotAndRestore() throws InterruptedException, IOException { byte[] outputBytes = outputStream.toByteArray(); - LinkedBloomFilter linkedBloomFilter2 = new LinkedBloomFilter(partitionedBloomFilter, 1000, 2); + LinkedShrinkableBloomFilter linkedBloomFilter2 = new LinkedShrinkableBloomFilter(partitionedBloomFilter, 1000, 2); linkedBloomFilter2.restore(new DataInputViewStreamWrapper(new ByteArrayInputStream(outputBytes))); @@ -106,7 +106,7 @@ public void testSnapshotAndRestore() throws InterruptedException, IOException { Assert.assertEquals(linkedBloomFilter.getInitSize(), linkedBloomFilter2.getInitSize()); Assert.assertEquals(String.valueOf(linkedBloomFilter.getGrowRate()), String.valueOf(linkedBloomFilter2.getGrowRate())); - List nodes2 = linkedBloomFilter2.getBloomFilterNodes(); + List nodes2 = linkedBloomFilter2.getBloomFilterNodes(); Assert.assertEquals(nodes.size(), nodes2.size()); for (int i = 0; i < nodes.size(); ++i) { @@ -120,9 +120,9 @@ public void testSnapshotAndRestore() throws InterruptedException, IOException { linkedBloomFilter.snapshot(outputViewStreamWrapper); outputBytes = outputStream.toByteArray(); - LinkedBloomFilter linkedBloomFilter3 = new LinkedBloomFilter(partitionedBloomFilter, 1000, 2); + LinkedShrinkableBloomFilter linkedBloomFilter3 = new LinkedShrinkableBloomFilter(partitionedBloomFilter, 1000, 2); linkedBloomFilter3.restore(new DataInputViewStreamWrapper(new ByteArrayInputStream(outputBytes))); - List nodes3 = linkedBloomFilter3.getBloomFilterNodes(); + List nodes3 = linkedBloomFilter3.getBloomFilterNodes(); Assert.assertEquals(1, nodes.size()); Assert.assertEquals(1, nodes3.size()); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterManagerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterManagerTest.java index a4a77d5d53e39..aab290d751575 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterManagerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterManagerTest.java @@ -15,7 +15,7 @@ import java.util.Random; /** - * {@link PartitionedBloomFilterManager} unit tests. + * {@link ElasticBloomFilterManager} unit tests. */ public class PartitionedBloomFilterManagerTest { @@ -36,13 +36,13 @@ public Object getCurrentKey() { } }; - PartitionedBloomFilterManager partitionedBloomFilterManager = new PartitionedBloomFilterManager( + ElasticBloomFilterManager partitionedBloomFilterManager = new ElasticBloomFilterManager( keyContext, TypeInformation.of(String.class).createSerializer(new ExecutionConfig()), 10, new KeyGroupRange(0, 9)); - PartitionedBloomFilter partitionedBloomFilter = partitionedBloomFilterManager.getOrCreateBloomFilterState( + ElasticBloomFilter partitionedBloomFilter = partitionedBloomFilterManager.getOrCreateBloomFilterState( new PartitionedBloomFilterDescriptor( "test-bf", TypeInformation.of(Integer.class).createSerializer(new ExecutionConfig()), @@ -72,7 +72,7 @@ public Object getCurrentKey() { } }; - PartitionedBloomFilterManager partitionedBloomFilterManager = new PartitionedBloomFilterManager( + ElasticBloomFilterManager partitionedBloomFilterManager = new ElasticBloomFilterManager( keyContext, TypeInformation.of(String.class).createSerializer(new ExecutionConfig()), 10, @@ -88,8 +88,8 @@ public Object getCurrentKey() { TypeInformation.of(Integer.class).createSerializer(new ExecutionConfig()), 10_000, 0.01, 60000); - PartitionedBloomFilter partitionedBloomFilter1 = partitionedBloomFilterManager.getOrCreateBloomFilterState(desc1); - PartitionedBloomFilter partitionedBloomFilter2 = partitionedBloomFilterManager.getOrCreateBloomFilterState(desc2); + ElasticBloomFilter partitionedBloomFilter1 = partitionedBloomFilterManager.getOrCreateBloomFilterState(desc1); + ElasticBloomFilter partitionedBloomFilter2 = partitionedBloomFilterManager.getOrCreateBloomFilterState(desc2); String[] keys1 = new String[10_000]; for (int i = 0; i < 10_000; ++i) { @@ -117,7 +117,7 @@ public Object getCurrentKey() { // restore ByteArrayInputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray()); DataInputViewStreamWrapper inputViewStreamWrapper = new DataInputViewStreamWrapper(inputStream); - PartitionedBloomFilterManager partitionedBloomFilterManager2 = new PartitionedBloomFilterManager( + ElasticBloomFilterManager partitionedBloomFilterManager2 = new ElasticBloomFilterManager( keyContext, TypeInformation.of(String.class).createSerializer(new ExecutionConfig()), 10, @@ -128,8 +128,8 @@ public Object getCurrentKey() { } // valid - PartitionedBloomFilter partitionedBloomFilter3 = partitionedBloomFilterManager2.getOrCreateBloomFilterState(desc1); - PartitionedBloomFilter partitionedBloomFilter4 = partitionedBloomFilterManager2.getOrCreateBloomFilterState(desc2); + ElasticBloomFilter partitionedBloomFilter3 = partitionedBloomFilterManager2.getOrCreateBloomFilterState(desc1); + ElasticBloomFilter partitionedBloomFilter4 = partitionedBloomFilterManager2.getOrCreateBloomFilterState(desc2); for (int i = 0; i < 10_000; ++i) { String key = keys1[i]; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterTest.java index 548d6c7112b73..e0d335ba5a3c4 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterTest.java @@ -15,7 +15,7 @@ import java.util.Random; /** - * {@link PartitionedBloomFilter} unit tests. + * {@link ElasticBloomFilter} unit tests. */ public class PartitionedBloomFilterTest { @@ -36,7 +36,7 @@ public Object getCurrentKey() { } }; - PartitionedBloomFilter partitionedBloomFilter = new PartitionedBloomFilter( + ElasticBloomFilter partitionedBloomFilter = new ElasticBloomFilter( new StringSerializer(), new IntSerializer(), 10, @@ -50,7 +50,7 @@ public Object getCurrentKey() { 2.0 ); - LinkedBloomFilter[] linkedBloomFilters = partitionedBloomFilter.getLinkedBloomFilters(); + LinkedShrinkableBloomFilter[] linkedBloomFilters = partitionedBloomFilter.getLinkedBloomFilters(); Assert.assertEquals(10, linkedBloomFilters.length); for (int i = 0; i < 10; ++i) { Assert.assertNull(linkedBloomFilters[i]); @@ -85,7 +85,7 @@ public Object getCurrentKey() { } }; - PartitionedBloomFilter partitionedBloomFilter = new PartitionedBloomFilter( + ElasticBloomFilter partitionedBloomFilter = new ElasticBloomFilter( new StringSerializer(), new IntSerializer(), 10, @@ -115,7 +115,7 @@ public Object getCurrentKey() { partitionedBloomFilter.snapshotStateForKeyGroup(outputViewStreamWrapper, i); } - PartitionedBloomFilter partitionedBloomFilter2 = new PartitionedBloomFilter( + ElasticBloomFilter partitionedBloomFilter2 = new ElasticBloomFilter( new StringSerializer(), new IntSerializer(), 10, diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index fe85b9602abd2..9a5c705020372 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -101,7 +101,7 @@ import org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer; import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; import org.apache.flink.streaming.api.operators.Output; -import org.apache.flink.streaming.api.operators.PartitionedBloomFilterManager; +import org.apache.flink.streaming.api.operators.ElasticBloomFilterManager; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperatorStateContext; import org.apache.flink.streaming.api.operators.StreamSource; @@ -1115,7 +1115,7 @@ public InternalTimeServiceManager internalTimerServiceManager() { } @Override - public PartitionedBloomFilterManager bloomFilterStateManager() { + public ElasticBloomFilterManager bloomFilterStateManager() { return context.bloomFilterStateManager(); } From e813d05dd181f14450310f703af129bd9fdb01a3 Mon Sep 17 00:00:00 2001 From: sihuazhou Date: Fri, 4 May 2018 16:21:08 +0800 Subject: [PATCH 3/5] add tests for ShrinkableBloomFilterNode. --- .../operators/ShrinkableBloomFilterNode.java | 61 ++++++-- .../operators/LinkedBloomFilterNodeTest.java | 118 +++++++------- .../api/operators/LinkedBloomFilterTest.java | 8 +- .../ShrinkableBloomFilterNodeTest.java | 146 ++++++++++++++++++ 4 files changed, 256 insertions(+), 77 deletions(-) create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNodeTest.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNode.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNode.java index 8aad7f2a3c6b5..95ea5623ad88e 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNode.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNode.java @@ -27,6 +27,8 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.Objects; @@ -241,9 +243,9 @@ void restore(DataInputView source) throws IOException { } /** - * + * The bloom filter unit to store records. */ - private class BloomFilterUnit { + static class BloomFilterUnit { private BloomFilter bloomFilter; @@ -251,6 +253,10 @@ private class BloomFilterUnit { private int size; + BloomFilterUnit() { + + } + public BloomFilterUnit(int capacity, double fpp) { bloomFilter = BloomFilter.create( @@ -289,21 +295,48 @@ public int size() { return size; } - public void snapshot(DataOutputView outputView) { -// ByteArrayOutputStream out = new ByteArrayOutputStream(); -// bloomFilter.writeTo(out); -// byte[] bytes = out.toByteArray(); -// outputView.writeInt(bytes.length); -// outputView.write(bytes); + public void snapshot(DataOutputView outputView) throws IOException { + outputView.writeInt(capacity); + outputView.writeInt(size); + + try(ByteArrayOutputStream out = new ByteArrayOutputStream()) { + bloomFilter.writeTo(out); + byte[] bytes = out.toByteArray(); + outputView.writeInt(bytes.length); + outputView.write(bytes); + } + } + + public void restore(DataInputView source) throws IOException { + this.capacity = source.readInt(); + this.size = source.readInt(); + + int byteLen = source.readInt(); + byte[] bytes = new byte[byteLen]; + source.read(bytes, 0, byteLen); + try(ByteArrayInputStream input = new ByteArrayInputStream(bytes)) { + bloomFilter = BloomFilter.readFrom(input, Funnels.byteArrayFunnel()); + } } - public void restore(DataInputView source) { -// int byteLen = source.readInt(); -// byte[] bytes = new byte[byteLen]; -// source.read(bytes, 0, byteLen); -// ByteArrayInputStream input = new ByteArrayInputStream(bytes); + @Override + public boolean equals(Object obj) { + + if (obj == this) { + return true; + } + + if (obj == null) { + return false; + } + + if (!(obj instanceof BloomFilterUnit)) { + return false; + } + + BloomFilterUnit other = (BloomFilterUnit) obj; -// bloomFilter = BloomFilter.readFrom(input, Funnels.byteArrayFunnel()); + return this.size == other.size && this.capacity == other.capacity && this.bloomFilter.equals(other.bloomFilter); } } } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNodeTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNodeTest.java index 174b14b466b16..ec914f245bc28 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNodeTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNodeTest.java @@ -1,59 +1,59 @@ -package org.apache.flink.streaming.api.operators; - -import org.apache.flink.core.memory.DataInputViewStreamWrapper; -import org.apache.flink.core.memory.DataOutputViewStreamWrapper; - -import org.junit.Assert; -import org.junit.Test; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.util.concurrent.TimeUnit; - -/** - * {@link ElasticBloomFilterNode} unit tests. - */ -public class LinkedBloomFilterNodeTest { - - @Test - public void basicTest() throws InterruptedException { - - ElasticBloomFilterNode node = new ElasticBloomFilterNode(100, 0.01, 1000); - - Assert.assertEquals(Long.MAX_VALUE, node.getDeleteTS()); - - for (int i = 0; i < 100; ++i) { - node.add(String.valueOf(i).getBytes()); - Assert.assertTrue(node.contains(String.valueOf(i).getBytes())); - } - Assert.assertTrue(node.isFull()); - TimeUnit.MILLISECONDS.sleep(1000); - Assert.assertNotEquals(Long.MAX_VALUE, node.getDeleteTS()); - } - - @Test - public void testSnapshotAndRestore() throws InterruptedException, IOException { - - ElasticBloomFilterNode node1 = new ElasticBloomFilterNode(100, 0.01, 1000); - for (int i = 0; i < 100; ++i) { - node1.add(String.valueOf(i).getBytes()); - Assert.assertTrue(node1.contains(String.valueOf(i).getBytes())); - } - - ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - DataOutputViewStreamWrapper outputViewStreamWrapper = new DataOutputViewStreamWrapper(outputStream); - - node1.snapshot(outputViewStreamWrapper); - - byte[] outputBytes = outputStream.toByteArray(); - - ElasticBloomFilterNode node2 = new ElasticBloomFilterNode(); - node2.restore(new DataInputViewStreamWrapper(new ByteArrayInputStream(outputBytes))); - - Assert.assertEquals(node1.getCapacity(), node2.getCapacity()); - Assert.assertEquals(String.valueOf(node1.getFpp()), String.valueOf(node2.getFpp())); - Assert.assertEquals(node1.getSize(), node2.getSize()); - Assert.assertEquals(node1.getBloomFilter(), node2.getBloomFilter()); - } -} +//package org.apache.flink.streaming.api.operators; +// +//import org.apache.flink.core.memory.DataInputViewStreamWrapper; +//import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +// +//import org.junit.Assert; +//import org.junit.Test; +// +//import java.io.ByteArrayInputStream; +//import java.io.ByteArrayOutputStream; +//import java.io.IOException; +//import java.util.concurrent.TimeUnit; +// +///** +// * {@link ElasticBloomFilterNode} unit tests. +// */ +//public class LinkedBloomFilterNodeTest { +// +// @Test +// public void basicTest() throws InterruptedException { +// +// ElasticBloomFilterNode node = new ElasticBloomFilterNode(100, 0.01, 1000); +// +// Assert.assertEquals(Long.MAX_VALUE, node.getDeleteTS()); +// +// for (int i = 0; i < 100; ++i) { +// node.add(String.valueOf(i).getBytes()); +// Assert.assertTrue(node.contains(String.valueOf(i).getBytes())); +// } +// Assert.assertTrue(node.isFull()); +// TimeUnit.MILLISECONDS.sleep(1000); +// Assert.assertNotEquals(Long.MAX_VALUE, node.getDeleteTS()); +// } +// +// @Test +// public void testSnapshotAndRestore() throws InterruptedException, IOException { +// +// ElasticBloomFilterNode node1 = new ElasticBloomFilterNode(100, 0.01, 1000); +// for (int i = 0; i < 100; ++i) { +// node1.add(String.valueOf(i).getBytes()); +// Assert.assertTrue(node1.contains(String.valueOf(i).getBytes())); +// } +// +// ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); +// DataOutputViewStreamWrapper outputViewStreamWrapper = new DataOutputViewStreamWrapper(outputStream); +// +// node1.snapshot(outputViewStreamWrapper); +// +// byte[] outputBytes = outputStream.toByteArray(); +// +// ElasticBloomFilterNode node2 = new ElasticBloomFilterNode(); +// node2.restore(new DataInputViewStreamWrapper(new ByteArrayInputStream(outputBytes))); +// +// Assert.assertEquals(node1.getCapacity(), node2.getCapacity()); +// Assert.assertEquals(String.valueOf(node1.getFpp()), String.valueOf(node2.getFpp())); +// Assert.assertEquals(node1.getSize(), node2.getSize()); +// Assert.assertEquals(node1.getBloomFilter(), node2.getBloomFilter()); +// } +//} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterTest.java index 3462d4fee7a94..c14265b38ae7c 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterTest.java @@ -43,7 +43,7 @@ public void basicTest() throws InterruptedException { LinkedShrinkableBloomFilter linkedBloomFilter = new LinkedShrinkableBloomFilter(partitionedBloomFilter, 1000, 2); - List nodes = linkedBloomFilter.getBloomFilterNodes(); + List nodes = linkedBloomFilter.getBloomFilterNodes(); Assert.assertEquals(0, nodes.size()); for (int i = 0; i < 1000; ++i) { @@ -79,7 +79,7 @@ public void testSnapshotAndRestore() throws InterruptedException, IOException { LinkedShrinkableBloomFilter linkedBloomFilter = new LinkedShrinkableBloomFilter(partitionedBloomFilter, 1000, 2); - List nodes = linkedBloomFilter.getBloomFilterNodes(); + List nodes = linkedBloomFilter.getBloomFilterNodes(); Assert.assertEquals(0, nodes.size()); for (int i = 0; i < 1000; ++i) { @@ -106,7 +106,7 @@ public void testSnapshotAndRestore() throws InterruptedException, IOException { Assert.assertEquals(linkedBloomFilter.getInitSize(), linkedBloomFilter2.getInitSize()); Assert.assertEquals(String.valueOf(linkedBloomFilter.getGrowRate()), String.valueOf(linkedBloomFilter2.getGrowRate())); - List nodes2 = linkedBloomFilter2.getBloomFilterNodes(); + List nodes2 = linkedBloomFilter2.getBloomFilterNodes(); Assert.assertEquals(nodes.size(), nodes2.size()); for (int i = 0; i < nodes.size(); ++i) { @@ -122,7 +122,7 @@ public void testSnapshotAndRestore() throws InterruptedException, IOException { LinkedShrinkableBloomFilter linkedBloomFilter3 = new LinkedShrinkableBloomFilter(partitionedBloomFilter, 1000, 2); linkedBloomFilter3.restore(new DataInputViewStreamWrapper(new ByteArrayInputStream(outputBytes))); - List nodes3 = linkedBloomFilter3.getBloomFilterNodes(); + List nodes3 = linkedBloomFilter3.getBloomFilterNodes(); Assert.assertEquals(1, nodes.size()); Assert.assertEquals(1, nodes3.size()); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNodeTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNodeTest.java new file mode 100644 index 0000000000000..442a938256745 --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNodeTest.java @@ -0,0 +1,146 @@ +/* + * 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.flink.streaming.api.operators; + +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.junit.Assert; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +/** + * Tests to guard {@link ShrinkableBloomFilterNode}. + */ +public class ShrinkableBloomFilterNodeTest { + + @Test + public void testBasicFunctionalityForBloomFilterUnit() { + ShrinkableBloomFilterNode.BloomFilterUnit unit = new ShrinkableBloomFilterNode.BloomFilterUnit(10000, 0.02); + + Assert.assertEquals(10000, unit.capacity()); + Assert.assertEquals(0, unit.size()); + Assert.assertFalse(unit.full()); + + for (int i = 0; i < 10000; ++i) { + Assert.assertFalse(unit.contains(String.valueOf(i).getBytes())); + } + + for (int i = 0; i < 10000; ++i) { + unit.add(String.valueOf(i).getBytes()); + } + + for (int i = 0; i < 10000; ++i) { + Assert.assertTrue(unit.contains(String.valueOf(i).getBytes())); + } + } + + @Test + public void testMergeBloomFilterUnit() { + + ShrinkableBloomFilterNode.BloomFilterUnit unit1 = new ShrinkableBloomFilterNode.BloomFilterUnit(10000, 0.02); + ShrinkableBloomFilterNode.BloomFilterUnit unit2 = new ShrinkableBloomFilterNode.BloomFilterUnit(10000, 0.02); + ShrinkableBloomFilterNode.BloomFilterUnit unit3 = new ShrinkableBloomFilterNode.BloomFilterUnit(20000, 0.02); + + for (int i = 0; i < 5000; ++i) { + unit1.add(String.valueOf(i).getBytes()); + } + + for (int i = 5000; i < 10000; ++i) { + unit2.add(String.valueOf(i).getBytes()); + } + + // before merging with unit2 + for (int i = 0; i < 5000; ++i) { + Assert.assertTrue(unit1.contains(String.valueOf(i).getBytes())); + } + + List records5000To10000 = new ArrayList<>(); + for (int i = 5000; i < 10000; ++i) { + records5000To10000.add(String.valueOf(i).getBytes()); + } + + verifyBloomFilterWithFpp(unit1, records5000To10000, false, 0.02); + + // after merging with unit1 + unit1.merge(unit2); + for (int i = 0; i < 10000; ++i) { + Assert.assertTrue(unit1.contains(String.valueOf(i).getBytes())); + } + + try { + unit2.merge(unit3); + Assert.fail(); // failed because the size of unit2 and unit3 is different. + } catch (Exception expected) { + + } + } + + @Test + public void testSnapshotAndRestoreForBloomFilterUnit() throws Exception { + ShrinkableBloomFilterNode.BloomFilterUnit unit = new ShrinkableBloomFilterNode.BloomFilterUnit(10000, 0.02); + + for (int i = 0; i < 5000; ++i) { + unit.add(String.valueOf(i).getBytes()); + } + + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(1024); + DataOutputViewStreamWrapper outputViewStreamWrapper = new DataOutputViewStreamWrapper(outputStream); + unit.snapshot(outputViewStreamWrapper); + + byte[] snapshottedBytes = outputStream.toByteArray(); + + ByteArrayInputStream inputStream = new ByteArrayInputStream(snapshottedBytes); + DataInputViewStreamWrapper inputViewStreamWrapper = new DataInputViewStreamWrapper(inputStream); + + ShrinkableBloomFilterNode.BloomFilterUnit restoredUnit = new ShrinkableBloomFilterNode.BloomFilterUnit(); + restoredUnit.restore(inputViewStreamWrapper); + + Assert.assertEquals(unit, restoredUnit); + } + + @Test + public void testBasicFunctionalityForShrinkableBloomFilterNode() { + + } + + @Test + public void testSnapshotAndRestoreForShrinkableBloomFilterNode() { + + } + + @Test + public void testShrinking() { + + } + + private boolean verifyBloomFilterWithFpp(ShrinkableBloomFilterNode.BloomFilterUnit unit, Collection records, boolean target, double expectedFpp) { + int count = 0; + for (byte[] record : records) { + if (unit.contains(record) != target) { + ++count; + } + } + return ((double) count / records.size()) <= expectedFpp; + } +} From adf7d27f9d29a67704176094c30a3c73f9d17810 Mon Sep 17 00:00:00 2001 From: sihuazhou Date: Sun, 10 Jun 2018 23:37:12 +0800 Subject: [PATCH 4/5] tmp. --- .../api/operators/ElasticBloomFilter.java | 22 +-- ...mFilter.java => LinkedTolerantFilter.java} | 81 +++++--- .../operators/ShrinkableBloomFilterNode.java | 108 +++++----- .../api/operators/TolerantFilterNode.java | 49 +++++ .../api/operators/LinkedBloomFilterTest.java | 131 ------------ .../operators/LinkedTolerantFilterTest.java | 187 ++++++++++++++++++ .../operators/PartitionedBloomFilterTest.java | 2 +- .../ShrinkableBloomFilterNodeTest.java | 81 +++++++- 8 files changed, 436 insertions(+), 225 deletions(-) rename flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/{LinkedShrinkableBloomFilter.java => LinkedTolerantFilter.java} (62%) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TolerantFilterNode.java delete mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterTest.java create mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedTolerantFilterTest.java diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticBloomFilter.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticBloomFilter.java index b17ba7e597b2e..174a99ab2b522 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticBloomFilter.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticBloomFilter.java @@ -53,7 +53,7 @@ public class ElasticBloomFilter { private final int totalKeyGroups; private final KeyGroupsList localKeyGroupRange; private final KeyContext keyContext; - private LinkedShrinkableBloomFilter[] linkedBloomFilters; + private LinkedTolerantFilter[] linkedBloomFilters; private final int localKeyGroupRangeStartIdx; private long totalMemSize; @@ -99,7 +99,7 @@ public ElasticBloomFilter(TypeSerializer keySerializer, this.keyContext = keyContext; this.totalKeyGroups = totalKeyGroups; this.localKeyGroupRange = localKeyGroupRange; - this.linkedBloomFilters = new LinkedShrinkableBloomFilter[localKeyGroupRange.getNumberOfKeyGroups()]; + this.linkedBloomFilters = new LinkedTolerantFilter[localKeyGroupRange.getNumberOfKeyGroups()]; // find the starting index of the local key-group range int startIdx = Integer.MAX_VALUE; @@ -118,9 +118,9 @@ public void add(V content) { int keyGroupIndex = KeyGroupRangeAssignment.assignToKeyGroup(keyContext.getCurrentKey(), totalKeyGroups); int index = getIndexForKeyGroup(keyGroupIndex); - LinkedShrinkableBloomFilter bloomFilter = linkedBloomFilters[index]; + LinkedTolerantFilter bloomFilter = linkedBloomFilters[index]; if (bloomFilter == null) { - bloomFilter = new LinkedShrinkableBloomFilter(this, miniExpectNum, growRate); + bloomFilter = new LinkedTolerantFilter(this, miniExpectNum, growRate); linkedBloomFilters[index] = bloomFilter; } bloomFilter.add(buildBloomFilterKey(content)); @@ -130,7 +130,7 @@ public boolean contains(V content) { int keyGroupIndex = KeyGroupRangeAssignment.assignToKeyGroup(keyContext.getCurrentKey(), totalKeyGroups); int index = getIndexForKeyGroup(keyGroupIndex); - LinkedShrinkableBloomFilter bloomFilter = linkedBloomFilters[index]; + LinkedTolerantFilter bloomFilter = linkedBloomFilters[index]; if (bloomFilter == null) { return false; } @@ -147,7 +147,7 @@ public void snapshotStateForKeyGroup(DataOutputViewStreamWrapper stream, int key LOG.info("snapshot state for group {} ", keyGroupIdx); int index = getIndexForKeyGroup(keyGroupIdx); - LinkedShrinkableBloomFilter bloomFilter = this.linkedBloomFilters[index]; + LinkedTolerantFilter bloomFilter = this.linkedBloomFilters[index]; if (bloomFilter != null) { stream.writeBoolean(true); stream.writeLong(this.restMemSize); @@ -165,7 +165,7 @@ public void restoreStateForKeyGroup( int index = getIndexForKeyGroup(keyGroupIdx); if (stream.readBoolean()) { this.restMemSize = stream.readLong(); - LinkedShrinkableBloomFilter linkedBloomFilter = new LinkedShrinkableBloomFilter(this, miniExpectNum, growRate); + LinkedTolerantFilter linkedBloomFilter = new LinkedTolerantFilter(this, miniExpectNum, growRate); linkedBloomFilter.restore(stream); this.linkedBloomFilters[index] = linkedBloomFilter; LOG.info("group {} restored.", keyGroupIdx); @@ -202,8 +202,8 @@ ShrinkableBloomFilterNode allocateBloomFilterNode(long expectNum, boolean force) return new ShrinkableBloomFilterNode((int) requestNum, fpp, ttl); } - void takeBack(ShrinkableBloomFilterNode node) { - restMemSize += BloomFilter.optimalNumOfBits(node.getCapacity(), node.getFpp()); + void takeBack(TolerantFilterNode node) { + restMemSize += BloomFilter.optimalNumOfBits(node.capacity(), node.fpp()); } long estimatePropExpectNum(long expectNum, double fpp) { @@ -236,7 +236,7 @@ public String toString() { .append("total memory:").append(totalMemSize).append("\t").append("rest memory:").append(restMemSize).append("\n"); for (int i = 0; i < localKeyGroupRange.getNumberOfKeyGroups(); ++i) { - LinkedShrinkableBloomFilter bloomFilter = this.linkedBloomFilters[i]; + LinkedTolerantFilter bloomFilter = this.linkedBloomFilters[i]; if (bloomFilter != null) { builder.append("group ").append(i + localKeyGroupRangeStartIdx).append(":").append(bloomFilter.toString()).append("\n"); } @@ -245,7 +245,7 @@ public String toString() { } @VisibleForTesting - LinkedShrinkableBloomFilter[] getLinkedBloomFilters() { + LinkedTolerantFilter[] getLinkedBloomFilters() { return this.linkedBloomFilters; } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedShrinkableBloomFilter.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedTolerantFilter.java similarity index 62% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedShrinkableBloomFilter.java rename to flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedTolerantFilter.java index 6e63b89cc88f7..8342f001d42b1 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedShrinkableBloomFilter.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedTolerantFilter.java @@ -22,6 +22,7 @@ import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; +import org.apache.flink.util.FlinkRuntimeException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -30,12 +31,12 @@ import java.util.LinkedList; /** - * A list of {@link ShrinkableBloomFilterNode} to avoid data skewed between key ranges. The size of nodes on the list + * A list of {@link TolerantFilterNode} to avoid data skewed between key ranges. The size of nodes on the list * grow by a {@code growRate} to avoid the list to be too long. */ -public class LinkedShrinkableBloomFilter { +public class LinkedTolerantFilter { - private static final Logger LOG = LoggerFactory.getLogger(LinkedShrinkableBloomFilter.class); + private static final Logger LOG = LoggerFactory.getLogger(LinkedTolerantFilter.class); private long currentSize; @@ -44,9 +45,9 @@ public class LinkedShrinkableBloomFilter { private ElasticBloomFilter partitionedBloomFilter; - private LinkedList bloomFilterNodes = new LinkedList<>(); + private LinkedList bloomFilterNodes = new LinkedList<>(); - public LinkedShrinkableBloomFilter(ElasticBloomFilter partitionedBloomFilter, long initSize, double growRate) { + public LinkedTolerantFilter(ElasticBloomFilter partitionedBloomFilter, long initSize, double growRate) { this.partitionedBloomFilter = partitionedBloomFilter; this.currentSize = initSize; this.initSize = initSize; @@ -55,10 +56,10 @@ public LinkedShrinkableBloomFilter(ElasticBloomFilter partitionedBloomFilter, lo public void add(byte[] content) { synchronized (bloomFilterNodes) { - ShrinkableBloomFilterNode node; + TolerantFilterNode node; if (bloomFilterNodes.size() > 0) { node = bloomFilterNodes.getLast(); - if (node.isFull()) { + if (node.full()) { LOG.info("allocate new node."); currentSize = (long) (this.initSize * Math.pow(growRate, bloomFilterNodes.size())); node = this.partitionedBloomFilter.allocateBloomFilterNode(currentSize); @@ -66,9 +67,10 @@ public void add(byte[] content) { LOG.info("allocate new node successfully."); bloomFilterNodes.add(node); } else { - LOG.warn("allocate new node failed (run out of configured capacity), reuse the last node."); - node = bloomFilterNodes.getLast(); - node.reSetTtl(); +// LOG.warn("allocate new node failed (run out of configured capacity), reuse the last node."); +// node = bloomFilterNodes.getLast(); +// node.reSetTtl(); + throw new FlinkRuntimeException("memory out."); } } } else { @@ -82,9 +84,9 @@ public void add(byte[] content) { public boolean contains(byte[] content) { synchronized (bloomFilterNodes) { - Iterator iter = bloomFilterNodes.descendingIterator(); + Iterator iter = bloomFilterNodes.descendingIterator(); while (iter.hasNext()) { - ShrinkableBloomFilterNode node = iter.next(); + TolerantFilterNode node = iter.next(); if (node.contains(content)) { return true; } @@ -92,35 +94,24 @@ public boolean contains(byte[] content) { return false; } } -// -// // for checkpoint and recovery -// public LinkedBloomFilter copy() { -// synchronized (bloomFilterNodes) { -// LinkedBloomFilter bloomFilter = new LinkedBloomFilter(partitionedBloomFilter, initSize, growRate); -// for (ShrinkableBloomFilterNode node : bloomFilterNodes) { -// bloomFilter.bloomFilterNodes.add(node.copy()); -// } -// return bloomFilter; -// } -// } @VisibleForTesting - long getCurrentSize() { + long size() { return currentSize; } @VisibleForTesting - long getInitSize() { + long initSize() { return initSize; } @VisibleForTesting - double getGrowRate() { + double growRate() { return growRate; } @VisibleForTesting - LinkedList getBloomFilterNodes() { + LinkedList getBloomFilterNodes() { return bloomFilterNodes; } @@ -128,7 +119,7 @@ LinkedList getBloomFilterNodes() { public String toString() { StringBuilder builder = new StringBuilder(); synchronized (bloomFilterNodes) { - for (ShrinkableBloomFilterNode node : bloomFilterNodes) { + for (TolerantFilterNode node : bloomFilterNodes) { if (builder.length() > 0) { builder.append(" -> "); } @@ -145,7 +136,7 @@ void snapshot(DataOutputView outputView) throws IOException { outputView.writeDouble(growRate); bloomFilterNodes.removeIf(node -> { - if (node.getDealine() <= ts) { + if (node.deadline() <= ts) { partitionedBloomFilter.takeBack(node); return true; } @@ -153,7 +144,7 @@ void snapshot(DataOutputView outputView) throws IOException { }); outputView.writeInt(bloomFilterNodes.size()); - for (ShrinkableBloomFilterNode node : bloomFilterNodes) { + for (TolerantFilterNode node : bloomFilterNodes) { node.snapshot(outputView); } } @@ -164,9 +155,37 @@ void restore(DataInputView source) throws IOException { growRate = source.readDouble(); int len = source.readInt(); for (int i = 0; i < len; ++i) { - ShrinkableBloomFilterNode node = new ShrinkableBloomFilterNode(1, 1, 1); + ShrinkableBloomFilterNode node = new ShrinkableBloomFilterNode(); node.restore(source); bloomFilterNodes.add(node); } } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + LinkedTolerantFilter that = (LinkedTolerantFilter) o; + + if (currentSize != that.currentSize) return false; + if (initSize != that.initSize) return false; + if (Double.compare(that.growRate, growRate) != 0) return false; + if (partitionedBloomFilter != null ? !partitionedBloomFilter.equals(that.partitionedBloomFilter) : that.partitionedBloomFilter != null) + return false; + return bloomFilterNodes != null ? bloomFilterNodes.equals(that.bloomFilterNodes) : that.bloomFilterNodes == null; + } + + @Override + public int hashCode() { + int result; + long temp; + result = (int) (currentSize ^ (currentSize >>> 32)); + result = 31 * result + (int) (initSize ^ (initSize >>> 32)); + temp = Double.doubleToLongBits(growRate); + result = 31 * result + (int) (temp ^ (temp >>> 32)); + result = 31 * result + (partitionedBloomFilter != null ? partitionedBloomFilter.hashCode() : 0); + result = 31 * result + (bloomFilterNodes != null ? bloomFilterNodes.hashCode() : 0); + return result; + } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNode.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNode.java index 95ea5623ad88e..d220a72309d2b 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNode.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNode.java @@ -18,44 +18,40 @@ package org.apache.flink.streaming.api.operators; +import org.apache.flink.annotation.VisibleForTesting; import org.apache.flink.core.memory.DataInputView; import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.shaded.guava18.com.google.common.hash.BloomFilter; import org.apache.flink.shaded.guava18.com.google.common.hash.Funnels; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.util.Arrays; import java.util.Objects; /** - * A shrinkable bloom filter node linked each other in {@link LinkedShrinkableBloomFilter} to avoid data skewed. + * A shrinkable bloom filter node linked each other in {@link LinkedTolerantFilter} to avoid data skewed. */ -public class ShrinkableBloomFilterNode { - - private static final Logger LOG = LoggerFactory.getLogger(ShrinkableBloomFilterNode.class); +public class ShrinkableBloomFilterNode implements TolerantFilterNode { - private static final int DEFAULT_UNITS_NUM = 8; + public static final int DEFAULT_UNITS_NUM = 8; private int capacity; private double fpp; - private long size; - private long rawTtl; private long ttl; private long deleteTS = Long.MAX_VALUE; public BloomFilterUnit[] bloomFilterUnits; + ShrinkableBloomFilterNode() { + } + public ShrinkableBloomFilterNode(int capacity, double fpp, long ttl) { this.capacity = capacity; this.fpp = fpp; - this.rawTtl = ttl; this.ttl = ttl; - this.size = 0; bloomFilterUnits = new BloomFilterUnit[DEFAULT_UNITS_NUM]; @@ -66,19 +62,30 @@ public ShrinkableBloomFilterNode(int capacity, double fpp, long ttl) { } } - public boolean isFull() { - return size >= capacity; + @Override + public boolean full() { + + for (BloomFilterUnit bloomFilterUnit : bloomFilterUnits) { + if (bloomFilterUnit.full()) { + deleteTS = System.currentTimeMillis() + ttl; + return true; + } + } + + return false; } - public long getCapacity() { + @Override + public long capacity() { int capacity = 0; for (BloomFilterUnit bloomFilterUnit : bloomFilterUnits) { - capacity += bloomFilterUnit.size(); + capacity += bloomFilterUnit.capacity(); } return capacity; } - public long getSize() { + @Override + public long size() { int size = 0; for (BloomFilterUnit bloomFilterUnit : bloomFilterUnits) { size += bloomFilterUnit.size(); @@ -86,35 +93,39 @@ public long getSize() { return size; } - public long getTTL() { + @Override + public long ttl() { return ttl; } - public double getFpp() { + @Override + public double fpp() { return fpp; } public void reSetTtl() { - this.ttl = this.rawTtl; this.deleteTS = Long.MAX_VALUE; } - public long getDealine() { + @Override + public long deadline() { return deleteTS; } + @Override public void add(byte[] content) { - int unitIndex = content.hashCode() % bloomFilterUnits.length; + int unitIndex = Arrays.hashCode(content) % bloomFilterUnits.length; BloomFilterUnit bloomFilterUnit = bloomFilterUnits[unitIndex]; bloomFilterUnit.add(content); } + @Override public boolean contains(byte[] content) { - int unitIndex = content.hashCode() % bloomFilterUnits.length; + int unitIndex = Arrays.hashCode(content) % bloomFilterUnits.length; BloomFilterUnit bloomFilterUnit = bloomFilterUnits[unitIndex]; @@ -126,22 +137,23 @@ public boolean contains(byte[] content) { */ public void shrink() { if (shrinkable()) { - int left = 0; - int right = bloomFilterUnits.length - 1; + int mergeIndex1 = 0; + int mergeIndex2 = bloomFilterUnits.length >>> 1; + final int newSize = mergeIndex2; + final BloomFilterUnit[] newBloomFilterUnits = new BloomFilterUnit[newSize]; - BloomFilterUnit[] newBloomFilterUnits = new BloomFilterUnit[bloomFilterUnits.length >> 1]; int index = 0; - while(left < right) { - BloomFilterUnit leftUnit = bloomFilterUnits[left]; - BloomFilterUnit rightUnit = bloomFilterUnits[right]; + while(index < newSize) { + BloomFilterUnit leftUnit = bloomFilterUnits[mergeIndex1]; + BloomFilterUnit rightUnit = bloomFilterUnits[mergeIndex2]; // merging leftUnit.merge(rightUnit); newBloomFilterUnits[index] = leftUnit; - ++left; - --right; + ++mergeIndex1; + ++mergeIndex2; ++index; } @@ -152,7 +164,8 @@ public void shrink() { /** * Check whether this Node is shrinkable, */ - private boolean shrinkable() { + @VisibleForTesting + boolean shrinkable() { // we can't shrink it when it has only one unit. if (bloomFilterUnits.length <= 1) { @@ -163,7 +176,7 @@ private boolean shrinkable() { int capacity = bloomFilterUnit.capacity(); int size = bloomFilterUnit.size(); - if (size > (capacity >> 1)) { + if (size > (capacity >>> 1)) { return false; } } @@ -171,12 +184,12 @@ private boolean shrinkable() { } private int computeUnitCapacity(int capacity, int units) { - return 0; + return Math.floorDiv(capacity, units); } @Override public String toString() { - return String.format("{c:%d s:%d}", capacity, size); + return String.format("{c:%d s:%d}", capacity, size()); } @Override @@ -184,8 +197,7 @@ public boolean equals(Object obj) { if (obj instanceof ShrinkableBloomFilterNode) { ShrinkableBloomFilterNode other = (ShrinkableBloomFilterNode) obj; if (other.capacity == this.capacity - && other.size == this.size - && other.rawTtl == this.rawTtl + && other.ttl == this.ttl && other.fpp == this.fpp && other.bloomFilterUnits.length == bloomFilterUnits.length) { @@ -200,16 +212,15 @@ public boolean equals(Object obj) { return false; } - void snapshot(DataOutputView outputView) throws IOException { - outputView.writeLong(capacity); - outputView.writeLong(rawTtl); + @Override + public void snapshot(DataOutputView outputView) throws IOException { + outputView.writeInt(capacity); + outputView.writeLong(ttl); if (deleteTS == Long.MAX_VALUE) { outputView.writeLong(ttl); //rest ttl } else { outputView.writeLong(deleteTS - System.currentTimeMillis()); //rest ttl } - outputView.writeLong(deleteTS); - outputView.writeLong(size); outputView.writeDouble(fpp); int unitNum = bloomFilterUnits.length; @@ -221,15 +232,16 @@ void snapshot(DataOutputView outputView) throws IOException { } } - void restore(DataInputView source) throws IOException { + @Override + public void restore(DataInputView source) throws IOException { capacity = source.readInt(); - rawTtl = source.readLong(); ttl = source.readLong(); - deleteTS = source.readLong(); - if (rawTtl != ttl) { - deleteTS = System.currentTimeMillis() + ttl; + final long restTtl = source.readLong(); + if (restTtl != ttl) { + deleteTS = System.currentTimeMillis() + restTtl; + } else { + deleteTS = Long.MAX_VALUE; } - size = source.readLong(); fpp = source.readDouble(); int unitNum = source.readInt(); diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TolerantFilterNode.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TolerantFilterNode.java new file mode 100644 index 0000000000000..d7de15022c193 --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/TolerantFilterNode.java @@ -0,0 +1,49 @@ +/* + * 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.flink.streaming.api.operators; + +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; + +/** + */ +interface TolerantFilterNode { + + boolean full(); + + long capacity(); + + long size(); + + long ttl(); + + double fpp(); + + long deadline(); + + void add(byte[] content); + + boolean contains(byte[] content); + + void snapshot(DataOutputView outputView) throws IOException; + + void restore(DataInputView source) throws IOException; +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterTest.java deleted file mode 100644 index c14265b38ae7c..0000000000000 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterTest.java +++ /dev/null @@ -1,131 +0,0 @@ -package org.apache.flink.streaming.api.operators; - -import org.apache.flink.api.common.typeutils.base.IntSerializer; -import org.apache.flink.api.common.typeutils.base.StringSerializer; -import org.apache.flink.core.memory.DataInputViewStreamWrapper; -import org.apache.flink.core.memory.DataOutputViewStreamWrapper; -import org.apache.flink.runtime.state.KeyGroupRange; - -import org.junit.Assert; -import org.junit.Test; - -import java.io.ByteArrayInputStream; -import java.io.ByteArrayOutputStream; -import java.io.IOException; -import java.util.List; -import java.util.concurrent.TimeUnit; - -/** - * {@link LinkedShrinkableBloomFilter} unit tests. - */ -public class LinkedBloomFilterTest { - - @Test - public void basicTest() throws InterruptedException { - - StreamOperator dumyKeyContext = new AbstractStreamOperator() { - private static final long serialVersionUID = 1L; - }; - - ElasticBloomFilter partitionedBloomFilter = new ElasticBloomFilter( - new StringSerializer(), - new IntSerializer(), - 10, - new KeyGroupRange(0, 9), - dumyKeyContext, - 10000, - 0.01, - 1000, - 1000, - 10000, - 2.0 - ); - - LinkedShrinkableBloomFilter linkedBloomFilter = new LinkedShrinkableBloomFilter(partitionedBloomFilter, 1000, 2); - - List nodes = linkedBloomFilter.getBloomFilterNodes(); - - Assert.assertEquals(0, nodes.size()); - for (int i = 0; i < 1000; ++i) { - linkedBloomFilter.add(String.valueOf(i).getBytes()); - Assert.assertTrue(linkedBloomFilter.contains(String.valueOf(i).getBytes())); - } - Assert.assertEquals(1, nodes.size()); - - linkedBloomFilter.add("1001".getBytes()); - Assert.assertEquals(2, nodes.size()); - } - - @Test - public void testSnapshotAndRestore() throws InterruptedException, IOException { - - StreamOperator dumyKeyContext = new AbstractStreamOperator() { - private static final long serialVersionUID = 1L; - }; - - ElasticBloomFilter partitionedBloomFilter = new ElasticBloomFilter( - new StringSerializer(), - new IntSerializer(), - 10, - new KeyGroupRange(0, 9), - dumyKeyContext, - 10000, - 0.01, - 1000, - 1000, - 10000, - 2.0 - ); - - LinkedShrinkableBloomFilter linkedBloomFilter = new LinkedShrinkableBloomFilter(partitionedBloomFilter, 1000, 2); - - List nodes = linkedBloomFilter.getBloomFilterNodes(); - - Assert.assertEquals(0, nodes.size()); - for (int i = 0; i < 1000; ++i) { - linkedBloomFilter.add(String.valueOf(i).getBytes()); - Assert.assertTrue(linkedBloomFilter.contains(String.valueOf(i).getBytes())); - } - Assert.assertEquals(1, nodes.size()); - - linkedBloomFilter.add(String.valueOf("1001").getBytes()); - Assert.assertEquals(2, nodes.size()); - - ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - - DataOutputViewStreamWrapper outputViewStreamWrapper = new DataOutputViewStreamWrapper(outputStream); - linkedBloomFilter.snapshot(outputViewStreamWrapper); - - byte[] outputBytes = outputStream.toByteArray(); - - LinkedShrinkableBloomFilter linkedBloomFilter2 = new LinkedShrinkableBloomFilter(partitionedBloomFilter, 1000, 2); - - linkedBloomFilter2.restore(new DataInputViewStreamWrapper(new ByteArrayInputStream(outputBytes))); - - Assert.assertEquals(linkedBloomFilter.getCurrentSize(), linkedBloomFilter2.getCurrentSize()); - Assert.assertEquals(linkedBloomFilter.getInitSize(), linkedBloomFilter2.getInitSize()); - Assert.assertEquals(String.valueOf(linkedBloomFilter.getGrowRate()), String.valueOf(linkedBloomFilter2.getGrowRate())); - - List nodes2 = linkedBloomFilter2.getBloomFilterNodes(); - - Assert.assertEquals(nodes.size(), nodes2.size()); - for (int i = 0; i < nodes.size(); ++i) { - Assert.assertEquals(nodes.get(i), nodes2.get(i)); - } - - // ---- test ttl - TimeUnit.SECONDS.sleep(2); - outputStream.reset(); - outputViewStreamWrapper = new DataOutputViewStreamWrapper(outputStream); - linkedBloomFilter.snapshot(outputViewStreamWrapper); - outputBytes = outputStream.toByteArray(); - - LinkedShrinkableBloomFilter linkedBloomFilter3 = new LinkedShrinkableBloomFilter(partitionedBloomFilter, 1000, 2); - linkedBloomFilter3.restore(new DataInputViewStreamWrapper(new ByteArrayInputStream(outputBytes))); - List nodes3 = linkedBloomFilter3.getBloomFilterNodes(); - - Assert.assertEquals(1, nodes.size()); - Assert.assertEquals(1, nodes3.size()); - Assert.assertEquals(nodes.get(0), nodes3.get(0)); - } -} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedTolerantFilterTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedTolerantFilterTest.java new file mode 100644 index 0000000000000..793f4df39ef5e --- /dev/null +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedTolerantFilterTest.java @@ -0,0 +1,187 @@ +package org.apache.flink.streaming.api.operators; + +import org.apache.flink.api.common.typeutils.base.IntSerializer; +import org.apache.flink.api.common.typeutils.base.StringSerializer; +import org.apache.flink.core.memory.DataInputViewStreamWrapper; +import org.apache.flink.core.memory.DataOutputViewStreamWrapper; +import org.apache.flink.runtime.state.KeyGroupRange; + +import org.junit.Assert; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.util.List; +import java.util.concurrent.TimeUnit; + +/** + * {@link LinkedTolerantFilter} unit tests. + */ +public class LinkedTolerantFilterTest { + + @Test + public void basicTest() throws InterruptedException { + + StreamOperator dumyKeyContext = new AbstractStreamOperator() { + private static final long serialVersionUID = 1L; + }; + + ElasticBloomFilter partitionedBloomFilter = new ElasticBloomFilter( + new StringSerializer(), + new IntSerializer(), + 10, + new KeyGroupRange(0, 9), + dumyKeyContext, + 10000, + 0.01, + 1000, + 1000, + 10000, + 2.0 + ); + + LinkedTolerantFilter linkedBloomFilter = new LinkedTolerantFilter(partitionedBloomFilter, 1000, 2); + + List nodes = linkedBloomFilter.getBloomFilterNodes(); + + Assert.assertEquals(0, nodes.size()); + for (int i = 0; i < 900; ++i) { + linkedBloomFilter.add(String.valueOf(i).getBytes()); + } + + for (int i = 0; i < 900; ++i) { + Assert.assertTrue(linkedBloomFilter.contains(String.valueOf(i).getBytes())); + } + + Assert.assertEquals(1, nodes.size()); + + for (int i = 900; i < 1001; ++i) { + linkedBloomFilter.add(String.valueOf(i).getBytes()); + } + Assert.assertEquals(2, nodes.size()); + } + + @Test + public void testSnapshotAndRestore() throws InterruptedException, IOException { + + StreamOperator dumyKeyContext = new AbstractStreamOperator() { + private static final long serialVersionUID = 1L; + }; + + ElasticBloomFilter partitionedBloomFilter = new ElasticBloomFilter( + new StringSerializer(), + new IntSerializer(), + 10, + new KeyGroupRange(0, 9), + dumyKeyContext, + 10000, + 0.01, + 1000, + 1000, + 10000, + 2.0 + ); + + LinkedTolerantFilter linkedBloomFilter = new LinkedTolerantFilter(partitionedBloomFilter, 1000, 2); + + List nodes = linkedBloomFilter.getBloomFilterNodes(); + + Assert.assertEquals(0, nodes.size()); + for (int i = 0; i < 900; ++i) { + Assert.assertFalse(linkedBloomFilter.contains(String.valueOf(i).getBytes())); + } + for (int i = 0; i < 900; ++i) { + linkedBloomFilter.add(String.valueOf(i).getBytes()); + } + + Assert.assertEquals(1, nodes.size()); + + for (int i = 900; i < 1001; ++i) { + linkedBloomFilter.add(String.valueOf(i).getBytes()); + } + + for (int i = 900; i < 1001; ++i) { + Assert.assertTrue(linkedBloomFilter.contains(String.valueOf(i).getBytes())); + } + + Assert.assertEquals(2, nodes.size()); + + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + + DataOutputViewStreamWrapper outputViewStreamWrapper = new DataOutputViewStreamWrapper(outputStream); + linkedBloomFilter.snapshot(outputViewStreamWrapper); + + byte[] outputBytes = outputStream.toByteArray(); + + LinkedTolerantFilter linkedBloomFilter2 = new LinkedTolerantFilter(partitionedBloomFilter, 1000, 2); + + linkedBloomFilter2.restore(new DataInputViewStreamWrapper(new ByteArrayInputStream(outputBytes))); + + Assert.assertEquals(linkedBloomFilter, linkedBloomFilter2); + } + + + @Test + public void testTTL() throws InterruptedException, IOException { + + StreamOperator dumyKeyContext = new AbstractStreamOperator() { + private static final long serialVersionUID = 1L; + }; + + ElasticBloomFilter partitionedBloomFilter = new ElasticBloomFilter( + new StringSerializer(), + new IntSerializer(), + 10, + new KeyGroupRange(0, 9), + dumyKeyContext, + 10000, + 0.01, + 1000, + 1000, + 10000, + 2.0 + ); + + LinkedTolerantFilter linkedBloomFilter = new LinkedTolerantFilter(partitionedBloomFilter, 1000, 2); + + List nodes = linkedBloomFilter.getBloomFilterNodes(); + + Assert.assertEquals(0, nodes.size()); + + for (int i = 0; i < 1001; ++i) { + linkedBloomFilter.add(String.valueOf(i).getBytes()); + } + + Assert.assertEquals(2, nodes.size()); + + TimeUnit.MILLISECONDS.sleep(2000); + + Assert.assertEquals(2, nodes.size()); + + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + + DataOutputViewStreamWrapper outputViewStreamWrapper = new DataOutputViewStreamWrapper(outputStream); + // this will release the expired node + linkedBloomFilter.snapshot(outputViewStreamWrapper); + + Assert.assertEquals(1, nodes.size()); + + // ---------> test TTL align when restoring + + for (int i = 0; i < 2001; ++i) { + linkedBloomFilter.add(String.valueOf(i).getBytes()); + } + + Assert.assertEquals(2, nodes.size()); + + outputStream.reset(); + linkedBloomFilter.snapshot(outputViewStreamWrapper); + + TimeUnit.MILLISECONDS.sleep(2000); + + linkedBloomFilter = new LinkedTolerantFilter(partitionedBloomFilter, 1000, 2); + linkedBloomFilter.restore(new DataInputViewStreamWrapper(new ByteArrayInputStream(outputStream.toByteArray()))); + Assert.assertEquals(2, linkedBloomFilter.getBloomFilterNodes().size()); + } +} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterTest.java index e0d335ba5a3c4..8490fd24ff165 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterTest.java @@ -50,7 +50,7 @@ public Object getCurrentKey() { 2.0 ); - LinkedShrinkableBloomFilter[] linkedBloomFilters = partitionedBloomFilter.getLinkedBloomFilters(); + LinkedTolerantFilter[] linkedBloomFilters = partitionedBloomFilter.getLinkedBloomFilters(); Assert.assertEquals(10, linkedBloomFilters.length); for (int i = 0; i < 10; ++i) { Assert.assertNull(linkedBloomFilters[i]); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNodeTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNodeTest.java index 442a938256745..1368047b8ef88 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNodeTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNodeTest.java @@ -108,9 +108,9 @@ public void testSnapshotAndRestoreForBloomFilterUnit() throws Exception { DataOutputViewStreamWrapper outputViewStreamWrapper = new DataOutputViewStreamWrapper(outputStream); unit.snapshot(outputViewStreamWrapper); - byte[] snapshottedBytes = outputStream.toByteArray(); + byte[] snapshotBytes = outputStream.toByteArray(); - ByteArrayInputStream inputStream = new ByteArrayInputStream(snapshottedBytes); + ByteArrayInputStream inputStream = new ByteArrayInputStream(snapshotBytes); DataInputViewStreamWrapper inputViewStreamWrapper = new DataInputViewStreamWrapper(inputStream); ShrinkableBloomFilterNode.BloomFilterUnit restoredUnit = new ShrinkableBloomFilterNode.BloomFilterUnit(); @@ -122,16 +122,81 @@ public void testSnapshotAndRestoreForBloomFilterUnit() throws Exception { @Test public void testBasicFunctionalityForShrinkableBloomFilterNode() { + ShrinkableBloomFilterNode shrinkableBloomFilterNode = new ShrinkableBloomFilterNode(10000, 0.02, 1000); + + Assert.assertTrue(10000 <= shrinkableBloomFilterNode.capacity()); + Assert.assertEquals(0, shrinkableBloomFilterNode.size()); + Assert.assertFalse(shrinkableBloomFilterNode.full()); + + for (int i = 0; i < 10000; ++i) { + Assert.assertFalse(shrinkableBloomFilterNode.contains(String.valueOf(i).getBytes())); + } + + for (int i = 0; i < 10000; ++i) { + shrinkableBloomFilterNode.add(String.valueOf(i).getBytes()); + } + + for (int i = 0; i < 10000; ++i) { + Assert.assertTrue(shrinkableBloomFilterNode.contains(String.valueOf(i).getBytes())); + } } @Test - public void testSnapshotAndRestoreForShrinkableBloomFilterNode() { + public void testSnapshotAndRestoreForShrinkableBloomFilterNode() throws Exception { + + ShrinkableBloomFilterNode shrinkableBloomFilterNode = new ShrinkableBloomFilterNode(10000, 0.02, 1000); + + for (int i = 0; i < 2000; ++i) { + shrinkableBloomFilterNode.add(String.valueOf(i).getBytes()); + } + + ByteArrayOutputStream outputStream = new ByteArrayOutputStream(1024); + DataOutputViewStreamWrapper outputViewStreamWrapper = new DataOutputViewStreamWrapper(outputStream); + shrinkableBloomFilterNode.snapshot(outputViewStreamWrapper); + + byte[] snapshotBytes = outputStream.toByteArray(); + + ByteArrayInputStream inputStream = new ByteArrayInputStream(snapshotBytes); + DataInputViewStreamWrapper inputViewStreamWrapper = new DataInputViewStreamWrapper(inputStream); + + ShrinkableBloomFilterNode restoredNode = new ShrinkableBloomFilterNode(10000, 0.02, 1000); + restoredNode.restore(inputViewStreamWrapper); + + Assert.assertEquals(shrinkableBloomFilterNode, restoredNode); } @Test public void testShrinking() { + ShrinkableBloomFilterNode shrinkableBloomFilterNode = new ShrinkableBloomFilterNode(10000, 0.02, 1000); + + for (int i = 0; i < 2000; ++i) { + shrinkableBloomFilterNode.add(String.valueOf(i).getBytes()); + } + + for (int i = 0; i < 2000; ++i) { + Assert.assertTrue(shrinkableBloomFilterNode.contains(String.valueOf(i).getBytes())); + } + + List records2000To10000 = new ArrayList<>(); + for (int i = 2000; i < 10000; ++i) { + records2000To10000.add(String.valueOf(i).getBytes()); + } + + verifyBloomFilterWithFpp(shrinkableBloomFilterNode, records2000To10000, false, 0.02); + + Assert.assertEquals(ShrinkableBloomFilterNode.DEFAULT_UNITS_NUM, shrinkableBloomFilterNode.bloomFilterUnits.length); + Assert.assertTrue(shrinkableBloomFilterNode.shrinkable()); + shrinkableBloomFilterNode.shrink(); + + Assert.assertEquals(ShrinkableBloomFilterNode.DEFAULT_UNITS_NUM >>> 1, shrinkableBloomFilterNode.bloomFilterUnits.length); + + for (int i = 0; i < 2000; ++i) { + Assert.assertTrue(shrinkableBloomFilterNode.contains(String.valueOf(i).getBytes())); + } + + verifyBloomFilterWithFpp(shrinkableBloomFilterNode, records2000To10000, false, 0.02); } private boolean verifyBloomFilterWithFpp(ShrinkableBloomFilterNode.BloomFilterUnit unit, Collection records, boolean target, double expectedFpp) { @@ -143,4 +208,14 @@ private boolean verifyBloomFilterWithFpp(ShrinkableBloomFilterNode.BloomFilterUn } return ((double) count / records.size()) <= expectedFpp; } + + private boolean verifyBloomFilterWithFpp(ShrinkableBloomFilterNode unit, Collection records, boolean target, double expectedFpp) { + int count = 0; + for (byte[] record : records) { + if (unit.contains(record) != target) { + ++count; + } + } + return ((double) count / records.size()) <= expectedFpp; + } } From 6710c518ed2dc86113d2baf620edeecc23e38cec Mon Sep 17 00:00:00 2001 From: sihuazhou Date: Tue, 12 Jun 2018 13:38:43 +0800 Subject: [PATCH 5/5] tmp. --- ...java => ElasticFilterStateDescriptor.java} | 38 ++---- .../bloomfilter/BloomFilterExample.java | 12 +- .../api/operators/AbstractStreamOperator.java | 4 +- ...TolerantFilter.java => ElasticFilter.java} | 10 +- ...oomFilter.java => ElasticFilterState.java} | 51 ++++---- ...er.java => ElasticFilterStateManager.java} | 30 ++--- .../operators/ShrinkableBloomFilterNode.java | 6 +- .../api/operators/SimpleBloomFilterNode.java | 109 ++++++++++++++++++ .../operators/StreamOperatorStateContext.java | 2 +- .../StreamTaskStateInitializerImpl.java | 12 +- .../operators/StreamingRuntimeContext.java | 4 +- ...ava => ElasticFilterStateManagerTest.java} | 28 ++--- ...rTest.java => ElasticFilterStateTest.java} | 35 +++--- ...FilterTest.java => ElasticFilterTest.java} | 20 ++-- .../operators/LinkedBloomFilterNodeTest.java | 59 ---------- .../runtime/tasks/StreamTaskTest.java | 4 +- 16 files changed, 224 insertions(+), 200 deletions(-) rename flink-core/src/main/java/org/apache/flink/api/common/state/{PartitionedBloomFilterDescriptor.java => ElasticFilterStateDescriptor.java} (64%) rename flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/{LinkedTolerantFilter.java => ElasticFilter.java} (94%) rename flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/{ElasticBloomFilter.java => ElasticFilterState.java} (80%) rename flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/{ElasticBloomFilterManager.java => ElasticFilterStateManager.java} (82%) create mode 100644 flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SimpleBloomFilterNode.java rename flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/{PartitionedBloomFilterManagerTest.java => ElasticFilterStateManagerTest.java} (78%) rename flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/{PartitionedBloomFilterTest.java => ElasticFilterStateTest.java} (77%) rename flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/{LinkedTolerantFilterTest.java => ElasticFilterTest.java} (83%) delete mode 100644 flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNodeTest.java diff --git a/flink-core/src/main/java/org/apache/flink/api/common/state/PartitionedBloomFilterDescriptor.java b/flink-core/src/main/java/org/apache/flink/api/common/state/ElasticFilterStateDescriptor.java similarity index 64% rename from flink-core/src/main/java/org/apache/flink/api/common/state/PartitionedBloomFilterDescriptor.java rename to flink-core/src/main/java/org/apache/flink/api/common/state/ElasticFilterStateDescriptor.java index b40c2fa7348ed..2b5f4565a04af 100644 --- a/flink-core/src/main/java/org/apache/flink/api/common/state/PartitionedBloomFilterDescriptor.java +++ b/flink-core/src/main/java/org/apache/flink/api/common/state/ElasticFilterStateDescriptor.java @@ -7,9 +7,9 @@ import java.io.Serializable; /** - * Created by zsh on 06/02/2018. + * To be. */ -public class PartitionedBloomFilterDescriptor implements Serializable { +public class ElasticFilterStateDescriptor implements Serializable { private static final long serialVersionUID = 1L; @@ -36,7 +36,7 @@ public class PartitionedBloomFilterDescriptor implements Serializable { * or lazily once the type is serialized or an ExecutionConfig is provided. */ protected TypeSerializer serializer; - public PartitionedBloomFilterDescriptor(String stateName, TypeInformation typeInfo, long capacity, double fpp) { + public ElasticFilterStateDescriptor(String stateName, TypeInformation typeInfo, long capacity, double fpp) { this( stateName, typeInfo, @@ -45,7 +45,7 @@ public PartitionedBloomFilterDescriptor(String stateName, TypeInformation typ Integer.MAX_VALUE); } - public PartitionedBloomFilterDescriptor(String stateName, TypeInformation typeInfo, long capacity, double fpp, long ttl) { + public ElasticFilterStateDescriptor(String stateName, TypeInformation typeInfo, long capacity, double fpp, long ttl) { this( stateName, typeInfo, @@ -57,7 +57,7 @@ public PartitionedBloomFilterDescriptor(String stateName, TypeInformation typ DEFAULT_GROW_RATE); } - public PartitionedBloomFilterDescriptor(String stateName, TypeInformation typeInfo, long capacity, double fpp, long ttl, long minExpectNum, long maxExpectNum, double growRate) { + public ElasticFilterStateDescriptor(String stateName, TypeInformation typeInfo, long capacity, double fpp, long ttl, long minExpectNum, long maxExpectNum, double growRate) { this.stateName = stateName; this.typeInfo = typeInfo; this.capacity = capacity; @@ -68,7 +68,7 @@ public PartitionedBloomFilterDescriptor(String stateName, TypeInformation typ this.growRate = growRate; } - public PartitionedBloomFilterDescriptor(String stateName, TypeSerializer serializer, long capacity, double fpp, long ttl) { + public ElasticFilterStateDescriptor(String stateName, TypeSerializer serializer, long capacity, double fpp, long ttl) { this( stateName, serializer, @@ -80,7 +80,7 @@ public PartitionedBloomFilterDescriptor(String stateName, TypeSerializer seri DEFAULT_GROW_RATE); } - public PartitionedBloomFilterDescriptor(String stateName, TypeSerializer serializer, long capacity, double fpp, long ttl, long minExpectNum, long maxExpectNum, double growRate) { + public ElasticFilterStateDescriptor(String stateName, TypeSerializer serializer, long capacity, double fpp, long ttl, long minExpectNum, long maxExpectNum, double growRate) { this.stateName = stateName; this.serializer = serializer; this.capacity = capacity; @@ -119,30 +119,6 @@ public double getGrowRate() { return growRate; } -// public void snapshot(DataOutputViewStreamWrapper out) throws IOException { -// out.writeUTF(stateName); -// out.writeLong(capacity); -// out.writeLong(ttl); -// out.writeDouble(fpp); -// -// out.writeLong(miniExpectNum); -// out.writeLong(maxExpectNum); -// out.writeDouble(growRate); -// } -// -// public static BloomFilterStateDescriptor restore(DataInputViewStreamWrapper in) throws IOException { -// BloomFilterStateDescriptor desc = new BloomFilterStateDescriptor(); -// desc.stateName = in.readUTF(); -// desc.capacity = in.readLong(); -// desc.ttl = in.readLong(); -// desc.fpp = in.readDouble(); -// -// desc.miniExpectNum = in.readLong(); -// desc.maxExpectNum = in.readLong(); -// desc.growRate = in.readDouble(); -// return desc; -// } - public void initializeSerializerUnlessSet(ExecutionConfig executionConfig) { if (serializer == null) { if (typeInfo != null) { diff --git a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/bloomfilter/BloomFilterExample.java b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/bloomfilter/BloomFilterExample.java index 05b82e9a3e4da..8ffda17db5422 100644 --- a/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/bloomfilter/BloomFilterExample.java +++ b/flink-examples/flink-examples-streaming/src/main/java/org/apache/flink/streaming/examples/bloomfilter/BloomFilterExample.java @@ -1,6 +1,6 @@ package org.apache.flink.streaming.examples.bloomfilter; -import org.apache.flink.api.common.state.PartitionedBloomFilterDescriptor; +import org.apache.flink.api.common.state.ElasticFilterStateDescriptor; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.api.java.functions.KeySelector; import org.apache.flink.configuration.Configuration; @@ -9,7 +9,7 @@ import org.apache.flink.streaming.api.environment.StreamExecutionEnvironment; import org.apache.flink.streaming.api.functions.source.SourceFunction; import org.apache.flink.streaming.api.functions.windowing.RichWindowFunction; -import org.apache.flink.streaming.api.operators.ElasticBloomFilter; +import org.apache.flink.streaming.api.operators.ElasticFilterState; import org.apache.flink.streaming.api.operators.StreamingRuntimeContext; import org.apache.flink.streaming.api.windowing.time.Time; import org.apache.flink.streaming.api.windowing.windows.TimeWindow; @@ -74,20 +74,20 @@ public Integer getKey(Integer value) throws Exception { }) .timeWindow(Time.milliseconds(1000)) .apply(new RichWindowFunction() { - private ElasticBloomFilter bf1; - private ElasticBloomFilter bf2; + private ElasticFilterState bf1; + private ElasticFilterState bf2; @Override public void open(Configuration parameters) throws Exception { super.open(parameters); StreamingRuntimeContext runtimeContext = (StreamingRuntimeContext) this.getRuntimeContext(); - bf1 = runtimeContext.getPartitionedBloomFilter(new PartitionedBloomFilterDescriptor( + bf1 = runtimeContext.getPartitionedBloomFilter(new ElasticFilterStateDescriptor( "0.01", TypeInformation.of(String.class), 60000, 0.01)); - bf2 = runtimeContext.getPartitionedBloomFilter(new PartitionedBloomFilterDescriptor( + bf2 = runtimeContext.getPartitionedBloomFilter(new ElasticFilterStateDescriptor( "0.02", TypeInformation.of(String.class), 60000, diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java index e1e2862d7e7ee..bf92e721cdc96 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/AbstractStreamOperator.java @@ -156,7 +156,7 @@ public abstract class AbstractStreamOperator // ---------------- bloom filter handler ------------------ /** Backend for bloomfilter. */ - private ElasticBloomFilterManager bloomFilterManager; + private ElasticFilterStateManager bloomFilterManager; // ---------------- two-input operator watermarks ------------------ @@ -628,7 +628,7 @@ public KeyedStateStore getKeyedStateStore() { return keyedStateStore; } - protected ElasticBloomFilterManager getBloomFilterStateManager() { + protected ElasticFilterStateManager getBloomFilterStateManager() { return bloomFilterManager; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedTolerantFilter.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticFilter.java similarity index 94% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedTolerantFilter.java rename to flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticFilter.java index 8342f001d42b1..aa019715e6b0c 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/LinkedTolerantFilter.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticFilter.java @@ -34,20 +34,20 @@ * A list of {@link TolerantFilterNode} to avoid data skewed between key ranges. The size of nodes on the list * grow by a {@code growRate} to avoid the list to be too long. */ -public class LinkedTolerantFilter { +public class ElasticFilter { - private static final Logger LOG = LoggerFactory.getLogger(LinkedTolerantFilter.class); + private static final Logger LOG = LoggerFactory.getLogger(ElasticFilter.class); private long currentSize; private long initSize; private double growRate; - private ElasticBloomFilter partitionedBloomFilter; + private ElasticFilterState partitionedBloomFilter; private LinkedList bloomFilterNodes = new LinkedList<>(); - public LinkedTolerantFilter(ElasticBloomFilter partitionedBloomFilter, long initSize, double growRate) { + public ElasticFilter(ElasticFilterState partitionedBloomFilter, long initSize, double growRate) { this.partitionedBloomFilter = partitionedBloomFilter; this.currentSize = initSize; this.initSize = initSize; @@ -166,7 +166,7 @@ public boolean equals(Object o) { if (this == o) return true; if (o == null || getClass() != o.getClass()) return false; - LinkedTolerantFilter that = (LinkedTolerantFilter) o; + ElasticFilter that = (ElasticFilter) o; if (currentSize != that.currentSize) return false; if (initSize != that.initSize) return false; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticBloomFilter.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticFilterState.java similarity index 80% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticBloomFilter.java rename to flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticFilterState.java index 174a99ab2b522..9ed52e5679f99 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticBloomFilter.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticFilterState.java @@ -25,8 +25,8 @@ import org.apache.flink.core.memory.DataOutputView; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; import org.apache.flink.runtime.operators.util.BloomFilter; +import org.apache.flink.runtime.state.KeyGroupRange; import org.apache.flink.runtime.state.KeyGroupRangeAssignment; -import org.apache.flink.runtime.state.KeyGroupsList; import org.apache.flink.util.FlinkRuntimeException; import org.slf4j.Logger; @@ -46,14 +46,14 @@ * @param The type of keys. * @param The type of values. */ -public class ElasticBloomFilter { +public class ElasticFilterState { - private static final Logger LOG = LoggerFactory.getLogger(ElasticBloomFilter.class); + private static final Logger LOG = LoggerFactory.getLogger(ElasticFilterState.class); private final int totalKeyGroups; - private final KeyGroupsList localKeyGroupRange; + private final KeyGroupRange localKeyGroupRange; private final KeyContext keyContext; - private LinkedTolerantFilter[] linkedBloomFilters; + private ElasticFilter[] linkedTolerantFilters; private final int localKeyGroupRangeStartIdx; private long totalMemSize; @@ -75,10 +75,10 @@ public class ElasticBloomFilter { private final ByteArrayOutputStreamWithPos valueSerializationStream; private final DataOutputView valueSerializationDataOutputView; - public ElasticBloomFilter(TypeSerializer keySerializer, + public ElasticFilterState(TypeSerializer keySerializer, TypeSerializer valueSerializer, int totalKeyGroups, - KeyGroupsList localKeyGroupRange, + KeyGroupRange localKeyGroupRange, KeyContext keyContext, long capacity, double fpp, @@ -99,14 +99,9 @@ public ElasticBloomFilter(TypeSerializer keySerializer, this.keyContext = keyContext; this.totalKeyGroups = totalKeyGroups; this.localKeyGroupRange = localKeyGroupRange; - this.linkedBloomFilters = new LinkedTolerantFilter[localKeyGroupRange.getNumberOfKeyGroups()]; + this.linkedTolerantFilters = new ElasticFilter[localKeyGroupRange.getNumberOfKeyGroups()]; - // find the starting index of the local key-group range - int startIdx = Integer.MAX_VALUE; - for (Integer keyGroupIdx : localKeyGroupRange) { - startIdx = Math.min(keyGroupIdx, startIdx); - } - this.localKeyGroupRangeStartIdx = startIdx; + this.localKeyGroupRangeStartIdx = localKeyGroupRange.getStartKeyGroup(); this.keySerializer = keySerializer; this.valueSerializer = valueSerializer; @@ -115,22 +110,22 @@ public ElasticBloomFilter(TypeSerializer keySerializer, } public void add(V content) { - int keyGroupIndex = KeyGroupRangeAssignment.assignToKeyGroup(keyContext.getCurrentKey(), totalKeyGroups); - int index = getIndexForKeyGroup(keyGroupIndex); + final int keyGroupIndex = KeyGroupRangeAssignment.assignToKeyGroup(keyContext.getCurrentKey(), totalKeyGroups); + final int index = getIndexForKeyGroup(keyGroupIndex); - LinkedTolerantFilter bloomFilter = linkedBloomFilters[index]; + ElasticFilter bloomFilter = linkedTolerantFilters[index]; if (bloomFilter == null) { - bloomFilter = new LinkedTolerantFilter(this, miniExpectNum, growRate); - linkedBloomFilters[index] = bloomFilter; + bloomFilter = new ElasticFilter(this, miniExpectNum, growRate); + linkedTolerantFilters[index] = bloomFilter; } bloomFilter.add(buildBloomFilterKey(content)); } public boolean contains(V content) { - int keyGroupIndex = KeyGroupRangeAssignment.assignToKeyGroup(keyContext.getCurrentKey(), totalKeyGroups); - int index = getIndexForKeyGroup(keyGroupIndex); + final int keyGroupIndex = KeyGroupRangeAssignment.assignToKeyGroup(keyContext.getCurrentKey(), totalKeyGroups); + final int index = getIndexForKeyGroup(keyGroupIndex); - LinkedTolerantFilter bloomFilter = linkedBloomFilters[index]; + ElasticFilter bloomFilter = linkedTolerantFilters[index]; if (bloomFilter == null) { return false; } @@ -147,7 +142,7 @@ public void snapshotStateForKeyGroup(DataOutputViewStreamWrapper stream, int key LOG.info("snapshot state for group {} ", keyGroupIdx); int index = getIndexForKeyGroup(keyGroupIdx); - LinkedTolerantFilter bloomFilter = this.linkedBloomFilters[index]; + ElasticFilter bloomFilter = this.linkedTolerantFilters[index]; if (bloomFilter != null) { stream.writeBoolean(true); stream.writeLong(this.restMemSize); @@ -165,9 +160,9 @@ public void restoreStateForKeyGroup( int index = getIndexForKeyGroup(keyGroupIdx); if (stream.readBoolean()) { this.restMemSize = stream.readLong(); - LinkedTolerantFilter linkedBloomFilter = new LinkedTolerantFilter(this, miniExpectNum, growRate); + ElasticFilter linkedBloomFilter = new ElasticFilter(this, miniExpectNum, growRate); linkedBloomFilter.restore(stream); - this.linkedBloomFilters[index] = linkedBloomFilter; + this.linkedTolerantFilters[index] = linkedBloomFilter; LOG.info("group {} restored.", keyGroupIdx); } else { LOG.info("nothing to restore."); @@ -236,7 +231,7 @@ public String toString() { .append("total memory:").append(totalMemSize).append("\t").append("rest memory:").append(restMemSize).append("\n"); for (int i = 0; i < localKeyGroupRange.getNumberOfKeyGroups(); ++i) { - LinkedTolerantFilter bloomFilter = this.linkedBloomFilters[i]; + ElasticFilter bloomFilter = this.linkedTolerantFilters[i]; if (bloomFilter != null) { builder.append("group ").append(i + localKeyGroupRangeStartIdx).append(":").append(bloomFilter.toString()).append("\n"); } @@ -245,7 +240,7 @@ public String toString() { } @VisibleForTesting - LinkedTolerantFilter[] getLinkedBloomFilters() { - return this.linkedBloomFilters; + ElasticFilter[] getLinkedTolerantFilters() { + return this.linkedTolerantFilters; } } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticBloomFilterManager.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticFilterStateManager.java similarity index 82% rename from flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticBloomFilterManager.java rename to flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticFilterStateManager.java index d0b556ecdb578..0844ae60a7fe9 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticBloomFilterManager.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ElasticFilterStateManager.java @@ -18,7 +18,7 @@ package org.apache.flink.streaming.api.operators; -import org.apache.flink.api.common.state.PartitionedBloomFilterDescriptor; +import org.apache.flink.api.common.state.ElasticFilterStateDescriptor; import org.apache.flink.api.common.typeutils.TypeSerializer; import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; @@ -35,16 +35,16 @@ import java.util.Map; /** - * Responsible for managing all {@link ElasticBloomFilter}. + * Responsible for managing all {@link ElasticFilterState}. * * @param The type of keys. */ -public class ElasticBloomFilterManager { +public class ElasticFilterStateManager { - private static final Logger LOG = LoggerFactory.getLogger(ElasticBloomFilterManager.class); + private static final Logger LOG = LoggerFactory.getLogger(ElasticFilterStateManager.class); - private Map bloomFilterStates = new HashMap<>(); - private Map bloomFilterStateDescriptors = new HashMap<>(); + private Map bloomFilterStates = new HashMap<>(); + private Map bloomFilterStateDescriptors = new HashMap<>(); private int numberOfKeyGroups; private KeyGroupRange keyGroupRange; private final KeyContext keyContext; @@ -52,7 +52,7 @@ public class ElasticBloomFilterManager { /** Serializer for the key. */ private final TypeSerializer keySerializer; - public ElasticBloomFilterManager( + public ElasticFilterStateManager( KeyContext keyContext, TypeSerializer keySerializer, int numberOfKeyGroups, @@ -65,11 +65,11 @@ public ElasticBloomFilterManager( } // ---------------------------------------------------- - public ElasticBloomFilter getOrCreateBloomFilterState(PartitionedBloomFilterDescriptor stateDescriptor) { + public ElasticFilterState getOrCreateBloomFilterState(ElasticFilterStateDescriptor stateDescriptor) { String stateName = stateDescriptor.getStateName(); - ElasticBloomFilter state = bloomFilterStates.get(stateName); + ElasticFilterState state = bloomFilterStates.get(stateName); if (state == null) { - state = new ElasticBloomFilter(keySerializer, + state = new ElasticFilterState(keySerializer, stateDescriptor.getSerializer(), numberOfKeyGroups, keyGroupRange, @@ -99,8 +99,8 @@ public void dispose() { public void snapshotStateForKeyGroup(DataOutputViewStreamWrapper stream, int keyGroupIdx) { try { stream.writeInt(this.bloomFilterStates.size()); - for (Map.Entry entry : this.bloomFilterStates.entrySet()) { - PartitionedBloomFilterDescriptor desc = this.bloomFilterStateDescriptors.get(entry.getKey()); + for (Map.Entry entry : this.bloomFilterStates.entrySet()) { + ElasticFilterStateDescriptor desc = this.bloomFilterStateDescriptors.get(entry.getKey()); ObjectOutputStream outputStream = new ObjectOutputStream(stream); outputStream.writeObject(desc); @@ -125,13 +125,13 @@ public void restoreStateForKeyGroup( int len = stream.readInt(); for (int i = 0; i < len; ++i) { ObjectInputStream inputStream = new ObjectInputStream(stream); - PartitionedBloomFilterDescriptor desc = (PartitionedBloomFilterDescriptor) inputStream.readObject(); + ElasticFilterStateDescriptor desc = (ElasticFilterStateDescriptor) inputStream.readObject(); - ElasticBloomFilter state = bloomFilterStates.get(desc.getStateName()); + ElasticFilterState state = bloomFilterStates.get(desc.getStateName()); LOG.info("restoring state [{}] for key group {}", desc.getStateName(), keyGroupIdx); if (state == null) { LOG.info("c:{} f:{} t:{} mie:{} mae:{} g:{}", desc.getCapacity(), desc.getFpp(), desc.getTtl(), desc.getMiniExpectNum(), desc.getMaxExpectNum(), desc.getGrowRate()); - state = new ElasticBloomFilter(keySerializer, + state = new ElasticFilterState(keySerializer, desc.getSerializer(), numberOfKeyGroups, keyGroupRange, diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNode.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNode.java index d220a72309d2b..c7715ead281f7 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNode.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/ShrinkableBloomFilterNode.java @@ -32,7 +32,7 @@ import java.util.Objects; /** - * A shrinkable bloom filter node linked each other in {@link LinkedTolerantFilter} to avoid data skewed. + * A shrinkable bloom filter node linked each other in {@link ElasticFilter} to avoid data skewed. */ public class ShrinkableBloomFilterNode implements TolerantFilterNode { @@ -115,7 +115,7 @@ public long deadline() { @Override public void add(byte[] content) { - int unitIndex = Arrays.hashCode(content) % bloomFilterUnits.length; + int unitIndex = Math.abs(Arrays.hashCode(content)) % bloomFilterUnits.length; BloomFilterUnit bloomFilterUnit = bloomFilterUnits[unitIndex]; @@ -125,7 +125,7 @@ public void add(byte[] content) { @Override public boolean contains(byte[] content) { - int unitIndex = Arrays.hashCode(content) % bloomFilterUnits.length; + int unitIndex = Math.abs(Arrays.hashCode(content)) % bloomFilterUnits.length; BloomFilterUnit bloomFilterUnit = bloomFilterUnits[unitIndex]; diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SimpleBloomFilterNode.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SimpleBloomFilterNode.java new file mode 100644 index 0000000000000..6e15b4702528f --- /dev/null +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/SimpleBloomFilterNode.java @@ -0,0 +1,109 @@ +/* + * 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.flink.streaming.api.operators; + +import org.apache.flink.core.memory.DataInputView; +import org.apache.flink.core.memory.DataOutputView; + +import java.io.IOException; + +/** + * To be continued. + */ +public class SimpleBloomFilterNode implements TolerantFilterNode { + + private int capacity; + private double fpp; + private long ttl; + private long deleteTS = Long.MAX_VALUE; + + SimpleBloomFilterNode() { + } + + public SimpleBloomFilterNode(int capacity, double fpp, long ttl) { + } + + @Override + public boolean full() { + return false; + } + + @Override + public long capacity() { + int capacity = 0; + return capacity; + } + + @Override + public long size() { + int size = 0; + return size; + } + + @Override + public long ttl() { + return ttl; + } + + @Override + public double fpp() { + return fpp; + } + + public void reSetTtl() { + this.deleteTS = Long.MAX_VALUE; + } + + @Override + public long deadline() { + return deleteTS; + } + + @Override + public void add(byte[] content) { + } + + @Override + public boolean contains(byte[] content) { + + return false; + } + + private int computeUnitCapacity(int capacity, int units) { + return Math.floorDiv(capacity, units); + } + + @Override + public String toString() { + return String.format("{c:%d s:%d}", capacity, size()); + } + + @Override + public boolean equals(Object obj) { + return false; + } + + @Override + public void snapshot(DataOutputView outputView) throws IOException { + } + + @Override + public void restore(DataInputView source) throws IOException { + } +} diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateContext.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateContext.java index d9f6d6f08b96b..6f465175e5bf5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateContext.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamOperatorStateContext.java @@ -51,7 +51,7 @@ public interface StreamOperatorStateContext { */ InternalTimeServiceManager internalTimerServiceManager(); - ElasticBloomFilterManager bloomFilterStateManager(); + ElasticFilterStateManager bloomFilterStateManager(); /** * Returns an iterable to obtain input streams for previously stored operator state partitions that are assigned to diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java index b43acd8eee79d..9b982d018f1a8 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamTaskStateInitializerImpl.java @@ -126,7 +126,7 @@ public StreamOperatorStateContext streamOperatorStateContext( CloseableIterable rawKeyedStateInputs = null; CloseableIterable rawOperatorStateInputs = null; InternalTimeServiceManager timeServiceManager; - ElasticBloomFilterManager bloomfilterStateManager; + ElasticFilterStateManager bloomfilterStateManager; try { @@ -230,7 +230,7 @@ protected InternalTimeServiceManager internalTimeServiceManager( return timeServiceManager; } - protected ElasticBloomFilterManager bloomfilterStateManager( + protected ElasticFilterStateManager bloomfilterStateManager( AbstractKeyedStateBackend keyedStatedBackend, KeyContext keyContext, //the operator Iterable rawKeyedStates) throws Exception { @@ -241,7 +241,7 @@ protected ElasticBloomFilterManager bloomfilterStateManager( final KeyGroupRange keyGroupRange = keyedStatedBackend.getKeyGroupRange(); - ElasticBloomFilterManager bloomFilterManager = new ElasticBloomFilterManager( + ElasticFilterStateManager bloomFilterManager = new ElasticFilterStateManager( keyContext, keyedStatedBackend.getKeySerializer(), keyedStatedBackend.getNumberOfKeyGroups(), @@ -583,7 +583,7 @@ private static class StreamOperatorStateContextImpl implements StreamOperatorSta private final OperatorStateBackend operatorStateBackend; private final AbstractKeyedStateBackend keyedStateBackend; private final InternalTimeServiceManager internalTimeServiceManager; - private final ElasticBloomFilterManager bloomFilterStateManager; + private final ElasticFilterStateManager bloomFilterStateManager; private final CloseableIterable rawOperatorStateInputs; private final CloseableIterable rawKeyedStateInputs; @@ -593,7 +593,7 @@ private static class StreamOperatorStateContextImpl implements StreamOperatorSta OperatorStateBackend operatorStateBackend, AbstractKeyedStateBackend keyedStateBackend, InternalTimeServiceManager internalTimeServiceManager, - ElasticBloomFilterManager bloomFilterStateManager, + ElasticFilterStateManager bloomFilterStateManager, CloseableIterable rawOperatorStateInputs, CloseableIterable rawKeyedStateInputs) { @@ -627,7 +627,7 @@ public InternalTimeServiceManager internalTimerServiceManager() { } @Override - public ElasticBloomFilterManager bloomFilterStateManager() { + public ElasticFilterStateManager bloomFilterStateManager() { return bloomFilterStateManager; } diff --git a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java index ed2bd5655d96c..b19d7ca5789d5 100644 --- a/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java +++ b/flink-streaming-java/src/main/java/org/apache/flink/streaming/api/operators/StreamingRuntimeContext.java @@ -31,7 +31,7 @@ import org.apache.flink.api.common.state.ListStateDescriptor; import org.apache.flink.api.common.state.MapState; import org.apache.flink.api.common.state.MapStateDescriptor; -import org.apache.flink.api.common.state.PartitionedBloomFilterDescriptor; +import org.apache.flink.api.common.state.ElasticFilterStateDescriptor; import org.apache.flink.api.common.state.ReducingState; import org.apache.flink.api.common.state.ReducingStateDescriptor; import org.apache.flink.api.common.state.StateDescriptor; @@ -171,7 +171,7 @@ public MapState getMapState(MapStateDescriptor statePro return keyedStateStore.getMapState(stateProperties); } - public ElasticBloomFilter getPartitionedBloomFilter(PartitionedBloomFilterDescriptor descriptor) { + public ElasticFilterState getPartitionedBloomFilter(ElasticFilterStateDescriptor descriptor) { descriptor.initializeSerializerUnlessSet(getExecutionConfig()); return operator.getBloomFilterStateManager().getOrCreateBloomFilterState(descriptor); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterManagerTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/ElasticFilterStateManagerTest.java similarity index 78% rename from flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterManagerTest.java rename to flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/ElasticFilterStateManagerTest.java index aab290d751575..0a579f6b4288a 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterManagerTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/ElasticFilterStateManagerTest.java @@ -1,7 +1,7 @@ package org.apache.flink.streaming.api.operators; import org.apache.flink.api.common.ExecutionConfig; -import org.apache.flink.api.common.state.PartitionedBloomFilterDescriptor; +import org.apache.flink.api.common.state.ElasticFilterStateDescriptor; import org.apache.flink.api.common.typeinfo.TypeInformation; import org.apache.flink.core.memory.DataInputViewStreamWrapper; import org.apache.flink.core.memory.DataOutputViewStreamWrapper; @@ -15,9 +15,9 @@ import java.util.Random; /** - * {@link ElasticBloomFilterManager} unit tests. + * {@link ElasticFilterStateManager} unit tests. */ -public class PartitionedBloomFilterManagerTest { +public class ElasticFilterStateManagerTest { @Test public void basicTest() throws Exception { @@ -36,14 +36,14 @@ public Object getCurrentKey() { } }; - ElasticBloomFilterManager partitionedBloomFilterManager = new ElasticBloomFilterManager( + ElasticFilterStateManager partitionedBloomFilterManager = new ElasticFilterStateManager( keyContext, TypeInformation.of(String.class).createSerializer(new ExecutionConfig()), 10, new KeyGroupRange(0, 9)); - ElasticBloomFilter partitionedBloomFilter = partitionedBloomFilterManager.getOrCreateBloomFilterState( - new PartitionedBloomFilterDescriptor( + ElasticFilterState partitionedBloomFilter = partitionedBloomFilterManager.getOrCreateBloomFilterState( + new ElasticFilterStateDescriptor( "test-bf", TypeInformation.of(Integer.class).createSerializer(new ExecutionConfig()), 100, 0.01, 10000)); @@ -72,24 +72,24 @@ public Object getCurrentKey() { } }; - ElasticBloomFilterManager partitionedBloomFilterManager = new ElasticBloomFilterManager( + ElasticFilterStateManager partitionedBloomFilterManager = new ElasticFilterStateManager( keyContext, TypeInformation.of(String.class).createSerializer(new ExecutionConfig()), 10, new KeyGroupRange(0, 9)); - PartitionedBloomFilterDescriptor desc1 = new PartitionedBloomFilterDescriptor( + ElasticFilterStateDescriptor desc1 = new ElasticFilterStateDescriptor( "test-bf-1", TypeInformation.of(Integer.class).createSerializer(new ExecutionConfig()), 10_000, 0.01, 60000); - PartitionedBloomFilterDescriptor desc2 = new PartitionedBloomFilterDescriptor( + ElasticFilterStateDescriptor desc2 = new ElasticFilterStateDescriptor( "test-bf-2", TypeInformation.of(Integer.class).createSerializer(new ExecutionConfig()), 10_000, 0.01, 60000); - ElasticBloomFilter partitionedBloomFilter1 = partitionedBloomFilterManager.getOrCreateBloomFilterState(desc1); - ElasticBloomFilter partitionedBloomFilter2 = partitionedBloomFilterManager.getOrCreateBloomFilterState(desc2); + ElasticFilterState partitionedBloomFilter1 = partitionedBloomFilterManager.getOrCreateBloomFilterState(desc1); + ElasticFilterState partitionedBloomFilter2 = partitionedBloomFilterManager.getOrCreateBloomFilterState(desc2); String[] keys1 = new String[10_000]; for (int i = 0; i < 10_000; ++i) { @@ -117,7 +117,7 @@ public Object getCurrentKey() { // restore ByteArrayInputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray()); DataInputViewStreamWrapper inputViewStreamWrapper = new DataInputViewStreamWrapper(inputStream); - ElasticBloomFilterManager partitionedBloomFilterManager2 = new ElasticBloomFilterManager( + ElasticFilterStateManager partitionedBloomFilterManager2 = new ElasticFilterStateManager( keyContext, TypeInformation.of(String.class).createSerializer(new ExecutionConfig()), 10, @@ -128,8 +128,8 @@ public Object getCurrentKey() { } // valid - ElasticBloomFilter partitionedBloomFilter3 = partitionedBloomFilterManager2.getOrCreateBloomFilterState(desc1); - ElasticBloomFilter partitionedBloomFilter4 = partitionedBloomFilterManager2.getOrCreateBloomFilterState(desc2); + ElasticFilterState partitionedBloomFilter3 = partitionedBloomFilterManager2.getOrCreateBloomFilterState(desc1); + ElasticFilterState partitionedBloomFilter4 = partitionedBloomFilterManager2.getOrCreateBloomFilterState(desc2); for (int i = 0; i < 10_000; ++i) { String key = keys1[i]; diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/ElasticFilterStateTest.java similarity index 77% rename from flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterTest.java rename to flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/ElasticFilterStateTest.java index 8490fd24ff165..cf630276b3c09 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/PartitionedBloomFilterTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/ElasticFilterStateTest.java @@ -15,14 +15,15 @@ import java.util.Random; /** - * {@link ElasticBloomFilter} unit tests. + * {@link ElasticFilterState} unit tests. */ -public class PartitionedBloomFilterTest { +public class ElasticFilterStateTest { @Test public void basicTest() throws InterruptedException { KeyContext keyContext = new KeyContext() { + private Object key; @Override @@ -36,7 +37,7 @@ public Object getCurrentKey() { } }; - ElasticBloomFilter partitionedBloomFilter = new ElasticBloomFilter( + ElasticFilterState elasticFilterState = new ElasticFilterState( new StringSerializer(), new IntSerializer(), 10, @@ -50,28 +51,29 @@ public Object getCurrentKey() { 2.0 ); - LinkedTolerantFilter[] linkedBloomFilters = partitionedBloomFilter.getLinkedBloomFilters(); - Assert.assertEquals(10, linkedBloomFilters.length); + ElasticFilter[] linkedTolerantFilters = elasticFilterState.getLinkedTolerantFilters(); + Assert.assertEquals(10, linkedTolerantFilters.length); for (int i = 0; i < 10; ++i) { - Assert.assertNull(linkedBloomFilters[i]); + Assert.assertNull(linkedTolerantFilters[i]); } String currentKey = "hello"; keyContext.setCurrentKey(currentKey); for (int i = 0; i < 1000; ++i) { - partitionedBloomFilter.add(i); - Assert.assertTrue(partitionedBloomFilter.contains(i)); + elasticFilterState.add(i); + Assert.assertTrue(elasticFilterState.contains(i)); } int currentGroup = KeyGroupRangeAssignment.assignToKeyGroup(currentKey, 10); - Assert.assertNotNull(linkedBloomFilters[currentGroup]); + Assert.assertNotNull(linkedTolerantFilters[currentGroup]); } @Test public void testSnapshotAndRestore() throws Exception { KeyContext keyContext = new KeyContext() { + private Object key; @Override @@ -85,22 +87,22 @@ public Object getCurrentKey() { } }; - ElasticBloomFilter partitionedBloomFilter = new ElasticBloomFilter( + ElasticFilterState partitionedBloomFilter = new ElasticFilterState( new StringSerializer(), new IntSerializer(), 10, new KeyGroupRange(0, 9), keyContext, - 10000, + 20000, 0.01, 1000, + 100, 1000, - 10000, 2.0 ); String[] keys = new String[10_000]; - for (int i = 0; i < 10_000; ++i) { + for (int i = 0; i < 10000; ++i) { String key = String.valueOf(new Random().nextInt(1000)); keyContext.setCurrentKey(key); partitionedBloomFilter.add(i); @@ -115,22 +117,23 @@ public Object getCurrentKey() { partitionedBloomFilter.snapshotStateForKeyGroup(outputViewStreamWrapper, i); } - ElasticBloomFilter partitionedBloomFilter2 = new ElasticBloomFilter( + ElasticFilterState partitionedBloomFilter2 = new ElasticFilterState( new StringSerializer(), new IntSerializer(), 10, new KeyGroupRange(0, 9), keyContext, - 10000, + 20000, 0.01, 1000, + 100, 1000, - 10000, 2.0 ); ByteArrayInputStream inputStream = new ByteArrayInputStream(outputStream.toByteArray()); DataInputViewStreamWrapper inputViewStreamWrapper = new DataInputViewStreamWrapper(inputStream); + // restore one by one for (int i = 0; i < 10; ++i) { partitionedBloomFilter2.restoreStateForKeyGroup(inputViewStreamWrapper, i); diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedTolerantFilterTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/ElasticFilterTest.java similarity index 83% rename from flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedTolerantFilterTest.java rename to flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/ElasticFilterTest.java index 793f4df39ef5e..f01f512b53948 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedTolerantFilterTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/ElasticFilterTest.java @@ -16,9 +16,9 @@ import java.util.concurrent.TimeUnit; /** - * {@link LinkedTolerantFilter} unit tests. + * {@link ElasticFilter} unit tests. */ -public class LinkedTolerantFilterTest { +public class ElasticFilterTest { @Test public void basicTest() throws InterruptedException { @@ -27,7 +27,7 @@ public void basicTest() throws InterruptedException { private static final long serialVersionUID = 1L; }; - ElasticBloomFilter partitionedBloomFilter = new ElasticBloomFilter( + ElasticFilterState partitionedBloomFilter = new ElasticFilterState( new StringSerializer(), new IntSerializer(), 10, @@ -41,7 +41,7 @@ public void basicTest() throws InterruptedException { 2.0 ); - LinkedTolerantFilter linkedBloomFilter = new LinkedTolerantFilter(partitionedBloomFilter, 1000, 2); + ElasticFilter linkedBloomFilter = new ElasticFilter(partitionedBloomFilter, 1000, 2); List nodes = linkedBloomFilter.getBloomFilterNodes(); @@ -69,7 +69,7 @@ public void testSnapshotAndRestore() throws InterruptedException, IOException { private static final long serialVersionUID = 1L; }; - ElasticBloomFilter partitionedBloomFilter = new ElasticBloomFilter( + ElasticFilterState partitionedBloomFilter = new ElasticFilterState( new StringSerializer(), new IntSerializer(), 10, @@ -83,7 +83,7 @@ public void testSnapshotAndRestore() throws InterruptedException, IOException { 2.0 ); - LinkedTolerantFilter linkedBloomFilter = new LinkedTolerantFilter(partitionedBloomFilter, 1000, 2); + ElasticFilter linkedBloomFilter = new ElasticFilter(partitionedBloomFilter, 1000, 2); List nodes = linkedBloomFilter.getBloomFilterNodes(); @@ -114,7 +114,7 @@ public void testSnapshotAndRestore() throws InterruptedException, IOException { byte[] outputBytes = outputStream.toByteArray(); - LinkedTolerantFilter linkedBloomFilter2 = new LinkedTolerantFilter(partitionedBloomFilter, 1000, 2); + ElasticFilter linkedBloomFilter2 = new ElasticFilter(partitionedBloomFilter, 1000, 2); linkedBloomFilter2.restore(new DataInputViewStreamWrapper(new ByteArrayInputStream(outputBytes))); @@ -129,7 +129,7 @@ public void testTTL() throws InterruptedException, IOException { private static final long serialVersionUID = 1L; }; - ElasticBloomFilter partitionedBloomFilter = new ElasticBloomFilter( + ElasticFilterState partitionedBloomFilter = new ElasticFilterState( new StringSerializer(), new IntSerializer(), 10, @@ -143,7 +143,7 @@ public void testTTL() throws InterruptedException, IOException { 2.0 ); - LinkedTolerantFilter linkedBloomFilter = new LinkedTolerantFilter(partitionedBloomFilter, 1000, 2); + ElasticFilter linkedBloomFilter = new ElasticFilter(partitionedBloomFilter, 1000, 2); List nodes = linkedBloomFilter.getBloomFilterNodes(); @@ -180,7 +180,7 @@ public void testTTL() throws InterruptedException, IOException { TimeUnit.MILLISECONDS.sleep(2000); - linkedBloomFilter = new LinkedTolerantFilter(partitionedBloomFilter, 1000, 2); + linkedBloomFilter = new ElasticFilter(partitionedBloomFilter, 1000, 2); linkedBloomFilter.restore(new DataInputViewStreamWrapper(new ByteArrayInputStream(outputStream.toByteArray()))); Assert.assertEquals(2, linkedBloomFilter.getBloomFilterNodes().size()); } diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNodeTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNodeTest.java deleted file mode 100644 index ec914f245bc28..0000000000000 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/api/operators/LinkedBloomFilterNodeTest.java +++ /dev/null @@ -1,59 +0,0 @@ -//package org.apache.flink.streaming.api.operators; -// -//import org.apache.flink.core.memory.DataInputViewStreamWrapper; -//import org.apache.flink.core.memory.DataOutputViewStreamWrapper; -// -//import org.junit.Assert; -//import org.junit.Test; -// -//import java.io.ByteArrayInputStream; -//import java.io.ByteArrayOutputStream; -//import java.io.IOException; -//import java.util.concurrent.TimeUnit; -// -///** -// * {@link ElasticBloomFilterNode} unit tests. -// */ -//public class LinkedBloomFilterNodeTest { -// -// @Test -// public void basicTest() throws InterruptedException { -// -// ElasticBloomFilterNode node = new ElasticBloomFilterNode(100, 0.01, 1000); -// -// Assert.assertEquals(Long.MAX_VALUE, node.getDeleteTS()); -// -// for (int i = 0; i < 100; ++i) { -// node.add(String.valueOf(i).getBytes()); -// Assert.assertTrue(node.contains(String.valueOf(i).getBytes())); -// } -// Assert.assertTrue(node.isFull()); -// TimeUnit.MILLISECONDS.sleep(1000); -// Assert.assertNotEquals(Long.MAX_VALUE, node.getDeleteTS()); -// } -// -// @Test -// public void testSnapshotAndRestore() throws InterruptedException, IOException { -// -// ElasticBloomFilterNode node1 = new ElasticBloomFilterNode(100, 0.01, 1000); -// for (int i = 0; i < 100; ++i) { -// node1.add(String.valueOf(i).getBytes()); -// Assert.assertTrue(node1.contains(String.valueOf(i).getBytes())); -// } -// -// ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); -// DataOutputViewStreamWrapper outputViewStreamWrapper = new DataOutputViewStreamWrapper(outputStream); -// -// node1.snapshot(outputViewStreamWrapper); -// -// byte[] outputBytes = outputStream.toByteArray(); -// -// ElasticBloomFilterNode node2 = new ElasticBloomFilterNode(); -// node2.restore(new DataInputViewStreamWrapper(new ByteArrayInputStream(outputBytes))); -// -// Assert.assertEquals(node1.getCapacity(), node2.getCapacity()); -// Assert.assertEquals(String.valueOf(node1.getFpp()), String.valueOf(node2.getFpp())); -// Assert.assertEquals(node1.getSize(), node2.getSize()); -// Assert.assertEquals(node1.getBloomFilter(), node2.getBloomFilter()); -// } -//} diff --git a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java index 9a5c705020372..bb2af11348d18 100644 --- a/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java +++ b/flink-streaming-java/src/test/java/org/apache/flink/streaming/runtime/tasks/StreamTaskTest.java @@ -101,7 +101,7 @@ import org.apache.flink.streaming.api.operators.OperatorSnapshotFinalizer; import org.apache.flink.streaming.api.operators.OperatorSnapshotFutures; import org.apache.flink.streaming.api.operators.Output; -import org.apache.flink.streaming.api.operators.ElasticBloomFilterManager; +import org.apache.flink.streaming.api.operators.ElasticFilterStateManager; import org.apache.flink.streaming.api.operators.StreamOperator; import org.apache.flink.streaming.api.operators.StreamOperatorStateContext; import org.apache.flink.streaming.api.operators.StreamSource; @@ -1115,7 +1115,7 @@ public InternalTimeServiceManager internalTimerServiceManager() { } @Override - public ElasticBloomFilterManager bloomFilterStateManager() { + public ElasticFilterStateManager bloomFilterStateManager() { return context.bloomFilterStateManager(); }