-
Notifications
You must be signed in to change notification settings - Fork 3.6k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
PIP-34 Key_Shared subscription core implementation. #4079
Merged
jiazhai
merged 9 commits into
apache:master
from
codelipenghui:feature_key_shared_subscription
Apr 22, 2019
Merged
Changes from 8 commits
Commits
Show all changes
9 commits
Select commit
Hold shift + click to select a range
df0c589
PIP-34 Key_Shared subscription core implementation.
codelipenghui 77142fc
PIP-34 Add more unit test.
codelipenghui 181696e
PIP-34 Fix alignment issue of Pulsar.proto
codelipenghui 625e359
PIP-34 Fix alignment issue of Pulsar.proto
codelipenghui 7fe81b6
PIP-34 Fix TODO: format
codelipenghui 4fdce72
PIP-34 Fix hash and ordering key issues
codelipenghui 3088932
PIP-34 documentation for Key_Shared subscription
codelipenghui 92f561f
PIP-34 Fix cpp test issue.
codelipenghui 0f74d9d
PIP-34 Fix cpp format issue.
codelipenghui File filter
Filter by extension
Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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
149 changes: 149 additions & 0 deletions
149
...er/src/main/java/org/apache/pulsar/broker/service/HashRangeStickyKeyConsumerSelector.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,149 @@ | ||
/** | ||
* 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.broker.service; | ||
|
||
import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerAssignException; | ||
import org.apache.pulsar.common.util.Murmur3_32Hash; | ||
|
||
import java.util.HashMap; | ||
import java.util.Map; | ||
import java.util.Map.Entry; | ||
import java.util.concurrent.ConcurrentSkipListMap; | ||
|
||
/** | ||
* This is a consumer selector based fixed hash range. | ||
* | ||
* 1.Each consumer serves a fixed range of hash value | ||
* 2.The whole range of hash value could be covered by all the consumers. | ||
* 3.Once a consumer is removed, the left consumers could still serve the whole range. | ||
* | ||
* Initializing with a fixed hash range, by default 2 << 5. | ||
* First consumer added, hash range looks like: | ||
* | ||
* 0 -> 65536(consumer-1) | ||
* | ||
* Second consumer added, will find a biggest range to split: | ||
* | ||
* 0 -> 32768(consumer-2) -> 65536(consumer-1) | ||
* | ||
* While a consumer removed, The range for this consumer will be taken over | ||
* by other consumer, consumer-2 removed: | ||
* | ||
* 0 -> 65536(consumer-1) | ||
* | ||
* In this approach use skip list map to maintain the hash range and consumers. | ||
* | ||
* Select consumer will return the ceiling key of message key hashcode % range size. | ||
* | ||
*/ | ||
public class HashRangeStickyKeyConsumerSelector implements StickyKeyConsumerSelector { | ||
|
||
public static final int DEFAULT_RANGE_SIZE = 2 << 15; | ||
|
||
private final int rangeSize; | ||
|
||
private final ConcurrentSkipListMap<Integer, Consumer> rangeMap; | ||
private final Map<Consumer, Integer> consumerRange; | ||
|
||
public HashRangeStickyKeyConsumerSelector() { | ||
this(DEFAULT_RANGE_SIZE); | ||
} | ||
|
||
public HashRangeStickyKeyConsumerSelector(int rangeSize) { | ||
if (rangeSize < 2) { | ||
throw new IllegalArgumentException("range size must greater than 2"); | ||
} | ||
if (!is2Power(rangeSize)) { | ||
throw new IllegalArgumentException("range size must be nth power with 2"); | ||
} | ||
this.rangeMap = new ConcurrentSkipListMap<>(); | ||
this.consumerRange = new HashMap<>(); | ||
this.rangeSize = rangeSize; | ||
} | ||
|
||
@Override | ||
public synchronized void addConsumer(Consumer consumer) throws ConsumerAssignException { | ||
if (rangeMap.size() == 0) { | ||
rangeMap.put(rangeSize, consumer); | ||
consumerRange.put(consumer, rangeSize); | ||
} else { | ||
splitRange(findBiggestRange(), consumer); | ||
} | ||
} | ||
|
||
@Override | ||
public synchronized void removeConsumer(Consumer consumer) { | ||
Integer removeRange = consumerRange.get(consumer); | ||
if (removeRange != null) { | ||
if (removeRange == rangeSize && rangeMap.size() > 1) { | ||
Consumer lowerConsumer = rangeMap.lowerEntry(removeRange).getValue(); | ||
rangeMap.put(removeRange, lowerConsumer); | ||
consumerRange.put(lowerConsumer, removeRange); | ||
} else { | ||
rangeMap.remove(removeRange); | ||
consumerRange.remove(consumer); | ||
} | ||
} | ||
} | ||
|
||
@Override | ||
public Consumer select(byte[] stickyKey) { | ||
if (rangeMap.size() > 0) { | ||
int slot = Murmur3_32Hash.getInstance().makeHash(stickyKey) % rangeSize; | ||
return rangeMap.ceilingEntry(slot).getValue(); | ||
} else { | ||
return null; | ||
} | ||
} | ||
|
||
private int findBiggestRange() { | ||
int slots = 0; | ||
int busiestRange = rangeSize; | ||
for (Entry<Integer, Consumer> entry : rangeMap.entrySet()) { | ||
Integer lowerKey = rangeMap.lowerKey(entry.getKey()); | ||
if (lowerKey == null) { | ||
lowerKey = 0; | ||
} | ||
if (entry.getKey() - lowerKey > slots) { | ||
slots = entry.getKey() - lowerKey; | ||
busiestRange = entry.getKey(); | ||
} | ||
} | ||
return busiestRange; | ||
} | ||
|
||
private void splitRange(int range, Consumer targetConsumer) throws ConsumerAssignException { | ||
Integer lowerKey = rangeMap.lowerKey(range); | ||
if (lowerKey == null) { | ||
lowerKey = 0; | ||
} | ||
if (range - lowerKey <= 1) { | ||
throw new ConsumerAssignException("No more range can assigned to new consumer, assigned consumers " | ||
+ rangeMap.size()); | ||
} | ||
int splitRange = range - ((range - lowerKey) >> 1); | ||
rangeMap.put(splitRange, targetConsumer); | ||
consumerRange.put(targetConsumer, splitRange); | ||
} | ||
|
||
private boolean is2Power(int num) { | ||
if(num < 2) return false; | ||
return (num & num - 1) == 0; | ||
} | ||
} |
45 changes: 45 additions & 0 deletions
45
pulsar-broker/src/main/java/org/apache/pulsar/broker/service/StickyKeyConsumerSelector.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,45 @@ | ||
/** | ||
* 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.broker.service; | ||
|
||
import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerAssignException; | ||
|
||
public interface StickyKeyConsumerSelector { | ||
|
||
/** | ||
* Add a new consumer | ||
* @param consumer new consumer | ||
*/ | ||
void addConsumer(Consumer consumer) throws ConsumerAssignException; | ||
|
||
/** | ||
* Remove the consumer | ||
* @param consumer consumer to be removed | ||
*/ | ||
void removeConsumer(Consumer consumer); | ||
|
||
/** | ||
* Select a consumer by sticky key | ||
* | ||
* @param stickyKey sticky key | ||
* @return consumer | ||
*/ | ||
Consumer select(byte[] stickyKey); | ||
|
||
} |
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.
Add this suggestion to a batch that can be applied as a single commit.
This suggestion is invalid because no changes were made to the code.
Suggestions cannot be applied while the pull request is closed.
Suggestions cannot be applied while viewing a subset of changes.
Only one suggestion per line can be applied in a batch.
Add this suggestion to a batch that can be applied as a single commit.
Applying suggestions on deleted lines is not supported.
You must change the existing code in this line in order to create a valid suggestion.
Outdated suggestions cannot be applied.
This suggestion has been applied or marked resolved.
Suggestions cannot be applied from pending reviews.
Suggestions cannot be applied on multi-line comments.
Suggestions cannot be applied while the pull request is queued to merge.
Suggestion cannot be applied right now. Please check back later.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
nit: I would suggest you adding more descriptions into the javadoc. It might be worthing using an example to explain how does your algorithm work here.