-
Notifications
You must be signed in to change notification settings - Fork 3.5k
/
NonPersistentDispatcherMultipleConsumers.java
223 lines (193 loc) · 8.28 KB
/
NonPersistentDispatcherMultipleConsumers.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
/**
* 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.nonpersistent;
import java.util.List;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.atomic.AtomicIntegerFieldUpdater;
import org.apache.bookkeeper.mledger.Entry;
import org.apache.pulsar.broker.service.AbstractDispatcherMultipleConsumers;
import org.apache.pulsar.broker.service.BrokerServiceException;
import org.apache.pulsar.broker.service.BrokerServiceException.ConsumerBusyException;
import org.apache.pulsar.broker.service.Consumer;
import org.apache.pulsar.broker.service.EntryBatchSizes;
import org.apache.pulsar.broker.service.RedeliveryTracker;
import org.apache.pulsar.broker.service.RedeliveryTrackerDisabled;
import org.apache.pulsar.broker.service.SendMessageInfo;
import org.apache.pulsar.broker.service.Subscription;
import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType;
import org.apache.pulsar.common.protocol.Commands;
import org.apache.pulsar.common.stats.Rate;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
/**
*/
public class NonPersistentDispatcherMultipleConsumers extends AbstractDispatcherMultipleConsumers
implements NonPersistentDispatcher {
private final NonPersistentTopic topic;
protected final Subscription subscription;
private CompletableFuture<Void> closeFuture = null;
private final String name;
protected final Rate msgDrop;
protected static final AtomicIntegerFieldUpdater<NonPersistentDispatcherMultipleConsumers>
TOTAL_AVAILABLE_PERMITS_UPDATER = AtomicIntegerFieldUpdater
.newUpdater(NonPersistentDispatcherMultipleConsumers.class, "totalAvailablePermits");
@SuppressWarnings("unused")
private volatile int totalAvailablePermits = 0;
private final RedeliveryTracker redeliveryTracker;
public NonPersistentDispatcherMultipleConsumers(NonPersistentTopic topic, Subscription subscription) {
super(subscription, topic.getBrokerService().pulsar().getConfiguration());
this.topic = topic;
this.subscription = subscription;
this.name = topic.getName() + " / " + subscription.getName();
this.msgDrop = new Rate();
this.redeliveryTracker = RedeliveryTrackerDisabled.REDELIVERY_TRACKER_DISABLED;
}
@Override
public synchronized void addConsumer(Consumer consumer) throws BrokerServiceException {
if (IS_CLOSED_UPDATER.get(this) == TRUE) {
log.warn("[{}] Dispatcher is already closed. Closing consumer {}", name, consumer);
consumer.disconnect();
return;
}
if (isConsumersExceededOnSubscription()) {
log.warn("[{}] Attempting to add consumer to subscription which reached max consumers limit", name);
throw new ConsumerBusyException("Subscription reached max consumers limit");
}
consumerList.add(consumer);
consumerSet.add(consumer);
}
@Override
protected boolean isConsumersExceededOnSubscription() {
return isConsumersExceededOnSubscription(topic.getBrokerService(), topic.getName(), consumerList.size());
}
@Override
public synchronized void removeConsumer(Consumer consumer) throws BrokerServiceException {
if (consumerSet.removeAll(consumer) == 1) {
consumerList.remove(consumer);
log.info("Removed consumer {}", consumer);
if (consumerList.isEmpty()) {
if (closeFuture != null) {
log.info("[{}] All consumers removed. Subscription is disconnected", name);
closeFuture.complete(null);
}
TOTAL_AVAILABLE_PERMITS_UPDATER.set(this, 0);
}
} else {
if (log.isDebugEnabled()) {
log.debug("[{}] Trying to remove a non-connected consumer: {}", name, consumer);
}
TOTAL_AVAILABLE_PERMITS_UPDATER.addAndGet(this, -consumer.getAvailablePermits());
}
}
@Override
public boolean isConsumerConnected() {
return !consumerList.isEmpty();
}
@Override
public CopyOnWriteArrayList<Consumer> getConsumers() {
return consumerList;
}
@Override
public synchronized boolean canUnsubscribe(Consumer consumer) {
return consumerList.size() == 1 && consumerSet.contains(consumer);
}
@Override
public CompletableFuture<Void> close() {
IS_CLOSED_UPDATER.set(this, TRUE);
return disconnectAllConsumers();
}
@Override
public synchronized void consumerFlow(Consumer consumer, int additionalNumberOfMessages) {
if (!consumerSet.contains(consumer)) {
if (log.isDebugEnabled()) {
log.debug("[{}] Ignoring flow control from disconnected consumer {}", name, consumer);
}
return;
}
TOTAL_AVAILABLE_PERMITS_UPDATER.addAndGet(this, additionalNumberOfMessages);
if (log.isDebugEnabled()) {
log.debug("[{}] Trigger new read after receiving flow control message", consumer);
}
}
@Override
public synchronized CompletableFuture<Void> disconnectAllConsumers(boolean isResetCursor) {
closeFuture = new CompletableFuture<>();
if (consumerList.isEmpty()) {
closeFuture.complete(null);
} else {
consumerList.forEach(Consumer::disconnect);
}
return closeFuture;
}
@Override
public CompletableFuture<Void> disconnectActiveConsumers(boolean isResetCursor) {
return disconnectAllConsumers(isResetCursor);
}
@Override
public synchronized void resetCloseFuture() {
closeFuture = null;
}
@Override
public void reset() {
resetCloseFuture();
IS_CLOSED_UPDATER.set(this, FALSE);
}
@Override
public SubType getType() {
return SubType.Shared;
}
@Override
public RedeliveryTracker getRedeliveryTracker() {
return redeliveryTracker;
}
@Override
public void sendMessages(List<Entry> entries) {
Consumer consumer = TOTAL_AVAILABLE_PERMITS_UPDATER.get(this) > 0 ? getNextConsumer() : null;
if (consumer != null) {
SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal();
EntryBatchSizes batchSizes = EntryBatchSizes.get(entries.size());
filterEntriesForConsumer(entries, batchSizes, sendMessageInfo, null, null, false);
consumer.sendMessages(entries, batchSizes, null, sendMessageInfo.getTotalMessages(),
sendMessageInfo.getTotalBytes(), sendMessageInfo.getTotalChunkedMessages(), getRedeliveryTracker());
TOTAL_AVAILABLE_PERMITS_UPDATER.addAndGet(this, -sendMessageInfo.getTotalMessages());
} else {
entries.forEach(entry -> {
int totalMsgs = Commands.getNumberOfMessagesInBatch(entry.getDataBuffer(), subscription.toString(), -1);
if (totalMsgs > 0) {
msgDrop.recordEvent(totalMsgs);
}
entry.release();
});
}
}
@Override
public boolean hasPermits() {
return TOTAL_AVAILABLE_PERMITS_UPDATER.get(this) > 0;
}
@Override
public Rate getMessageDropRate() {
return msgDrop;
}
@Override
public boolean isConsumerAvailable(Consumer consumer) {
return consumer != null && consumer.getAvailablePermits() > 0 && consumer.isWritable();
}
private static final Logger log = LoggerFactory.getLogger(NonPersistentDispatcherMultipleConsumers.class);
}