/
PersistentStreamingDispatcherSingleActiveConsumer.java
214 lines (192 loc) · 8.97 KB
/
PersistentStreamingDispatcherSingleActiveConsumer.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
/**
* 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.persistent;
import static org.apache.bookkeeper.mledger.util.SafeRun.safeRun;
import com.google.common.collect.Lists;
import java.util.concurrent.TimeUnit;
import lombok.extern.slf4j.Slf4j;
import org.apache.bookkeeper.mledger.Entry;
import org.apache.bookkeeper.mledger.ManagedCursor;
import org.apache.bookkeeper.mledger.impl.ManagedCursorImpl;
import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
import org.apache.bookkeeper.mledger.util.SafeRun;
import org.apache.commons.lang3.tuple.Pair;
import org.apache.pulsar.broker.service.Consumer;
import org.apache.pulsar.broker.service.EntryBatchIndexesAcks;
import org.apache.pulsar.broker.service.EntryBatchSizes;
import org.apache.pulsar.broker.service.SendMessageInfo;
import org.apache.pulsar.broker.service.Subscription;
import org.apache.pulsar.broker.service.streamingdispatch.PendingReadEntryRequest;
import org.apache.pulsar.broker.service.streamingdispatch.StreamingDispatcher;
import org.apache.pulsar.broker.service.streamingdispatch.StreamingEntryReader;
import org.apache.pulsar.common.api.proto.CommandSubscribe.SubType;
/**
* A {@link PersistentDispatcherSingleActiveConsumer} implemented {@link StreamingDispatcher}.
* It'll use {@link StreamingEntryReader} to read new entries instead read as micro batch from managed ledger.
*/
@Slf4j
public class PersistentStreamingDispatcherSingleActiveConsumer extends PersistentDispatcherSingleActiveConsumer
implements StreamingDispatcher {
private final StreamingEntryReader streamingEntryReader = new StreamingEntryReader((ManagedCursorImpl) cursor,
this, topic);
public PersistentStreamingDispatcherSingleActiveConsumer(ManagedCursor cursor, SubType subscriptionType,
int partitionIndex, PersistentTopic topic,
Subscription subscription) {
super(cursor, subscriptionType, partitionIndex, topic, subscription);
}
/**
* {@inheritDoc}
*/
@Override
public void canReadMoreEntries(boolean withBackoff) {
havePendingRead = false;
topic.getBrokerService().executor().schedule(() -> {
topic.getBrokerService().getTopicOrderedExecutor().executeOrdered(topicName, SafeRun.safeRun(() -> {
synchronized (PersistentStreamingDispatcherSingleActiveConsumer.this) {
Consumer currentConsumer = ACTIVE_CONSUMER_UPDATER.get(this);
if (currentConsumer != null && !havePendingRead) {
if (log.isDebugEnabled()) {
log.debug("[{}-{}] Scheduling read ", name, currentConsumer);
}
readMoreEntries(currentConsumer);
} else {
log.info("[{}-{}] Skipping read as we still havePendingRead {}", name,
currentConsumer, havePendingRead);
}
}
}));
}, withBackoff
? readFailureBackoff.next() : 0, TimeUnit.MILLISECONDS);
}
@Override
protected void cancelPendingRead() {
if (havePendingRead && streamingEntryReader.cancelReadRequests()) {
havePendingRead = false;
}
}
/**
* {@inheritDoc}
*/
@Override
public synchronized void notifyConsumersEndOfTopic() {
if (cursor.getNumberOfEntriesInBacklog(false) == 0) {
// Topic has been terminated and there are no more entries to read
// Notify the consumer only if all the messages were already acknowledged
consumers.forEach(Consumer::reachedEndOfTopic);
}
}
@Override
public String getName() {
return name;
}
/**
* {@inheritDoc}
*/
@Override
public void readEntryComplete(Entry entry, PendingReadEntryRequest ctx) {
topic.getBrokerService().getTopicOrderedExecutor().executeOrdered(name, safeRun(() -> {
internalReadEntryComplete(entry, ctx);
}));
}
public synchronized void internalReadEntryComplete(Entry entry, PendingReadEntryRequest ctx) {
if (ctx.isLast()) {
readFailureBackoff.reduceToHalf();
havePendingRead = false;
}
isFirstRead = false;
if (readBatchSize < serviceConfig.getDispatcherMaxReadBatchSize()) {
int newReadBatchSize = Math.min(readBatchSize * 2, serviceConfig.getDispatcherMaxReadBatchSize());
if (log.isDebugEnabled()) {
log.debug("[{}-{}] Increasing read batch size from {} to {}", name,
((Consumer) ctx.ctx).consumerName(), readBatchSize, newReadBatchSize);
}
readBatchSize = newReadBatchSize;
}
Consumer currentConsumer = ACTIVE_CONSUMER_UPDATER.get(this);
if (isKeyHashRangeFiltered) {
byte[] key = peekStickyKey(entry.getDataBuffer());
Consumer consumer = stickyKeyConsumerSelector.select(key);
// Skip the entry if it's not for current active consumer.
if (consumer == null || currentConsumer != consumer) {
entry.release();
return;
}
}
Consumer consumer = (Consumer) ctx.ctx;
ctx.recycle();
if (currentConsumer == null || consumer != currentConsumer) {
// Active consumer has changed since the read request has been issued. We need to rewind the cursor and
// re-issue the read request for the new consumer
if (log.isDebugEnabled()) {
log.debug("[{}] Rewind because no available consumer found to dispatch message to.", name);
}
entry.release();
streamingEntryReader.cancelReadRequests();
havePendingRead = false;
if (currentConsumer != null) {
notifyActiveConsumerChanged(currentConsumer);
readMoreEntries(currentConsumer);
}
} else {
EntryBatchSizes batchSizes = EntryBatchSizes.get(1);
SendMessageInfo sendMessageInfo = SendMessageInfo.getThreadLocal();
EntryBatchIndexesAcks batchIndexesAcks = EntryBatchIndexesAcks.get(1);
filterEntriesForConsumer(Lists.newArrayList(entry), batchSizes, sendMessageInfo, batchIndexesAcks,
cursor, false);
// Update cursor's read position.
cursor.seek(((ManagedLedgerImpl) cursor.getManagedLedger())
.getNextValidPosition((PositionImpl) entry.getPosition()));
dispatchEntriesToConsumer(currentConsumer, Lists.newArrayList(entry), batchSizes,
batchIndexesAcks, sendMessageInfo);
}
}
@Override
protected void readMoreEntries(Consumer consumer) {
// consumer can be null when all consumers are disconnected from broker.
// so skip reading more entries if currently there is no active consumer.
if (null == consumer) {
return;
}
if (!havePendingRead && consumer.getAvailablePermits() > 0) {
Pair<Integer, Long> calculateResult = calculateToRead(consumer);
int messagesToRead = calculateResult.getLeft();
long bytesToRead = calculateResult.getRight();
if (-1 == messagesToRead || bytesToRead == -1) {
// Skip read as topic/dispatcher has exceed the dispatch rate.
return;
}
// Schedule read
if (log.isDebugEnabled()) {
log.debug("[{}-{}] Schedule read of {} messages", name, consumer, messagesToRead);
}
havePendingRead = true;
if (consumer.readCompacted()) {
topic.getCompactedTopic().asyncReadEntriesOrWait(cursor, messagesToRead, isFirstRead,
this, consumer);
} else {
streamingEntryReader.asyncReadEntries(messagesToRead, bytesToRead, consumer);
}
} else {
if (log.isDebugEnabled()) {
log.debug("[{}-{}] Consumer buffer is full, pause reading", name, consumer);
}
}
}
}