forked from apache/pulsar
-
Notifications
You must be signed in to change notification settings - Fork 0
/
AbstractMetadataStore.java
347 lines (304 loc) · 13.4 KB
/
AbstractMetadataStore.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
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
/**
* 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.metadata.impl;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.type.TypeFactory;
import com.github.benmanes.caffeine.cache.AsyncCacheLoader;
import com.github.benmanes.caffeine.cache.AsyncLoadingCache;
import com.github.benmanes.caffeine.cache.Caffeine;
import com.google.common.annotations.VisibleForTesting;
import io.netty.util.concurrent.DefaultThreadFactory;
import java.util.EnumSet;
import java.util.List;
import java.util.Optional;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CopyOnWriteArrayList;
import java.util.concurrent.Executor;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.Executors;
import java.util.concurrent.RejectedExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.function.Consumer;
import java.util.stream.Collectors;
import lombok.Getter;
import lombok.extern.slf4j.Slf4j;
import org.apache.commons.lang3.StringUtils;
import org.apache.pulsar.common.util.FutureUtil;
import org.apache.pulsar.metadata.api.GetResult;
import org.apache.pulsar.metadata.api.MetadataCache;
import org.apache.pulsar.metadata.api.MetadataSerde;
import org.apache.pulsar.metadata.api.MetadataStoreException;
import org.apache.pulsar.metadata.api.Notification;
import org.apache.pulsar.metadata.api.NotificationType;
import org.apache.pulsar.metadata.api.Stat;
import org.apache.pulsar.metadata.api.extended.CreateOption;
import org.apache.pulsar.metadata.api.extended.MetadataStoreExtended;
import org.apache.pulsar.metadata.api.extended.SessionEvent;
import org.apache.pulsar.metadata.cache.impl.MetadataCacheImpl;
@Slf4j
public abstract class AbstractMetadataStore implements MetadataStoreExtended, Consumer<Notification> {
private static final long CACHE_REFRESH_TIME_MILLIS = TimeUnit.MINUTES.toMillis(5);
private final CopyOnWriteArrayList<Consumer<Notification>> listeners = new CopyOnWriteArrayList<>();
private final CopyOnWriteArrayList<Consumer<SessionEvent>> sessionListeners = new CopyOnWriteArrayList<>();
private final ExecutorService executor;
private final AsyncLoadingCache<String, List<String>> childrenCache;
private final AsyncLoadingCache<String, Boolean> existsCache;
private final CopyOnWriteArrayList<MetadataCacheImpl<?>> metadataCaches = new CopyOnWriteArrayList<>();
// We don't strictly need to use 'volatile' here because we don't need the precise consistent semantic. Instead,
// we want to avoid the overhead of 'volatile'.
@Getter
private boolean isConnected = true;
protected abstract CompletableFuture<List<String>> getChildrenFromStore(String path);
protected abstract CompletableFuture<Boolean> existsFromStore(String path);
protected AbstractMetadataStore() {
this.executor = Executors
.newSingleThreadExecutor(new DefaultThreadFactory("metadata-store"));
registerListener(this);
this.childrenCache = Caffeine.newBuilder()
.refreshAfterWrite(CACHE_REFRESH_TIME_MILLIS, TimeUnit.MILLISECONDS)
.buildAsync(new AsyncCacheLoader<String, List<String>>() {
@Override
public CompletableFuture<List<String>> asyncLoad(String key, Executor executor) {
return getChildrenFromStore(key);
}
@Override
public CompletableFuture<List<String>> asyncReload(String key, List<String> oldValue,
Executor executor) {
if (isConnected) {
return getChildrenFromStore(key);
} else {
// Do not refresh if we're not connected
return CompletableFuture.completedFuture(oldValue);
}
}
});
this.existsCache = Caffeine.newBuilder()
.refreshAfterWrite(CACHE_REFRESH_TIME_MILLIS, TimeUnit.MILLISECONDS)
.buildAsync(new AsyncCacheLoader<String, Boolean>() {
@Override
public CompletableFuture<Boolean> asyncLoad(String key, Executor executor) {
return existsFromStore(key);
}
@Override
public CompletableFuture<Boolean> asyncReload(String key, Boolean oldValue,
Executor executor) {
if (isConnected) {
return existsFromStore(key);
} else {
// Do not refresh if we're not connected
return CompletableFuture.completedFuture(oldValue);
}
}
});
}
@Override
public <T> MetadataCache<T> getMetadataCache(Class<T> clazz) {
MetadataCacheImpl<T> metadataCache = new MetadataCacheImpl<T>(this,
TypeFactory.defaultInstance().constructSimpleType(clazz, null));
metadataCaches.add(metadataCache);
return metadataCache;
}
@Override
public <T> MetadataCache<T> getMetadataCache(TypeReference<T> typeRef) {
MetadataCacheImpl<T> metadataCache = new MetadataCacheImpl<T>(this, typeRef);
metadataCaches.add(metadataCache);
return metadataCache;
}
@Override
public <T> MetadataCache<T> getMetadataCache(MetadataSerde<T> serde) {
MetadataCacheImpl<T> metadataCache = new MetadataCacheImpl<>(this, serde);
metadataCaches.add(metadataCache);
return metadataCache;
}
@Override
public CompletableFuture<Optional<GetResult>> get(String path) {
if (!isValidPath(path)) {
return FutureUtil.failedFuture(new MetadataStoreException.InvalidPathException(path));
}
return storeGet(path);
}
protected abstract CompletableFuture<Optional<GetResult>> storeGet(String path);
@Override
public CompletableFuture<Stat> put(String path, byte[] value, Optional<Long> expectedVersion) {
return put(path, value, expectedVersion, EnumSet.noneOf(CreateOption.class));
}
@Override
public final CompletableFuture<List<String>> getChildren(String path) {
if (!isValidPath(path)) {
return FutureUtil.failedFuture(new MetadataStoreException.InvalidPathException(path));
}
return childrenCache.get(path);
}
@Override
public final CompletableFuture<Boolean> exists(String path) {
if (!isValidPath(path)) {
return FutureUtil.failedFuture(new MetadataStoreException.InvalidPathException(path));
}
return existsCache.get(path);
}
@Override
public void registerListener(Consumer<Notification> listener) {
listeners.add(listener);
}
protected CompletableFuture<Void> receivedNotification(Notification notification) {
try {
return CompletableFuture.supplyAsync(() -> {
listeners.forEach(listener -> {
try {
listener.accept(notification);
} catch (Throwable t) {
log.error("Failed to process metadata store notification", t);
}
});
return null;
}, executor);
} catch (RejectedExecutionException e) {
return FutureUtil.failedFuture(e);
}
}
@Override
public void accept(Notification n) {
String path = n.getPath();
NotificationType type = n.getType();
if (type == NotificationType.Created || type == NotificationType.Deleted) {
existsCache.synchronous().invalidate(path);
String parent = parent(path);
if (parent != null) {
childrenCache.synchronous().invalidate(parent);
}
}
if (type == NotificationType.ChildrenChanged) {
childrenCache.synchronous().invalidate(path);
}
if (type == NotificationType.Created || type == NotificationType.Deleted || type == NotificationType.Modified) {
metadataCaches.forEach(c -> c.accept(n));
}
}
protected abstract CompletableFuture<Void> storeDelete(String path, Optional<Long> expectedVersion);
@Override
public final CompletableFuture<Void> delete(String path, Optional<Long> expectedVersion) {
if (!isValidPath(path)) {
return FutureUtil.failedFuture(new MetadataStoreException.InvalidPathException(path));
}
// Ensure caches are invalidated before the operation is confirmed
return storeDelete(path, expectedVersion)
.thenRun(() -> {
existsCache.synchronous().invalidate(path);
String parent = parent(path);
if (parent != null) {
childrenCache.synchronous().invalidate(parent);
}
metadataCaches.forEach(c -> c.invalidate(path));
});
}
@Override
public CompletableFuture<Void> deleteRecursive(String path) {
return getChildren(path)
.thenCompose(children -> FutureUtil.waitForAll(
children.stream()
.map(child -> deleteRecursive(path + "/" + child))
.collect(Collectors.toList())))
.thenCompose(__ -> exists(path))
.thenCompose(exists -> {
if (exists) {
return delete(path, Optional.empty());
} else {
return CompletableFuture.completedFuture(null);
}
});
}
protected abstract CompletableFuture<Stat> storePut(String path, byte[] data, Optional<Long> optExpectedVersion,
EnumSet<CreateOption> options);
@Override
public final CompletableFuture<Stat> put(String path, byte[] data, Optional<Long> optExpectedVersion,
EnumSet<CreateOption> options) {
if (!isValidPath(path)) {
return FutureUtil.failedFuture(new MetadataStoreException.InvalidPathException(path));
}
// Ensure caches are invalidated before the operation is confirmed
return storePut(path, data, optExpectedVersion, options)
.thenApply(stat -> {
NotificationType type = stat.getVersion() == 0 ? NotificationType.Created
: NotificationType.Modified;
if (type == NotificationType.Created) {
existsCache.synchronous().invalidate(path);
String parent = parent(path);
if (parent != null) {
childrenCache.synchronous().invalidate(parent);
}
}
metadataCaches.forEach(c -> c.refresh(path));
return stat;
});
}
@Override
public void registerSessionListener(Consumer<SessionEvent> listener) {
sessionListeners.add(listener);
}
protected void receivedSessionEvent(SessionEvent event) {
isConnected = event.isConnected();
sessionListeners.forEach(l -> {
try {
l.accept(event);
} catch (Throwable t) {
log.warn("Error in processing session event", t);
}
});
}
@Override
public void close() throws Exception {
executor.shutdownNow();
executor.awaitTermination(10, TimeUnit.SECONDS);
}
@VisibleForTesting
public void invalidateAll() {
childrenCache.synchronous().invalidateAll();
existsCache.synchronous().invalidateAll();
}
/**
* Run the task in the executor thread and fail the future if the executor is shutting down
*/
protected void execute(Runnable task, CompletableFuture<?> future) {
try {
executor.execute(task);
} catch (Throwable t) {
future.completeExceptionally(t);
}
}
protected static String parent(String path) {
int idx = path.lastIndexOf('/');
if (idx <= 0) {
// No parent
return null;
}
return path.substring(0, idx);
}
/**
* valid path in metadata store should be
* 1. not blank
* 2. starts with '/'
* 3. not ends with '/', except root path "/"
*/
static boolean isValidPath(String path) {
return StringUtils.equals(path, "/")
|| StringUtils.isNotBlank(path)
&& path.startsWith("/")
&& !path.endsWith("/");
}
}