forked from apache/pulsar
-
Notifications
You must be signed in to change notification settings - Fork 6
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[improve][broker] Make MessageRedeliveryController work more efficien…
…tly (apache#17804) (cherry picked from commit c60f895)
- Loading branch information
1 parent
aa68ef8
commit 19eb842
Showing
7 changed files
with
391 additions
and
41 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
143 changes: 143 additions & 0 deletions
143
pulsar-broker/src/main/java/org/apache/pulsar/utils/ConcurrentBitmapSortedLongPairSet.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,143 @@ | ||
/** | ||
* 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.pulsar.utils; | ||
|
||
import java.util.Iterator; | ||
import java.util.Map; | ||
import java.util.NavigableMap; | ||
import java.util.NavigableSet; | ||
import java.util.Set; | ||
import java.util.TreeMap; | ||
import java.util.TreeSet; | ||
import java.util.concurrent.locks.ReadWriteLock; | ||
import java.util.concurrent.locks.ReentrantReadWriteLock; | ||
import org.apache.pulsar.common.util.collections.LongPairSet; | ||
import org.roaringbitmap.RoaringBitmap; | ||
|
||
public class ConcurrentBitmapSortedLongPairSet { | ||
|
||
private final NavigableMap<Long, RoaringBitmap> map = new TreeMap<>(); | ||
private final ReadWriteLock lock = new ReentrantReadWriteLock(); | ||
|
||
public void add(long item1, long item2) { | ||
lock.writeLock().lock(); | ||
try { | ||
RoaringBitmap bitSet = map.computeIfAbsent(item1, k -> new RoaringBitmap()); | ||
bitSet.add(item2, item2 + 1); | ||
} finally { | ||
lock.writeLock().unlock(); | ||
} | ||
} | ||
|
||
public void remove(long item1, long item2) { | ||
lock.writeLock().lock(); | ||
try { | ||
RoaringBitmap bitSet = map.get(item1); | ||
if (bitSet != null) { | ||
bitSet.remove(item2, item2 + 1); | ||
if (bitSet.isEmpty()) { | ||
map.remove(item1, bitSet); | ||
} | ||
} | ||
} finally { | ||
lock.writeLock().unlock(); | ||
} | ||
} | ||
|
||
public boolean contains(long item1, long item2) { | ||
lock.readLock().lock(); | ||
try { | ||
RoaringBitmap bitSet = map.get(item1); | ||
return bitSet != null && bitSet.contains(item2, item2 + 1); | ||
} finally { | ||
lock.readLock().unlock(); | ||
} | ||
} | ||
|
||
public void removeUpTo(long item1, long item2) { | ||
lock.writeLock().lock(); | ||
try { | ||
Map.Entry<Long, RoaringBitmap> firstEntry = map.firstEntry(); | ||
while (firstEntry != null && firstEntry.getKey() <= item1) { | ||
if (firstEntry.getKey() < item1) { | ||
map.remove(firstEntry.getKey(), firstEntry.getValue()); | ||
} else { | ||
RoaringBitmap bitSet = firstEntry.getValue(); | ||
if (bitSet != null) { | ||
bitSet.remove(0, item2); | ||
if (bitSet.isEmpty()) { | ||
map.remove(firstEntry.getKey(), bitSet); | ||
} | ||
} | ||
break; | ||
} | ||
firstEntry = map.firstEntry(); | ||
} | ||
} finally { | ||
lock.writeLock().unlock(); | ||
} | ||
} | ||
|
||
|
||
public <T> Set<T> items(int numberOfItems, LongPairSet.LongPairFunction<T> longPairConverter) { | ||
NavigableSet<T> items = new TreeSet<>(); | ||
lock.readLock().lock(); | ||
try { | ||
for (Map.Entry<Long, RoaringBitmap> entry : map.entrySet()) { | ||
Iterator<Integer> iterator = entry.getValue().stream().iterator(); | ||
while (iterator.hasNext() && items.size() < numberOfItems) { | ||
items.add(longPairConverter.apply(entry.getKey(), iterator.next())); | ||
} | ||
if (items.size() == numberOfItems) { | ||
break; | ||
} | ||
} | ||
} finally { | ||
lock.readLock().unlock(); | ||
} | ||
return items; | ||
} | ||
|
||
public boolean isEmpty() { | ||
lock.readLock().lock(); | ||
try { | ||
return map.isEmpty() || map.values().stream().allMatch(RoaringBitmap::isEmpty); | ||
} finally { | ||
lock.readLock().unlock(); | ||
} | ||
} | ||
|
||
public void clear() { | ||
lock.writeLock().lock(); | ||
try { | ||
map.clear(); | ||
} finally { | ||
lock.writeLock().unlock(); | ||
} | ||
} | ||
|
||
public int size() { | ||
lock.readLock().lock(); | ||
try { | ||
return map.isEmpty() ? 0 : map.values().stream().mapToInt(RoaringBitmap::getCardinality).sum(); | ||
} finally { | ||
lock.readLock().unlock(); | ||
} | ||
} | ||
} |
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Oops, something went wrong.