Skip to content
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
merged 9 commits into from
Apr 22, 2019
Merged
Show file tree
Hide file tree
Changes from 8 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@

import org.apache.pulsar.broker.service.schema.IncompatibleSchemaException;
import org.apache.pulsar.common.api.proto.PulsarApi;
import org.apache.pulsar.common.api.proto.PulsarApi.ServerError;

/**
* Base type of exception thrown by Pulsar Broker Service
Expand Down Expand Up @@ -146,6 +147,12 @@ public AlreadyRunningException(String msg) {
}
}

public static class ConsumerAssignException extends BrokerServiceException {
public ConsumerAssignException(String msg) {
super(msg);
}
}

public static PulsarApi.ServerError getClientErrorCode(Throwable t) {
if (t instanceof ServerMetadataException) {
return PulsarApi.ServerError.MetadataError;
Expand All @@ -166,6 +173,8 @@ public static PulsarApi.ServerError getClientErrorCode(Throwable t) {
return PulsarApi.ServerError.ServiceNotReady;
} else if (t instanceof IncompatibleSchemaException) {
return PulsarApi.ServerError.IncompatibleSchema;
} else if (t instanceof ConsumerAssignException) {
return ServerError.ConsumerAssignError;
} else {
return PulsarApi.ServerError.UnknownError;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -145,7 +145,7 @@ public Consumer(Subscription subscription, SubType subType, String topicName, lo
stats.setClientVersion(cnx.getClientVersion());
stats.metadata = this.metadata;

if (subType == SubType.Shared) {
if (subType == SubType.Shared || subType == SubType.Key_Shared) {
this.pendingAcks = new ConcurrentLongLongPairHashMap(256, 1);
} else {
// We don't need to keep track of pending acks if the subscription is not shared
Expand Down
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.
Copy link
Member

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.

*
* 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;
}
}
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);

}
Original file line number Diff line number Diff line change
Expand Up @@ -65,8 +65,8 @@
*/
public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMultipleConsumers implements Dispatcher, ReadEntriesCallback {

private final PersistentTopic topic;
private final ManagedCursor cursor;
protected final PersistentTopic topic;
protected final ManagedCursor cursor;

private CompletableFuture<Void> closeFuture = null;
LongPairSet messagesToReplay = new ConcurrentSortedLongPairSet(128, 2);
Expand All @@ -75,9 +75,9 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMu
private boolean havePendingRead = false;
private boolean havePendingReplayRead = false;
private boolean shouldRewindBeforeReadingOrReplaying = false;
private final String name;
protected final String name;

private int totalAvailablePermits = 0;
protected int totalAvailablePermits = 0;
private int readBatchSize;
private final Backoff readFailureBackoff = new Backoff(15, TimeUnit.SECONDS, 1, TimeUnit.MINUTES, 0, TimeUnit.MILLISECONDS);
private static final AtomicIntegerFieldUpdater<PersistentDispatcherMultipleConsumers> TOTAL_UNACKED_MESSAGES_UPDATER =
Expand All @@ -87,8 +87,8 @@ public class PersistentDispatcherMultipleConsumers extends AbstractDispatcherMu
private volatile int blockedDispatcherOnUnackedMsgs = FALSE;
private static final AtomicIntegerFieldUpdater<PersistentDispatcherMultipleConsumers> BLOCKED_DISPATCHER_ON_UNACKMSG_UPDATER =
AtomicIntegerFieldUpdater.newUpdater(PersistentDispatcherMultipleConsumers.class, "blockedDispatcherOnUnackedMsgs");
private final ServiceConfiguration serviceConfig;
private Optional<DispatchRateLimiter> dispatchRateLimiter = Optional.empty();
protected final ServiceConfiguration serviceConfig;
protected Optional<DispatchRateLimiter> dispatchRateLimiter = Optional.empty();

enum ReadType {
Normal, Replay
Expand Down Expand Up @@ -374,9 +374,6 @@ public SubType getType() {
@Override
public synchronized void readEntriesComplete(List<Entry> entries, Object ctx) {
ReadType readType = (ReadType) ctx;
int start = 0;
int entriesToDispatch = entries.size();

if (readType == ReadType.Normal) {
havePendingRead = false;
} else {
Expand Down Expand Up @@ -407,8 +404,17 @@ public synchronized void readEntriesComplete(List<Entry> entries, Object ctx) {
log.debug("[{}] Distributing {} messages to {} consumers", name, entries.size(), consumerList.size());
}

sendMessagesToConsumers(readType, entries);

readMoreEntries();
}

protected void sendMessagesToConsumers(ReadType readType, List<Entry> entries) {
int start = 0;
int entriesToDispatch = entries.size();
long totalMessagesSent = 0;
long totalBytesSent = 0;

while (entriesToDispatch > 0 && totalAvailablePermits > 0 && isAtleastOneConsumerAvailable()) {
Consumer c = getNextConsumer();
if (c == null) {
Expand All @@ -421,8 +427,8 @@ public synchronized void readEntriesComplete(List<Entry> entries, Object ctx) {

// round-robin dispatch batch size for this consumer
int messagesForC = Math.min(
Math.min(entriesToDispatch, c.getAvailablePermits()),
serviceConfig.getDispatcherMaxRoundRobinBatchSize());
Math.min(entriesToDispatch, c.getAvailablePermits()),
serviceConfig.getDispatcherMaxRoundRobinBatchSize());

if (messagesForC > 0) {

Expand Down Expand Up @@ -465,8 +471,6 @@ public synchronized void readEntriesComplete(List<Entry> entries, Object ctx) {
entry.release();
});
}

readMoreEntries();
}

@Override
Expand Down Expand Up @@ -531,7 +535,7 @@ public synchronized void readEntriesFailed(ManagedLedgerException exception, Obj
*
* @return
*/
private boolean isAtleastOneConsumerAvailable() {
protected boolean isAtleastOneConsumerAvailable() {
if (consumerList.isEmpty() || IS_CLOSED_UPDATER.get(this) == TRUE) {
// abort read if no consumers are connected or if disconnect is initiated
return false;
Expand Down
Loading