-
Notifications
You must be signed in to change notification settings - Fork 3.5k
/
TestBaseOffload.java
298 lines (244 loc) · 13.7 KB
/
TestBaseOffload.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
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
/**
* 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.tests.integration.offload;
import java.util.List;
import java.util.concurrent.TimeUnit;
import lombok.extern.slf4j.Slf4j;
import org.apache.bookkeeper.client.BKException;
import org.apache.bookkeeper.client.BookKeeper;
import org.apache.bookkeeper.client.BookKeeperAdmin;
import org.apache.bookkeeper.conf.ClientConfiguration;
import org.apache.pulsar.client.admin.PulsarAdmin;
import org.apache.pulsar.client.api.Consumer;
import org.apache.pulsar.client.api.Message;
import org.apache.pulsar.client.api.Producer;
import org.apache.pulsar.client.api.PulsarClient;
import org.apache.pulsar.common.policies.data.PersistentTopicInternalStats;
import org.apache.pulsar.tests.integration.suites.PulsarTieredStorageTestSuite;
import org.testng.Assert;
@Slf4j
public abstract class TestBaseOffload extends PulsarTieredStorageTestSuite {
private static final int ENTRY_SIZE = 1024;
private static byte[] buildEntry(String pattern) {
byte[] entry = new byte[ENTRY_SIZE];
byte[] patternBytes = pattern.getBytes();
for (int i = 0; i < entry.length; i++) {
entry[i] = patternBytes[i % patternBytes.length];
}
return entry;
}
protected void testPublishOffloadAndConsumeViaCLI(String serviceUrl, String adminUrl) throws Exception {
final String tenant = "offload-test-cli-" + randomName(4);
final String namespace = tenant + "/ns1";
final String topic = "persistent://" + namespace + "/topic1";
pulsarCluster.runAdminCommandOnAnyBroker( "tenants",
"create", "--allowed-clusters", pulsarCluster.getClusterName(),
"--admin-roles", "offload-admin", tenant);
pulsarCluster.runAdminCommandOnAnyBroker(
"namespaces",
"create", "--clusters", pulsarCluster.getClusterName(), namespace);
long firstLedger = -1;
try(PulsarClient client = PulsarClient.builder().serviceUrl(serviceUrl).build();
Producer<byte[]> producer = client.newProducer().topic(topic)
.blockIfQueueFull(true).enableBatching(false).create();) {
client.newConsumer().topic(topic).subscriptionName("my-sub").subscribe().close();
// write enough to topic to make it roll
int i = 0;
for (; i < ENTRIES_PER_LEDGER * 1.5; i++) {
producer.sendAsync(buildEntry("offload-message" + i));
}
producer.flush();
}
try (PulsarAdmin admin = PulsarAdmin.builder().serviceHttpUrl(adminUrl).build()) {
// read managed ledger info, check ledgers exist
firstLedger = admin.topics().getInternalStats(topic).ledgers.get(0).ledgerId;
// first offload with a high threshold, nothing should offload
String output = pulsarCluster.runAdminCommandOnAnyBroker("topics",
"offload", "--size-threshold", "100G", topic).getStdout();
Assert.assertTrue(output.contains("Nothing to offload"));
output = pulsarCluster.runAdminCommandOnAnyBroker( "topics",
"offload-status", topic).getStdout();
Assert.assertTrue(output.contains("Offload has not been run"));
// offload with a low threshold
output = pulsarCluster.runAdminCommandOnAnyBroker("topics",
"offload", "--size-threshold", "1M", topic).getStdout();
Assert.assertTrue(output.contains("Offload triggered"));
output = pulsarCluster.runAdminCommandOnAnyBroker("topics",
"offload-status", "-w", topic).getStdout();
Assert.assertTrue(output.contains("Offload was a success"));
// delete the first ledger, so that we cannot possibly read from it
ClientConfiguration bkConf = new ClientConfiguration();
bkConf.setZkServers(pulsarCluster.getZKConnString());
try (BookKeeper bk = new BookKeeper(bkConf)) {
bk.deleteLedger(firstLedger);
}
// Unload topic to clear all caches, open handles, etc
admin.topics().unload(topic);
}
log.info("Read back the data (which would be in that first ledger)");
try(PulsarClient client = PulsarClient.builder().serviceUrl(serviceUrl).build();
Consumer<byte[]> consumer = client.newConsumer().topic(topic).subscriptionName("my-sub").subscribe()) {
// read back from topic
for (int i = 0; i < ENTRIES_PER_LEDGER * 1.5; i++) {
Message<byte[]> m = consumer.receive(1, TimeUnit.MINUTES);
Assert.assertEquals(buildEntry("offload-message" + i), m.getData());
}
}
}
protected void testPublishOffloadAndConsumeViaThreshold(String serviceUrl, String adminUrl) throws Exception {
final String tenant = "offload-test-threshold-" + randomName(4);
final String namespace = tenant + "/ns1";
final String topic = "persistent://" + namespace + "/topic1";
pulsarCluster.runAdminCommandOnAnyBroker("tenants",
"create", "--allowed-clusters", pulsarCluster.getClusterName(),
"--admin-roles", "offload-admin", tenant);
pulsarCluster.runAdminCommandOnAnyBroker("namespaces",
"create", "--clusters", pulsarCluster.getClusterName(), namespace);
pulsarCluster.runAdminCommandOnAnyBroker("namespaces",
"set-offload-threshold", "--size", "1M", namespace);
long firstLedger = 0;
try(PulsarClient client = PulsarClient.builder().serviceUrl(serviceUrl).build();
Producer<byte[]> producer = client.newProducer().topic(topic)
.blockIfQueueFull(true).enableBatching(false).create();
) {
client.newConsumer().topic(topic).subscriptionName("my-sub").subscribe().close();
// write enough to topic to make it roll twice
for (int i = 0; i < ENTRIES_PER_LEDGER * 2.5; i++) {
producer.sendAsync(buildEntry("offload-message" + i));
}
producer.flush();
}
try (PulsarAdmin admin = PulsarAdmin.builder().serviceHttpUrl(adminUrl).build()) {
firstLedger = admin.topics().getInternalStats(topic).ledgers.get(0).ledgerId;
// wait up to 30 seconds for offload to occur
for (int i = 0; i < 300 && !admin.topics().getInternalStats(topic).ledgers.get(0).offloaded; i++) {
Thread.sleep(100);
}
Assert.assertTrue(admin.topics().getInternalStats(topic).ledgers.get(0).offloaded);
// delete the first ledger, so that we cannot possibly read from it
ClientConfiguration bkConf = new ClientConfiguration();
bkConf.setZkServers(pulsarCluster.getZKConnString());
try (BookKeeper bk = new BookKeeper(bkConf)) {
bk.deleteLedger(firstLedger);
}
// Unload topic to clear all caches, open handles, etc
admin.topics().unload(topic);
}
log.info("Read back the data (which would be in that first ledger)");
try (PulsarClient client = PulsarClient.builder().serviceUrl(serviceUrl).build();
Consumer<byte[]> consumer = client.newConsumer().topic(topic).subscriptionName("my-sub").subscribe()) {
// read back from topic
for (int i = 0; i < ENTRIES_PER_LEDGER * 2.5; i++) {
Message<byte[]> m = consumer.receive(1, TimeUnit.MINUTES);
Assert.assertEquals(buildEntry("offload-message" + i), m.getData());
}
}
// try disabling
pulsarCluster.runAdminCommandOnAnyBroker("namespaces",
"set-offload-threshold", "--size", "-1", namespace);
// hard to validate that it has been disabled as we'd be waiting for
// something _not_ to happen (i.e. waiting for ages), so just check
try (PulsarAdmin admin = PulsarAdmin.builder().serviceHttpUrl(adminUrl).build()) {
Assert.assertEquals(admin.namespaces().getOffloadThreshold(namespace).longValue(), -1L);
}
}
private boolean ledgerOffloaded(List<PersistentTopicInternalStats.LedgerInfo> ledgers, long ledgerId) {
return ledgers.stream().filter(l -> l.ledgerId == ledgerId)
.map(l -> l.offloaded).findFirst().get();
}
private long writeAndWaitForOffload(String serviceUrl, String adminUrl, String topic) throws Exception {
try(PulsarClient client = PulsarClient.builder().serviceUrl(serviceUrl).build();
Producer producer = client.newProducer().topic(topic)
.blockIfQueueFull(true).enableBatching(false).create();
PulsarAdmin admin = PulsarAdmin.builder().serviceHttpUrl(adminUrl).build()) {
List<PersistentTopicInternalStats.LedgerInfo> ledgers = admin.topics().getInternalStats(topic).ledgers;
long currentLedger = ledgers.get(ledgers.size() - 1).ledgerId;
client.newConsumer().topic(topic).subscriptionName("my-sub").subscribe().close();
// write enough to topic to make it roll twice
for (int i = 0; i < ENTRIES_PER_LEDGER * 2.5; i++) {
producer.sendAsync(buildEntry("offload-message" + i));
}
producer.send(buildEntry("final-offload-message"));
// wait up to 30 seconds for offload to occur
for (int i = 0;
i < 300 && !ledgerOffloaded(admin.topics().getInternalStats(topic).ledgers, currentLedger);
i++) {
Thread.sleep(100);
}
Assert.assertTrue(ledgerOffloaded(admin.topics().getInternalStats(topic).ledgers, currentLedger));
return currentLedger;
}
}
public boolean ledgerExistsInBookKeeper(long ledgerId) throws Exception {
ClientConfiguration bkConf = new ClientConfiguration();
bkConf.setZkServers(pulsarCluster.getZKConnString());
try (BookKeeperAdmin bk = new BookKeeperAdmin(bkConf)) {
try {
bk.openLedger(ledgerId).close();
return true;
} catch (BKException.BKNoSuchLedgerExistsException
| BKException.BKNoSuchLedgerExistsOnMetadataServerException e) {
return false;
}
}
}
protected void testPublishOffloadAndConsumeDeletionLag(String serviceUrl, String adminUrl) throws Exception {
final String tenant = "offload-test-deletion-lag-" + randomName(4);
final String namespace = tenant + "/ns1";
final String topic = "persistent://" + namespace + "/topic1";
pulsarCluster.runAdminCommandOnAnyBroker("tenants",
"create", "--allowed-clusters", pulsarCluster.getClusterName(),
"--admin-roles", "offload-admin", tenant);
pulsarCluster.runAdminCommandOnAnyBroker("namespaces",
"create", "--clusters", pulsarCluster.getClusterName(), namespace);
// set threshold to offload runs immediately after role
pulsarCluster.runAdminCommandOnAnyBroker("namespaces",
"set-offload-threshold", "--size", "0", namespace);
String output = pulsarCluster.runAdminCommandOnAnyBroker(
"namespaces", "get-offload-deletion-lag", namespace).getStdout();
Assert.assertTrue(output.contains("Unset for namespace"));
long offloadedLedger = writeAndWaitForOffload(serviceUrl, adminUrl, topic);
// give it up to 5 seconds to delete, it shouldn't
// so we wait this every time
Thread.sleep(5000);
Assert.assertTrue(ledgerExistsInBookKeeper(offloadedLedger));
pulsarCluster.runAdminCommandOnAnyBroker("namespaces", "set-offload-deletion-lag", namespace,
"--lag", "0m");
output = pulsarCluster.runAdminCommandOnAnyBroker(
"namespaces", "get-offload-deletion-lag", namespace).getStdout();
Assert.assertTrue(output.contains("0 minute(s)"));
offloadedLedger = writeAndWaitForOffload(serviceUrl, adminUrl, topic);
// wait up to 10 seconds for ledger to be deleted
for (int i = 0; i < 10 && ledgerExistsInBookKeeper(offloadedLedger); i++) {
writeAndWaitForOffload(serviceUrl, adminUrl, topic);
Thread.sleep(1000);
}
Assert.assertFalse(ledgerExistsInBookKeeper(offloadedLedger));
pulsarCluster.runAdminCommandOnAnyBroker("namespaces", "clear-offload-deletion-lag", namespace);
Thread.sleep(5); // wait 5 seconds to allow broker to see update
output = pulsarCluster.runAdminCommandOnAnyBroker(
"namespaces", "get-offload-deletion-lag", namespace).getStdout();
Assert.assertTrue(output.contains("Unset for namespace"));
offloadedLedger = writeAndWaitForOffload(serviceUrl, adminUrl, topic);
// give it up to 5 seconds to delete, it shouldn't
// so we wait this every time
Thread.sleep(5000);
Assert.assertTrue(ledgerExistsInBookKeeper(offloadedLedger));
}
}