forked from apache/druid
-
Notifications
You must be signed in to change notification settings - Fork 32
/
CommonCacheNotifier.java
240 lines (216 loc) · 8.82 KB
/
CommonCacheNotifier.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
/*
* 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.druid.security.basic;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import org.apache.druid.discovery.DiscoveryDruidNode;
import org.apache.druid.discovery.DruidNodeDiscovery;
import org.apache.druid.discovery.DruidNodeDiscoveryProvider;
import org.apache.druid.discovery.NodeRole;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.java.util.http.client.HttpClient;
import org.apache.druid.java.util.http.client.Request;
import org.apache.druid.java.util.http.client.response.ClientResponse;
import org.apache.druid.java.util.http.client.response.HttpResponseHandler;
import org.apache.druid.java.util.http.client.response.StatusResponseHolder;
import org.apache.druid.server.DruidNode;
import org.jboss.netty.handler.codec.http.HttpChunk;
import org.jboss.netty.handler.codec.http.HttpMethod;
import org.jboss.netty.handler.codec.http.HttpResponse;
import org.joda.time.Duration;
import javax.ws.rs.core.MediaType;
import java.net.MalformedURLException;
import java.net.URL;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.ExecutorService;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.TimeUnit;
public class CommonCacheNotifier
{
private static final EmittingLogger LOG = new EmittingLogger(CommonCacheNotifier.class);
/**
* {@link NodeRole#COORDINATOR} is intentionally omitted because it gets its information about the auth state directly
* from metadata storage.
*/
private static final List<NodeRole> NODE_TYPES = Arrays.asList(
NodeRole.BROKER,
NodeRole.OVERLORD,
NodeRole.HISTORICAL,
NodeRole.PEON,
NodeRole.ROUTER,
NodeRole.MIDDLE_MANAGER
);
private final DruidNodeDiscoveryProvider discoveryProvider;
private final HttpClient httpClient;
private final BlockingQueue<Pair<String, byte[]>> updateQueue;
private final Map<String, BasicAuthDBConfig> itemConfigMap;
private final String baseUrl;
private final String callerName;
private final ExecutorService exec;
public CommonCacheNotifier(
Map<String, BasicAuthDBConfig> itemConfigMap,
DruidNodeDiscoveryProvider discoveryProvider,
HttpClient httpClient,
String baseUrl,
String callerName
)
{
this.exec = Execs.scheduledSingleThreaded(StringUtils.format("%s-notifierThread-", callerName) + "%d");
this.callerName = callerName;
this.updateQueue = new LinkedBlockingQueue<>();
this.itemConfigMap = itemConfigMap;
this.discoveryProvider = discoveryProvider;
this.httpClient = httpClient;
this.baseUrl = baseUrl;
}
public void start()
{
exec.submit(
() -> {
while (!Thread.interrupted()) {
try {
LOG.debug(callerName + ":Waiting for cache update notification");
Pair<String, byte[]> update = updateQueue.take();
String authorizer = update.lhs;
byte[] serializedMap = update.rhs;
BasicAuthDBConfig authorizerConfig = itemConfigMap.get(update.lhs);
if (!authorizerConfig.isEnableCacheNotifications()) {
continue;
}
LOG.debug(callerName + ":Sending cache update notifications");
// Best effort, if a notification fails, the remote node will eventually poll to update its state
// We wait for responses however, to avoid flooding remote nodes with notifications.
List<ListenableFuture<StatusResponseHolder>> futures = sendUpdate(
authorizer,
serializedMap
);
try {
List<StatusResponseHolder> responses = Futures.allAsList(futures)
.get(
authorizerConfig.getCacheNotificationTimeout(),
TimeUnit.MILLISECONDS
);
for (StatusResponseHolder response : responses) {
LOG.debug(callerName + ":Got status: " + response.getStatus());
}
}
catch (Exception e) {
LOG.makeAlert(e, callerName + ":Failed to get response for cache notification.").emit();
}
LOG.debug(callerName + ":Received responses for cache update notifications.");
}
catch (Throwable t) {
LOG.makeAlert(t, callerName + ":Error occured while handling updates for cachedUserMaps.").emit();
}
}
}
);
}
public void stop()
{
exec.shutdownNow();
}
public void addUpdate(String updatedItemName, byte[] updatedItemData)
{
updateQueue.add(
new Pair<>(updatedItemName, updatedItemData)
);
}
private List<ListenableFuture<StatusResponseHolder>> sendUpdate(String updatedAuthorizerPrefix, byte[] serializedUserMap)
{
List<ListenableFuture<StatusResponseHolder>> futures = new ArrayList<>();
for (NodeRole nodeRole : NODE_TYPES) {
DruidNodeDiscovery nodeDiscovery = discoveryProvider.getForNodeRole(nodeRole);
Collection<DiscoveryDruidNode> nodes = nodeDiscovery.getAllNodes();
for (DiscoveryDruidNode node : nodes) {
URL listenerURL = getListenerURL(node.getDruidNode(), baseUrl, updatedAuthorizerPrefix);
// best effort, if this fails, remote node will poll and pick up the update eventually
Request req = new Request(HttpMethod.POST, listenerURL);
req.setContent(MediaType.APPLICATION_JSON, serializedUserMap);
BasicAuthDBConfig itemConfig = itemConfigMap.get(updatedAuthorizerPrefix);
ListenableFuture<StatusResponseHolder> future = httpClient.go(
req,
new ResponseHandler(),
Duration.millis(itemConfig.getCacheNotificationTimeout())
);
futures.add(future);
}
}
return futures;
}
private URL getListenerURL(DruidNode druidNode, String baseUrl, String itemName)
{
try {
return new URL(
druidNode.getServiceScheme(),
druidNode.getHost(),
druidNode.getPortToUse(),
StringUtils.format(baseUrl, StringUtils.urlEncode(itemName))
);
}
catch (MalformedURLException mue) {
LOG.error(callerName + ":WTF? Malformed url for DruidNode[%s] and itemName[%s]", druidNode, itemName);
throw new RuntimeException(mue);
}
}
// Based off StatusResponseHandler, but with response content ignored
private static class ResponseHandler implements HttpResponseHandler<StatusResponseHolder, StatusResponseHolder>
{
protected static final Logger log = new Logger(ResponseHandler.class);
@Override
public ClientResponse<StatusResponseHolder> handleResponse(HttpResponse response, TrafficCop trafficCop)
{
return ClientResponse.unfinished(
new StatusResponseHolder(
response.getStatus(),
null
)
);
}
@Override
public ClientResponse<StatusResponseHolder> handleChunk(
ClientResponse<StatusResponseHolder> response,
HttpChunk chunk,
long chunkNum
)
{
return response;
}
@Override
public ClientResponse<StatusResponseHolder> done(ClientResponse<StatusResponseHolder> response)
{
return ClientResponse.finished(response.getObj());
}
@Override
public void exceptionCaught(ClientResponse<StatusResponseHolder> clientResponse, Throwable e)
{
// Its safe to Ignore as the ClientResponse returned in handleChunk were unfinished
log.error(e, "exceptionCaught in CommonCacheNotifier ResponseHandler.");
}
}
}