forked from voldemort/voldemort
/
VoldemortServer.java
311 lines (269 loc) · 13.8 KB
/
VoldemortServer.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
/*
* Copyright 2008-2009 LinkedIn, Inc
*
* Licensed 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 voldemort.server;
import static voldemort.utils.Utils.croak;
import java.io.File;
import java.util.ArrayList;
import java.util.List;
import org.apache.log4j.Logger;
import voldemort.VoldemortException;
import voldemort.annotations.jmx.JmxOperation;
import voldemort.client.protocol.RequestFormatType;
import voldemort.client.protocol.admin.AdminClient;
import voldemort.cluster.Cluster;
import voldemort.cluster.Node;
import voldemort.server.gossip.GossipService;
import voldemort.server.http.HttpService;
import voldemort.server.jmx.JmxService;
import voldemort.server.niosocket.NioSocketService;
import voldemort.server.protocol.RequestHandlerFactory;
import voldemort.server.protocol.SocketRequestHandlerFactory;
import voldemort.server.protocol.admin.AsyncOperationRunner;
import voldemort.server.rebalance.Rebalancer;
import voldemort.server.rebalance.RebalancerService;
import voldemort.server.scheduler.SchedulerService;
import voldemort.server.socket.SocketService;
import voldemort.server.storage.StorageService;
import voldemort.store.configuration.ConfigurationStorageEngine;
import voldemort.store.metadata.MetadataStore;
import voldemort.utils.RebalanceUtils;
import voldemort.utils.SystemTime;
import voldemort.utils.Utils;
import voldemort.versioning.Versioned;
import voldemort.xml.ClusterMapper;
import com.google.common.collect.ImmutableList;
/**
* This is the main server, it bootstraps all the services.
*
* It can be embedded or run directly via it's main method.
*
* @author jay
*
*/
public class VoldemortServer extends AbstractService {
private static final Logger logger = Logger.getLogger(VoldemortServer.class.getName());
public static final long DEFAULT_PUSHER_POLL_MS = 60 * 1000;
private final static int ASYNC_REQUEST_CACHE_SIZE = 64;
private final Node identityNode;
private final List<VoldemortService> services;
private final StoreRepository storeRepository;
private final VoldemortConfig voldemortConfig;
private final MetadataStore metadata;
private AsyncOperationRunner asyncRunner;
public VoldemortServer(VoldemortConfig config) {
super(ServiceType.VOLDEMORT);
this.voldemortConfig = config;
this.storeRepository = new StoreRepository();
this.metadata = MetadataStore.readFromDirectory(new File(this.voldemortConfig.getMetadataDirectory()),
voldemortConfig.getNodeId());
this.identityNode = metadata.getCluster().getNodeById(voldemortConfig.getNodeId());
this.services = createServices();
}
public VoldemortServer(VoldemortConfig config, Cluster cluster) {
super(ServiceType.VOLDEMORT);
this.voldemortConfig = config;
this.identityNode = cluster.getNodeById(voldemortConfig.getNodeId());
this.storeRepository = new StoreRepository();
// update cluster details in metaDataStore
ConfigurationStorageEngine metadataInnerEngine = new ConfigurationStorageEngine("metadata-config-store",
voldemortConfig.getMetadataDirectory());
metadataInnerEngine.put(MetadataStore.CLUSTER_KEY,
new Versioned<String>(new ClusterMapper().writeCluster(cluster)));
this.metadata = new MetadataStore(metadataInnerEngine, voldemortConfig.getNodeId());
this.services = createServices();
}
public AsyncOperationRunner getAsyncRunner() {
return asyncRunner;
}
private List<VoldemortService> createServices() {
/* Services are given in the order they must be started */
List<VoldemortService> services = new ArrayList<VoldemortService>();
SchedulerService scheduler = new SchedulerService(voldemortConfig.getSchedulerThreads(),
SystemTime.INSTANCE);
StorageService storageService = new StorageService(storeRepository,
metadata,
scheduler,
voldemortConfig);
asyncRunner = new AsyncOperationRunner(scheduler, ASYNC_REQUEST_CACHE_SIZE);
services.add(storageService);
services.add(scheduler);
services.add(asyncRunner);
if(voldemortConfig.isHttpServerEnabled())
services.add(new HttpService(this,
storageService,
storeRepository,
RequestFormatType.VOLDEMORT_V1,
voldemortConfig.getMaxThreads(),
identityNode.getHttpPort()));
if(voldemortConfig.isSocketServerEnabled()) {
RequestHandlerFactory socketRequestHandlerFactory = new SocketRequestHandlerFactory(storageService,
this.storeRepository,
this.metadata,
this.voldemortConfig,
this.asyncRunner,
null);
if(voldemortConfig.getUseNioConnector()) {
logger.info("Using NIO Connector.");
services.add(new NioSocketService(socketRequestHandlerFactory,
identityNode.getSocketPort(),
voldemortConfig.getSocketBufferSize(),
voldemortConfig.getNioConnectorSelectors(),
"nio-socket-server",
voldemortConfig.isJmxEnabled()));
} else {
logger.info("Using BIO Connector.");
services.add(new SocketService(socketRequestHandlerFactory,
identityNode.getSocketPort(),
voldemortConfig.getCoreThreads(),
voldemortConfig.getMaxThreads(),
voldemortConfig.getSocketBufferSize(),
"socket-server",
voldemortConfig.isJmxEnabled()));
}
}
if(voldemortConfig.isAdminServerEnabled()) {
Rebalancer rebalancer = null;
if(voldemortConfig.isEnableRebalanceService()) {
RebalancerService rebalancerService = new RebalancerService(metadata,
voldemortConfig,
asyncRunner,
scheduler);
services.add(rebalancerService);
rebalancer = rebalancerService.getRebalancer();
}
SocketRequestHandlerFactory adminRequestHandlerFactory = new SocketRequestHandlerFactory(storageService,
this.storeRepository,
this.metadata,
this.voldemortConfig,
this.asyncRunner,
rebalancer);
if(voldemortConfig.getUseNioConnector()) {
logger.info("Using NIO Connector for Admin Service.");
services.add(new NioSocketService(adminRequestHandlerFactory,
identityNode.getAdminPort(),
voldemortConfig.getAdminSocketBufferSize(),
voldemortConfig.getNioAdminConnectorSelectors(),
"admin-server",
voldemortConfig.isJmxEnabled()));
} else {
logger.info("Using BIO Connector for Admin Service.");
services.add(new SocketService(adminRequestHandlerFactory,
identityNode.getAdminPort(),
voldemortConfig.getAdminCoreThreads(),
voldemortConfig.getAdminMaxThreads(),
voldemortConfig.getAdminSocketBufferSize(),
"admin-server",
voldemortConfig.isJmxEnabled()));
}
}
if(voldemortConfig.isGossipEnabled()) {
services.add(new GossipService(this.metadata, scheduler, voldemortConfig));
}
if(voldemortConfig.isJmxEnabled())
services.add(new JmxService(this, this.metadata.getCluster(), storeRepository, services));
return ImmutableList.copyOf(services);
}
@Override
protected void startInner() throws VoldemortException {
logger.info("Starting " + services.size() + " services.");
long start = System.currentTimeMillis();
for(VoldemortService service: services)
service.start();
long end = System.currentTimeMillis();
logger.info("Startup completed in " + (end - start) + " ms.");
}
/**
* Attempt to shutdown the server. As much shutdown as possible will be
* completed, even if intermediate errors are encountered.
*
* @throws VoldemortException
*/
@Override
protected void stopInner() throws VoldemortException {
List<VoldemortException> exceptions = new ArrayList<VoldemortException>();
logger.info("Stopping services:" + getIdentityNode().getId());
/* Stop in reverse order */
for(VoldemortService service: Utils.reversed(services)) {
try {
service.stop();
} catch(VoldemortException e) {
exceptions.add(e);
logger.error(e);
}
}
logger.info("All services stopped for Node:" + getIdentityNode().getId());
if(exceptions.size() > 0)
throw exceptions.get(0);
}
public static void main(String[] args) throws Exception {
VoldemortConfig config = null;
try {
if(args.length == 0)
config = VoldemortConfig.loadFromEnvironmentVariable();
else if(args.length == 1)
config = VoldemortConfig.loadFromVoldemortHome(args[0]);
else
croak("USAGE: java " + VoldemortServer.class.getName() + " [voldemort_home_dir]");
} catch(Exception e) {
logger.error(e);
Utils.croak("Error while loading configuration: " + e.getMessage());
}
final VoldemortServer server = new VoldemortServer(config);
if(!server.isStarted())
server.start();
// add a shutdown hook to stop the server
Runtime.getRuntime().addShutdownHook(new Thread() {
@Override
public void run() {
if(server.isStarted())
server.stop();
}
});
}
public Node getIdentityNode() {
return this.identityNode;
}
public List<VoldemortService> getServices() {
return services;
}
public VoldemortService getService(ServiceType type) {
for(VoldemortService service: services)
if(service.getType().equals(type))
return service;
throw new IllegalStateException(type.getDisplayName() + " has not been initialized.");
}
public VoldemortConfig getVoldemortConfig() {
return this.voldemortConfig;
}
public StoreRepository getStoreRepository() {
return this.storeRepository;
}
public MetadataStore getMetadataStore() {
return metadata;
}
@JmxOperation(description = "force restore data from replication")
public void restoreDataFromReplication(int numberOfParallelTransfers) {
AdminClient adminClient = RebalanceUtils.createTempAdminClient(voldemortConfig,
metadata.getCluster(),
numberOfParallelTransfers * 2,
numberOfParallelTransfers * 2);
try {
adminClient.restoreDataFromReplications(metadata.getNodeId(), numberOfParallelTransfers);
} finally {
adminClient.stop();
}
}
}