/
BacklogQuotaManager.java
283 lines (260 loc) · 13.6 KB
/
BacklogQuotaManager.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
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
/**
* 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 static org.apache.pulsar.broker.cache.ConfigurationCacheService.POLICIES;
import com.google.common.collect.Lists;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import org.apache.bookkeeper.mledger.ManagedCursor;
import org.apache.bookkeeper.mledger.ManagedCursor.IndividualDeletedEntries;
import org.apache.bookkeeper.mledger.impl.ManagedLedgerImpl;
import org.apache.bookkeeper.mledger.impl.PositionImpl;
import org.apache.bookkeeper.mledger.proto.MLDataFormats;
import org.apache.pulsar.broker.PulsarService;
import org.apache.pulsar.broker.admin.AdminResource;
import org.apache.pulsar.broker.service.persistent.PersistentTopic;
import org.apache.pulsar.common.naming.TopicName;
import org.apache.pulsar.common.policies.data.BacklogQuota;
import org.apache.pulsar.common.policies.data.BacklogQuota.BacklogQuotaType;
import org.apache.pulsar.common.policies.data.Policies;
import org.apache.pulsar.common.policies.data.TopicPolicies;
import org.apache.pulsar.common.policies.data.impl.BacklogQuotaImpl;
import org.apache.pulsar.common.util.FutureUtil;
import org.apache.pulsar.zookeeper.ZooKeeperDataCache;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
public class BacklogQuotaManager {
private static final Logger log = LoggerFactory.getLogger(BacklogQuotaManager.class);
private final BacklogQuotaImpl defaultQuota;
private final ZooKeeperDataCache<Policies> zkCache;
private final PulsarService pulsar;
private final boolean isTopicLevelPoliciesEnable;
public BacklogQuotaManager(PulsarService pulsar) {
this.isTopicLevelPoliciesEnable = pulsar.getConfiguration().isTopicLevelPoliciesEnabled();
this.defaultQuota = BacklogQuotaImpl.builder()
.limitSize(pulsar.getConfiguration().getBacklogQuotaDefaultLimitGB() * 1024 * 1024 * 1024)
.limitTime(pulsar.getConfiguration().getBacklogQuotaDefaultLimitSecond())
.retentionPolicy(pulsar.getConfiguration().getBacklogQuotaDefaultRetentionPolicy())
.build();
this.zkCache = pulsar.getConfigurationCache().policiesCache();
this.pulsar = pulsar;
}
public BacklogQuotaImpl getDefaultQuota() {
return this.defaultQuota;
}
public BacklogQuotaImpl getBacklogQuota(String namespace, String policyPath) {
try {
return zkCache.get(policyPath)
.map(p -> (BacklogQuotaImpl) p.backlog_quota_map
.getOrDefault(BacklogQuotaType.destination_storage, defaultQuota))
.orElse(defaultQuota);
} catch (Exception e) {
log.warn("Failed to read policies data, will apply the default backlog quota: namespace={}", namespace, e);
return this.defaultQuota;
}
}
public BacklogQuotaImpl getBacklogQuota(TopicName topicName) {
String policyPath = AdminResource.path(POLICIES, topicName.getNamespace());
if (!isTopicLevelPoliciesEnable) {
return getBacklogQuota(topicName.getNamespace(), policyPath);
}
try {
return Optional.ofNullable(pulsar.getTopicPoliciesService().getTopicPolicies(topicName))
.map(TopicPolicies::getBackLogQuotaMap)
.map(map -> map.get(BacklogQuotaType.destination_storage.name()))
.orElseGet(() -> getBacklogQuota(topicName.getNamespace(), policyPath));
} catch (Exception e) {
log.warn("Failed to read topic policies data, will apply the namespace backlog quota: topicName={}",
topicName, e);
}
return getBacklogQuota(topicName.getNamespace(), policyPath);
}
public long getBacklogQuotaLimitInSize(TopicName topicName) {
return getBacklogQuota(topicName).getLimitSize();
}
public int getBacklogQuotaLimitInTime(TopicName topicName) {
return getBacklogQuota(topicName).getLimitTime();
}
/**
* Handle exceeded size backlog by using policies set in the zookeeper for given topic.
*
* @param persistentTopic Topic on which backlog has been exceeded
*/
public void handleExceededBacklogQuota(PersistentTopic persistentTopic, BacklogQuotaType backlogQuotaType,
boolean preciseTimeBasedBacklogQuotaCheck) {
TopicName topicName = TopicName.get(persistentTopic.getName());
BacklogQuota quota = getBacklogQuota(topicName);
log.info("Backlog quota type {} exceeded for topic [{}]. Applying [{}] policy", backlogQuotaType,
persistentTopic.getName(), quota.getPolicy());
switch (quota.getPolicy()) {
case consumer_backlog_eviction:
switch (backlogQuotaType) {
case destination_storage:
dropBacklogForSizeLimit(persistentTopic, quota);
break;
case message_age:
dropBacklogForTimeLimit(persistentTopic, quota, preciseTimeBasedBacklogQuotaCheck);
break;
default:
break;
}
break;
case producer_exception:
case producer_request_hold:
disconnectProducers(persistentTopic);
break;
default:
break;
}
}
/**
* Drop the backlog on the topic.
*
* @param persistentTopic
* The topic from which backlog should be dropped
* @param quota
* Backlog quota set for the topic
*/
private void dropBacklogForSizeLimit(PersistentTopic persistentTopic, BacklogQuota quota) {
// Set the reduction factor to 90%. The aim is to drop down the backlog to 90% of the quota limit.
double reductionFactor = 0.9;
double targetSize = reductionFactor * quota.getLimitSize();
// Get estimated unconsumed size for the managed ledger associated with this topic. Estimated size is more
// useful than the actual storage size. Actual storage size gets updated only when managed ledger is trimmed.
ManagedLedgerImpl mLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger();
long backlogSize = mLedger.getEstimatedBacklogSize();
if (log.isDebugEnabled()) {
log.debug("[{}] target size is [{}] for quota limit [{}], backlog size is [{}]", persistentTopic.getName(),
targetSize, targetSize / reductionFactor, backlogSize);
}
ManagedCursor previousSlowestConsumer = null;
while (backlogSize > targetSize) {
// Get the slowest consumer for this managed ledger and save the ledger id of the marked delete position of
// slowest consumer. Calculate the factor which is used in calculating number of messages to be skipped.
ManagedCursor slowestConsumer = mLedger.getSlowestConsumer();
if (slowestConsumer == null) {
if (log.isDebugEnabled()) {
log.debug("[{}] slowest consumer null.", persistentTopic.getName());
}
break;
}
double messageSkipFactor = ((backlogSize - targetSize) / backlogSize);
if (slowestConsumer == previousSlowestConsumer) {
log.info("[{}] Cursors not progressing, target size is [{}] for quota limit [{}], backlog size is [{}]",
persistentTopic.getName(), targetSize, targetSize / reductionFactor, backlogSize);
break;
}
// Calculate number of messages to be skipped using the current backlog and the skip factor.
long entriesInBacklog = slowestConsumer.getNumberOfEntriesInBacklog(false);
int messagesToSkip = (int) (messageSkipFactor * entriesInBacklog);
try {
// If there are no messages to skip, break out of the loop
if (messagesToSkip == 0) {
if (log.isDebugEnabled()) {
log.debug("no messages to skip for [{}]", slowestConsumer);
}
break;
}
// Skip messages on the slowest consumer
if (log.isDebugEnabled()) {
log.debug("[{}] Skipping [{}] messages on slowest consumer [{}] having backlog entries : [{}]",
persistentTopic.getName(), messagesToSkip, slowestConsumer.getName(), entriesInBacklog);
}
slowestConsumer.skipEntries(messagesToSkip, IndividualDeletedEntries.Include);
} catch (Exception e) {
log.error("[{}] Error skipping [{}] messages from slowest consumer [{}]", persistentTopic.getName(),
messagesToSkip, slowestConsumer.getName(), e);
}
// Make sure that unconsumed size is updated every time when we skip the messages.
backlogSize = mLedger.getEstimatedBacklogSize();
previousSlowestConsumer = slowestConsumer;
if (log.isDebugEnabled()) {
log.debug("[{}] Updated unconsumed size = [{}]. skipFactor: [{}]", persistentTopic.getName(),
backlogSize, messageSkipFactor);
}
}
}
/**
* Drop the backlog on the topic.
*
* @param persistentTopic
* The topic from which backlog should be dropped
* @param quota
* Backlog quota set for the topic
*/
private void dropBacklogForTimeLimit(PersistentTopic persistentTopic, BacklogQuota quota,
boolean preciseTimeBasedBacklogQuotaCheck) {
// If enabled precise time based backlog quota check, will expire message based on the timeBaseQuota
if (preciseTimeBasedBacklogQuotaCheck) {
// Set the reduction factor to 90%. The aim is to drop down the backlog to 90% of the quota limit.
double reductionFactor = 0.9;
int target = (int) (reductionFactor * quota.getLimitTime());
if (log.isDebugEnabled()) {
log.debug("[{}] target backlog expire time is [{}]", persistentTopic.getName(), target);
}
persistentTopic.getSubscriptions().forEach((__, subscription) ->
subscription.getExpiryMonitor().expireMessages(target)
);
} else {
// If disabled precise time based backlog quota check, will try to remove whole ledger from cursor's backlog
Long currentMillis = ((ManagedLedgerImpl) persistentTopic.getManagedLedger()).getClock().millis();
ManagedLedgerImpl mLedger = (ManagedLedgerImpl) persistentTopic.getManagedLedger();
try {
Long ledgerId = mLedger.getCursors().getSlowestReaderPosition().getLedgerId();
MLDataFormats.ManagedLedgerInfo.LedgerInfo ledgerInfo = mLedger.getLedgerInfo(ledgerId).get();
// Timestamp only > 0 if ledger has been closed
while (ledgerInfo.getTimestamp() > 0
&& currentMillis - ledgerInfo.getTimestamp() > quota.getLimitTime()) {
ManagedCursor slowestConsumer = mLedger.getSlowestConsumer();
// skip whole ledger for the slowest cursor
slowestConsumer.resetCursor(mLedger.getNextValidPosition(
PositionImpl.get(ledgerInfo.getLedgerId(), ledgerInfo.getEntries() - 1)));
ledgerId = mLedger.getCursors().getSlowestReaderPosition().getLedgerId();
ledgerInfo = mLedger.getLedgerInfo(ledgerId).get();
}
} catch (Exception e) {
log.error("[{}] Error resetting cursor for slowest consumer [{}]", persistentTopic.getName(),
mLedger.getSlowestConsumer().getName(), e);
}
}
}
/**
* Disconnect producers on given topic.
*
* @param persistentTopic
* The topic on which all producers should be disconnected
*/
private void disconnectProducers(PersistentTopic persistentTopic) {
List<CompletableFuture<Void>> futures = Lists.newArrayList();
Map<String, Producer> producers = persistentTopic.getProducers();
producers.values().forEach(producer -> {
log.info("Producer [{}] has exceeded backlog quota on topic [{}]. Disconnecting producer",
producer.getProducerName(), persistentTopic.getName());
futures.add(producer.disconnect());
});
FutureUtil.waitForAll(futures).thenRun(() -> {
log.info("All producers on topic [{}] are disconnected", persistentTopic.getName());
}).exceptionally(exception -> {
log.error("Error in disconnecting producers on topic [{}] [{}]", persistentTopic.getName(), exception);
return null;
});
}
}