blob: c87f117fc634eb1630d674a4f9039c481414c1a3 [file] [log] [blame]
Madan Jampani25461112015-02-17 14:17:29 -08001/*
2 * Copyright 2015 Open Networking Laboratory
3 *
4 * Licensed under the Apache License, Version 2.0 (the "License");
5 * you may not use this file except in compliance with the License.
6 * You may obtain a copy of the License at
7 *
8 * http://www.apache.org/licenses/LICENSE-2.0
9 *
10 * Unless required by applicable law or agreed to in writing, software
11 * distributed under the License is distributed on an "AS IS" BASIS,
12 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
13 * See the License for the specific language governing permissions and
14 * limitations under the License.
15 */
16
Madan Jampani09342702015-02-05 23:32:40 -080017package org.onosproject.store.consistent.impl;
18
Madan Jampani63c659f2015-06-11 00:52:58 -070019import com.google.common.base.Charsets;
Madan Jampanie8af1cc2015-06-23 14:23:31 -070020import com.google.common.collect.ArrayListMultimap;
21import com.google.common.collect.ImmutableList;
Thomas Vachuska8dc1a692015-03-31 01:01:37 -070022import com.google.common.collect.ImmutableSet;
Madan Jampanie8af1cc2015-06-23 14:23:31 -070023import com.google.common.collect.ListMultimap;
Madan Jampanif1b8e172015-03-23 11:42:02 -070024import com.google.common.collect.Lists;
Madan Jampanib5d72d52015-04-03 16:53:50 -070025import com.google.common.collect.Maps;
Jonathan Hart054da972015-02-18 17:30:28 -080026import com.google.common.collect.Sets;
Madan Jampani0d03d602015-04-20 12:03:56 -070027import com.google.common.util.concurrent.Futures;
Madan Jampani08706ce2015-04-01 14:49:28 -070028
Madan Jampanif1b8e172015-03-23 11:42:02 -070029import net.kuujo.copycat.CopycatConfig;
Madan Jampani09342702015-02-05 23:32:40 -080030import net.kuujo.copycat.cluster.ClusterConfig;
Jonathan Hart054da972015-02-18 17:30:28 -080031import net.kuujo.copycat.cluster.Member;
Madan Jampanif1b8e172015-03-23 11:42:02 -070032import net.kuujo.copycat.cluster.Member.Type;
33import net.kuujo.copycat.cluster.internal.coordinator.ClusterCoordinator;
34import net.kuujo.copycat.cluster.internal.coordinator.DefaultClusterCoordinator;
35import net.kuujo.copycat.log.BufferedLog;
Madan Jampani09342702015-02-05 23:32:40 -080036import net.kuujo.copycat.log.FileLog;
Madan Jampanif1b8e172015-03-23 11:42:02 -070037import net.kuujo.copycat.log.Log;
Madan Jampani09342702015-02-05 23:32:40 -080038import net.kuujo.copycat.protocol.Consistency;
Madan Jampanif1b8e172015-03-23 11:42:02 -070039import net.kuujo.copycat.protocol.Protocol;
40import net.kuujo.copycat.util.concurrent.NamedThreadFactory;
Madan Jampani08706ce2015-04-01 14:49:28 -070041
Madan Jampanibff6d8f2015-03-31 16:53:47 -070042import org.apache.commons.lang.math.RandomUtils;
Madan Jampani09342702015-02-05 23:32:40 -080043import org.apache.felix.scr.annotations.Activate;
44import org.apache.felix.scr.annotations.Component;
45import org.apache.felix.scr.annotations.Deactivate;
46import org.apache.felix.scr.annotations.Reference;
47import org.apache.felix.scr.annotations.ReferenceCardinality;
Madan Jampanie8af1cc2015-06-23 14:23:31 -070048import org.apache.felix.scr.annotations.ReferencePolicy;
Madan Jampani09342702015-02-05 23:32:40 -080049import org.apache.felix.scr.annotations.Service;
Madan Jampani50589ac2015-06-08 11:38:46 -070050
51import static org.onlab.util.Tools.groupedThreads;
52
Madan Jampanie8af1cc2015-06-23 14:23:31 -070053import org.onosproject.app.ApplicationEvent;
54import org.onosproject.app.ApplicationListener;
55import org.onosproject.app.ApplicationService;
Madan Jampani09342702015-02-05 23:32:40 -080056import org.onosproject.cluster.ClusterService;
Madan Jampani63c659f2015-06-11 00:52:58 -070057import org.onosproject.cluster.NodeId;
Madan Jampanie8af1cc2015-06-23 14:23:31 -070058import org.onosproject.core.ApplicationId;
Madan Jampanibff6d8f2015-03-31 16:53:47 -070059import org.onosproject.core.IdGenerator;
Madan Jampaniafeebbd2015-05-19 15:26:01 -070060import org.onosproject.store.cluster.impl.ClusterDefinitionManager;
Madan Jampani0cb00672015-02-27 00:27:22 -080061import org.onosproject.store.cluster.impl.NodeInfo;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070062import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
Madan Jampani50589ac2015-06-08 11:38:46 -070063import org.onosproject.store.cluster.messaging.MessageSubject;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070064import org.onosproject.store.ecmap.EventuallyConsistentMapBuilderImpl;
Madan Jampanib5d72d52015-04-03 16:53:50 -070065import org.onosproject.store.service.AtomicCounterBuilder;
Madan Jampanif1b8e172015-03-23 11:42:02 -070066import org.onosproject.store.service.ConsistentMapBuilder;
Madan Jampania89f8f92015-04-01 14:39:54 -070067import org.onosproject.store.service.ConsistentMapException;
Madan Jampani63c659f2015-06-11 00:52:58 -070068import org.onosproject.store.service.DistributedQueueBuilder;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070069import org.onosproject.store.service.EventuallyConsistentMapBuilder;
Madan Jampani50589ac2015-06-08 11:38:46 -070070import org.onosproject.store.service.MapEvent;
Madan Jampania89f8f92015-04-01 14:39:54 -070071import org.onosproject.store.service.MapInfo;
Jonathan Hart054da972015-02-18 17:30:28 -080072import org.onosproject.store.service.PartitionInfo;
Madan Jampani50589ac2015-06-08 11:38:46 -070073import org.onosproject.store.service.DistributedSetBuilder;
Jonathan Hart054da972015-02-18 17:30:28 -080074import org.onosproject.store.service.StorageAdminService;
Madan Jampani393e0f02015-02-12 07:35:39 +053075import org.onosproject.store.service.StorageService;
Madan Jampanibff6d8f2015-03-31 16:53:47 -070076import org.onosproject.store.service.Transaction;
Ayaka Koshibe3a321562015-04-29 13:24:07 -070077import org.onosproject.store.service.TransactionContextBuilder;
Madan Jampani09342702015-02-05 23:32:40 -080078import org.slf4j.Logger;
79
Jonathan Hart054da972015-02-18 17:30:28 -080080import java.io.File;
81import java.io.IOException;
Madan Jampanibff6d8f2015-03-31 16:53:47 -070082import java.util.Collection;
Jonathan Hart054da972015-02-18 17:30:28 -080083import java.util.List;
84import java.util.Map;
85import java.util.Set;
Madan Jampanif1b8e172015-03-23 11:42:02 -070086import java.util.concurrent.CompletableFuture;
Madan Jampania89f8f92015-04-01 14:39:54 -070087import java.util.concurrent.ExecutionException;
Madan Jampani50589ac2015-06-08 11:38:46 -070088import java.util.concurrent.ExecutorService;
Madan Jampanif1b8e172015-03-23 11:42:02 -070089import java.util.concurrent.Executors;
Madan Jampanid14166a2015-02-24 17:37:51 -080090import java.util.concurrent.TimeUnit;
Madan Jampania89f8f92015-04-01 14:39:54 -070091import java.util.concurrent.TimeoutException;
Jonathan Hart054da972015-02-18 17:30:28 -080092import java.util.stream.Collectors;
93
94import static org.slf4j.LoggerFactory.getLogger;
Madan Jampanie8af1cc2015-06-23 14:23:31 -070095import static org.onosproject.app.ApplicationEvent.Type.APP_UNINSTALLED;
96import static org.onosproject.app.ApplicationEvent.Type.APP_DEACTIVATED;
Madan Jampani09342702015-02-05 23:32:40 -080097
98/**
99 * Database manager.
100 */
101@Component(immediate = true, enabled = true)
102@Service
Jonathan Hart054da972015-02-18 17:30:28 -0800103public class DatabaseManager implements StorageService, StorageAdminService {
Madan Jampani09342702015-02-05 23:32:40 -0800104
105 private final Logger log = getLogger(getClass());
Thomas Vachuskade563cf2015-04-01 00:28:50 -0700106
Thomas Vachuska06f44e12015-06-01 16:38:05 -0700107 public static final int COPYCAT_TCP_PORT = 9876;
Thomas Vachuskade563cf2015-04-01 00:28:50 -0700108 public static final String PARTITION_DEFINITION_FILE = "../config/tablets.json";
109 public static final String BASE_PARTITION_NAME = "p0";
110
Madan Jampani1a4eca02015-04-02 15:29:26 -0700111 private static final int RAFT_ELECTION_TIMEOUT_MILLIS = 3000;
Madan Jampania89f8f92015-04-01 14:39:54 -0700112 private static final int DATABASE_OPERATION_TIMEOUT_MILLIS = 5000;
Thomas Vachuskade563cf2015-04-01 00:28:50 -0700113
Madan Jampani63c659f2015-06-11 00:52:58 -0700114 protected static final MessageSubject QUEUE_UPDATED_TOPIC = new MessageSubject("distributed-queue-updated");
115
Madan Jampanif1b8e172015-03-23 11:42:02 -0700116 private ClusterCoordinator coordinator;
Madan Jampani50589ac2015-06-08 11:38:46 -0700117 protected PartitionedDatabase partitionedDatabase;
118 protected Database inMemoryDatabase;
Madan Jampani63c659f2015-06-11 00:52:58 -0700119 protected NodeId localNodeId;
Madan Jampanife3a9a72015-03-13 16:32:26 -0700120
Madan Jampanibff6d8f2015-03-31 16:53:47 -0700121 private TransactionManager transactionManager;
122 private final IdGenerator transactionIdGenerator = () -> RandomUtils.nextLong();
123
Madan Jampani50589ac2015-06-08 11:38:46 -0700124 private ExecutorService eventDispatcher;
Madan Jampani63c659f2015-06-11 00:52:58 -0700125 private ExecutorService queuePollExecutor;
Madan Jampanie8af1cc2015-06-23 14:23:31 -0700126 private ApplicationListener appListener = new InternalApplicationListener();
Madan Jampani50589ac2015-06-08 11:38:46 -0700127
Madan Jampani63c659f2015-06-11 00:52:58 -0700128 private final Map<String, DefaultAsyncConsistentMap> maps = Maps.newConcurrentMap();
Madan Jampanie8af1cc2015-06-23 14:23:31 -0700129 private final ListMultimap<ApplicationId, DefaultAsyncConsistentMap> mapsByApplication = ArrayListMultimap.create();
Madan Jampani63c659f2015-06-11 00:52:58 -0700130 private final Map<String, DefaultDistributedQueue> queues = Maps.newConcurrentMap();
Madan Jampani50589ac2015-06-08 11:38:46 -0700131
Madan Jampani09342702015-02-05 23:32:40 -0800132 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
133 protected ClusterService clusterService;
134
Madan Jampanie8af1cc2015-06-23 14:23:31 -0700135 @Reference(cardinality = ReferenceCardinality.OPTIONAL_UNARY, policy = ReferencePolicy.DYNAMIC)
136 protected ApplicationService applicationService;
137
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700138 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
139 protected ClusterCommunicationService clusterCommunicator;
140
Madan Jampani0cb00672015-02-27 00:27:22 -0800141 protected String nodeToUri(NodeInfo node) {
Madan Jampani27b69c62015-05-15 15:49:02 -0700142 return String.format("onos://%s:%d", node.getIp(), node.getTcpPort());
Madan Jampani09342702015-02-05 23:32:40 -0800143 }
144
Madan Jampanie8af1cc2015-06-23 14:23:31 -0700145 protected void bindApplicationService(ApplicationService service) {
146 applicationService = service;
147 applicationService.addListener(appListener);
148 }
149
150 protected void unbindApplicationService(ApplicationService service) {
151 applicationService.removeListener(appListener);
152 this.applicationService = null;
153 }
154
Madan Jampani09342702015-02-05 23:32:40 -0800155 @Activate
156 public void activate() {
Madan Jampani63c659f2015-06-11 00:52:58 -0700157 localNodeId = clusterService.getLocalNode().id();
Madan Jampani09342702015-02-05 23:32:40 -0800158 // load database configuration
Thomas Vachuskade563cf2015-04-01 00:28:50 -0700159 File databaseDefFile = new File(PARTITION_DEFINITION_FILE);
160 log.info("Loading database definition: {}", databaseDefFile.getAbsolutePath());
Madan Jampani09342702015-02-05 23:32:40 -0800161
Madan Jampani0cb00672015-02-27 00:27:22 -0800162 Map<String, Set<NodeInfo>> partitionMap;
Madan Jampani09342702015-02-05 23:32:40 -0800163 try {
Thomas Vachuskade563cf2015-04-01 00:28:50 -0700164 DatabaseDefinitionStore databaseDefStore = new DatabaseDefinitionStore(databaseDefFile);
165 if (!databaseDefFile.exists()) {
Thomas Vachuska8dc1a692015-03-31 01:01:37 -0700166 createDefaultDatabaseDefinition(databaseDefStore);
167 }
168 partitionMap = databaseDefStore.read().getPartitions();
Madan Jampani09342702015-02-05 23:32:40 -0800169 } catch (IOException e) {
Madan Jampani09342702015-02-05 23:32:40 -0800170 throw new IllegalStateException("Failed to load database config", e);
171 }
172
173 String[] activeNodeUris = partitionMap.values()
174 .stream()
175 .reduce((s1, s2) -> Sets.union(s1, s2))
176 .get()
177 .stream()
178 .map(this::nodeToUri)
179 .toArray(String[]::new);
180
Madan Jampani0cb00672015-02-27 00:27:22 -0800181 String localNodeUri = nodeToUri(NodeInfo.of(clusterService.getLocalNode()));
Madan Jampani27b69c62015-05-15 15:49:02 -0700182 Protocol protocol = new CopycatCommunicationProtocol(clusterService, clusterCommunicator);
Madan Jampani09342702015-02-05 23:32:40 -0800183
184 ClusterConfig clusterConfig = new ClusterConfig()
Madan Jampani27b69c62015-05-15 15:49:02 -0700185 .withProtocol(protocol)
Madan Jampani1a4eca02015-04-02 15:29:26 -0700186 .withElectionTimeout(electionTimeoutMillis(activeNodeUris))
187 .withHeartbeatInterval(heartbeatTimeoutMillis(activeNodeUris))
Madan Jampani09342702015-02-05 23:32:40 -0800188 .withMembers(activeNodeUris)
189 .withLocalMember(localNodeUri);
190
Madan Jampanif1b8e172015-03-23 11:42:02 -0700191 CopycatConfig copycatConfig = new CopycatConfig()
192 .withName("onos")
193 .withClusterConfig(clusterConfig)
194 .withDefaultSerializer(new DatabaseSerializer())
195 .withDefaultExecutor(Executors.newSingleThreadExecutor(new NamedThreadFactory("copycat-coordinator-%d")));
Madan Jampani09342702015-02-05 23:32:40 -0800196
Madan Jampanif1b8e172015-03-23 11:42:02 -0700197 coordinator = new DefaultClusterCoordinator(copycatConfig.resolve());
198
199 DatabaseConfig inMemoryDatabaseConfig =
200 newDatabaseConfig(BASE_PARTITION_NAME, newInMemoryLog(), activeNodeUris);
201 inMemoryDatabase = coordinator
202 .getResource(inMemoryDatabaseConfig.getName(), inMemoryDatabaseConfig.resolve(clusterConfig)
203 .withSerializer(copycatConfig.getDefaultSerializer())
204 .withDefaultExecutor(copycatConfig.getDefaultExecutor()));
205
206 List<Database> partitions = partitionMap.entrySet()
207 .stream()
208 .map(entry -> {
209 String[] replicas = entry.getValue().stream().map(this::nodeToUri).toArray(String[]::new);
210 return newDatabaseConfig(entry.getKey(), newPersistentLog(), replicas);
211 })
212 .map(config -> {
213 Database db = coordinator.getResource(config.getName(), config.resolve(clusterConfig)
214 .withSerializer(copycatConfig.getDefaultSerializer())
215 .withDefaultExecutor(copycatConfig.getDefaultExecutor()));
216 return db;
217 })
218 .collect(Collectors.toList());
219
220 partitionedDatabase = new PartitionedDatabase("onos-store", partitions);
Madan Jampani09342702015-02-05 23:32:40 -0800221
Madan Jampani0d03d602015-04-20 12:03:56 -0700222 CompletableFuture<Void> status = coordinator.open()
Madan Jampanif1b8e172015-03-23 11:42:02 -0700223 .thenCompose(v -> CompletableFuture.allOf(inMemoryDatabase.open(), partitionedDatabase.open())
224 .whenComplete((db, error) -> {
225 if (error != null) {
Madan Jampani0d03d602015-04-20 12:03:56 -0700226 log.error("Failed to initialize database.", error);
Madan Jampanif1b8e172015-03-23 11:42:02 -0700227 } else {
Madan Jampani0d03d602015-04-20 12:03:56 -0700228 log.info("Successfully initialized database.");
Madan Jampanif1b8e172015-03-23 11:42:02 -0700229 }
230 }));
231
Madan Jampani0d03d602015-04-20 12:03:56 -0700232 Futures.getUnchecked(status);
233
Madan Jampani50589ac2015-06-08 11:38:46 -0700234 transactionManager = new TransactionManager(partitionedDatabase, consistentMapBuilder());
Ayaka Koshibe94cc01b2015-06-26 15:39:11 -0700235 partitionedDatabase.setTransactionManager(transactionManager);
Madan Jampani50589ac2015-06-08 11:38:46 -0700236
237 eventDispatcher = Executors.newSingleThreadExecutor(
238 groupedThreads("onos/store/manager", "map-event-dispatcher"));
Madan Jampani63c659f2015-06-11 00:52:58 -0700239
240 queuePollExecutor = Executors.newFixedThreadPool(4,
241 groupedThreads("onos/store/manager", "queue-poll-handler"));
242
243 clusterCommunicator.<String>addSubscriber(QUEUE_UPDATED_TOPIC,
244 data -> new String(data, Charsets.UTF_8),
245 name -> {
246 DefaultDistributedQueue q = queues.get(name);
247 if (q != null) {
248 q.tryPoll();
249 }
250 },
251 queuePollExecutor);
Madan Jampani09342702015-02-05 23:32:40 -0800252 log.info("Started");
253 }
254
Thomas Vachuska8dc1a692015-03-31 01:01:37 -0700255 private void createDefaultDatabaseDefinition(DatabaseDefinitionStore store) {
256 // Assumes IPv4 is returned.
Madan Jampaniafeebbd2015-05-19 15:26:01 -0700257 String ip = ClusterDefinitionManager.getSiteLocalAddress();
Thomas Vachuskade563cf2015-04-01 00:28:50 -0700258 NodeInfo node = NodeInfo.from(ip, ip, COPYCAT_TCP_PORT);
Thomas Vachuska8dc1a692015-03-31 01:01:37 -0700259 try {
Thomas Vachuskade563cf2015-04-01 00:28:50 -0700260 store.write(DatabaseDefinition.from(ImmutableSet.of(node)));
Thomas Vachuska8dc1a692015-03-31 01:01:37 -0700261 } catch (IOException e) {
262 log.warn("Unable to write default cluster definition", e);
263 }
264 }
265
Madan Jampani09342702015-02-05 23:32:40 -0800266 @Deactivate
267 public void deactivate() {
Madan Jampanif1b8e172015-03-23 11:42:02 -0700268 CompletableFuture.allOf(inMemoryDatabase.close(), partitionedDatabase.close())
269 .thenCompose(v -> coordinator.close())
270 .whenComplete((result, error) -> {
271 if (error != null) {
272 log.warn("Failed to cleanly close databases.", error);
273 } else {
274 log.info("Successfully closed databases.");
275 }
276 });
Madan Jampani63c659f2015-06-11 00:52:58 -0700277 clusterCommunicator.removeSubscriber(QUEUE_UPDATED_TOPIC);
278 maps.values().forEach(this::unregisterMap);
Madan Jampanie8af1cc2015-06-23 14:23:31 -0700279 if (applicationService != null) {
280 applicationService.removeListener(appListener);
281 }
Madan Jampani50589ac2015-06-08 11:38:46 -0700282 eventDispatcher.shutdown();
Madan Jampani63c659f2015-06-11 00:52:58 -0700283 queuePollExecutor.shutdown();
Madan Jampani09342702015-02-05 23:32:40 -0800284 log.info("Stopped");
285 }
286
287 @Override
Ayaka Koshibe3a321562015-04-29 13:24:07 -0700288 public TransactionContextBuilder transactionContextBuilder() {
Madan Jampani50589ac2015-06-08 11:38:46 -0700289 return new DefaultTransactionContextBuilder(this, transactionIdGenerator.getNewId());
Madan Jampani64689552015-02-17 10:00:27 -0800290 }
Jonathan Hart054da972015-02-18 17:30:28 -0800291
292 @Override
293 public List<PartitionInfo> getPartitionInfo() {
Madan Jampanif1b8e172015-03-23 11:42:02 -0700294 return Lists.asList(
295 inMemoryDatabase,
296 partitionedDatabase.getPartitions().toArray(new Database[]{}))
Jonathan Hart054da972015-02-18 17:30:28 -0800297 .stream()
Madan Jampanif1b8e172015-03-23 11:42:02 -0700298 .map(DatabaseManager::toPartitionInfo)
Jonathan Hart054da972015-02-18 17:30:28 -0800299 .collect(Collectors.toList());
300 }
301
Madan Jampanif1b8e172015-03-23 11:42:02 -0700302 private Log newPersistentLog() {
303 String logDir = System.getProperty("karaf.data", "./data");
304 return new FileLog()
305 .withDirectory(logDir)
306 .withSegmentSize(1073741824) // 1GB
307 .withFlushOnWrite(true)
308 .withSegmentInterval(Long.MAX_VALUE);
309 }
310
311 private Log newInMemoryLog() {
312 return new BufferedLog()
313 .withFlushOnWrite(false)
314 .withFlushInterval(Long.MAX_VALUE)
315 .withSegmentSize(10485760) // 10MB
316 .withSegmentInterval(Long.MAX_VALUE);
317 }
318
319 private DatabaseConfig newDatabaseConfig(String name, Log log, String[] replicas) {
320 return new DatabaseConfig()
321 .withName(name)
Madan Jampani1a4eca02015-04-02 15:29:26 -0700322 .withElectionTimeout(electionTimeoutMillis(replicas))
323 .withHeartbeatInterval(heartbeatTimeoutMillis(replicas))
Madan Jampanie9b781a2015-07-07 16:23:20 -0700324 .withConsistency(Consistency.STRONG)
Madan Jampanif1b8e172015-03-23 11:42:02 -0700325 .withLog(log)
326 .withDefaultSerializer(new DatabaseSerializer())
327 .withReplicas(replicas);
328 }
329
Madan Jampani1a4eca02015-04-02 15:29:26 -0700330 private long electionTimeoutMillis(String[] replicas) {
331 return replicas.length == 1 ? 10L : RAFT_ELECTION_TIMEOUT_MILLIS;
332 }
333
334 private long heartbeatTimeoutMillis(String[] replicas) {
335 return electionTimeoutMillis(replicas) / 2;
336 }
337
Jonathan Hart054da972015-02-18 17:30:28 -0800338 /**
339 * Maps a Raft Database object to a PartitionInfo object.
340 *
341 * @param database database containing input data
342 * @return PartitionInfo object
343 */
Madan Jampanif1b8e172015-03-23 11:42:02 -0700344 private static PartitionInfo toPartitionInfo(Database database) {
Jonathan Hart054da972015-02-18 17:30:28 -0800345 return new PartitionInfo(database.name(),
346 database.cluster().term(),
Madan Jampanif1b8e172015-03-23 11:42:02 -0700347 database.cluster().members()
348 .stream()
349 .filter(member -> Type.ACTIVE.equals(member.type()))
Jonathan Hart054da972015-02-18 17:30:28 -0800350 .map(Member::uri)
Madan Jampanif1b8e172015-03-23 11:42:02 -0700351 .sorted()
Jonathan Hart054da972015-02-18 17:30:28 -0800352 .collect(Collectors.toList()),
353 database.cluster().leader() != null ?
354 database.cluster().leader().uri() : null);
355 }
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700356
357
358 @Override
359 public <K, V> EventuallyConsistentMapBuilder<K, V> eventuallyConsistentMapBuilder() {
360 return new EventuallyConsistentMapBuilderImpl<>(clusterService,
361 clusterCommunicator);
362 }
363
Madan Jampanif1b8e172015-03-23 11:42:02 -0700364 @Override
365 public <K, V> ConsistentMapBuilder<K, V> consistentMapBuilder() {
Madan Jampani50589ac2015-06-08 11:38:46 -0700366 return new DefaultConsistentMapBuilder<>(this);
Madan Jampanif1b8e172015-03-23 11:42:02 -0700367 }
Madan Jampania89f8f92015-04-01 14:39:54 -0700368
369 @Override
Madan Jampani50589ac2015-06-08 11:38:46 -0700370 public <E> DistributedSetBuilder<E> setBuilder() {
371 return new DefaultDistributedSetBuilder<>(this);
Madan Jampani08706ce2015-04-01 14:49:28 -0700372 }
373
Madan Jampani63c659f2015-06-11 00:52:58 -0700374
375 @Override
376 public <E> DistributedQueueBuilder<E> queueBuilder() {
377 return new DefaultDistributedQueueBuilder<>(this);
378 }
379
Madan Jampani08706ce2015-04-01 14:49:28 -0700380 @Override
Madan Jampanib5d72d52015-04-03 16:53:50 -0700381 public AtomicCounterBuilder atomicCounterBuilder() {
382 return new DefaultAtomicCounterBuilder(inMemoryDatabase, partitionedDatabase);
383 }
384
385 @Override
Madan Jampania89f8f92015-04-01 14:39:54 -0700386 public List<MapInfo> getMapInfo() {
387 List<MapInfo> maps = Lists.newArrayList();
388 maps.addAll(getMapInfo(inMemoryDatabase));
389 maps.addAll(getMapInfo(partitionedDatabase));
390 return maps;
391 }
392
393 private List<MapInfo> getMapInfo(Database database) {
394 return complete(database.tableNames())
395 .stream()
396 .map(name -> new MapInfo(name, complete(database.size(name))))
397 .filter(info -> info.size() > 0)
398 .collect(Collectors.toList());
399 }
400
Madan Jampanib5d72d52015-04-03 16:53:50 -0700401
402 @Override
403 public Map<String, Long> getCounters() {
404 Map<String, Long> counters = Maps.newHashMap();
405 counters.putAll(complete(inMemoryDatabase.counters()));
406 counters.putAll(complete(partitionedDatabase.counters()));
407 return counters;
408 }
409
Madan Jampanibff6d8f2015-03-31 16:53:47 -0700410 @Override
Kaouther Abrouguid8b565a2015-05-20 16:07:20 -0700411 public Map<String, Long> getPartitionedDatabaseCounters() {
412 Map<String, Long> counters = Maps.newHashMap();
413 counters.putAll(complete(partitionedDatabase.counters()));
414 return counters;
415 }
416
417 @Override
418 public Map<String, Long> getInMemoryDatabaseCounters() {
419 Map<String, Long> counters = Maps.newHashMap();
420 counters.putAll(complete(inMemoryDatabase.counters()));
421 return counters;
422 }
423
424 @Override
Madan Jampanibff6d8f2015-03-31 16:53:47 -0700425 public Collection<Transaction> getTransactions() {
426 return complete(transactionManager.getTransactions());
427 }
428
Madan Jampania89f8f92015-04-01 14:39:54 -0700429 private static <T> T complete(CompletableFuture<T> future) {
430 try {
431 return future.get(DATABASE_OPERATION_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS);
432 } catch (InterruptedException e) {
433 Thread.currentThread().interrupt();
434 throw new ConsistentMapException.Interrupted();
435 } catch (TimeoutException e) {
436 throw new ConsistentMapException.Timeout();
437 } catch (ExecutionException e) {
438 throw new ConsistentMapException(e.getCause());
439 }
440 }
Madan Jampanibff6d8f2015-03-31 16:53:47 -0700441
442 @Override
443 public void redriveTransactions() {
444 getTransactions().stream().forEach(transactionManager::execute);
445 }
Madan Jampani50589ac2015-06-08 11:38:46 -0700446
Brian O'Connor24cf6562015-06-11 18:42:30 -0700447 protected <K, V> DefaultAsyncConsistentMap<K, V> registerMap(DefaultAsyncConsistentMap<K, V> map) {
448 DefaultAsyncConsistentMap<K, V> existing = maps.putIfAbsent(map.name(), map);
449 if (existing != null) {
450 // FIXME: We need to cleanly support different map instances with same name.
451 log.info("Map by name {} already exists", map.name());
452 return existing;
Madan Jampanie8af1cc2015-06-23 14:23:31 -0700453 } else {
454 if (map.applicationId() != null) {
455 mapsByApplication.put(map.applicationId(), map);
456 }
Madan Jampani50589ac2015-06-08 11:38:46 -0700457 }
458
459 clusterCommunicator.<MapEvent<K, V>>addSubscriber(mapUpdatesSubject(map.name()),
460 map.serializer()::decode,
461 map::notifyLocalListeners,
462 eventDispatcher);
Brian O'Connor24cf6562015-06-11 18:42:30 -0700463 return map;
Madan Jampani50589ac2015-06-08 11:38:46 -0700464 }
465
Madan Jampani63c659f2015-06-11 00:52:58 -0700466 protected <K, V> void unregisterMap(DefaultAsyncConsistentMap<K, V> map) {
467 if (maps.remove(map.name()) != null) {
468 clusterCommunicator.removeSubscriber(mapUpdatesSubject(map.name()));
469 }
Madan Jampanie8af1cc2015-06-23 14:23:31 -0700470 if (map.applicationId() != null) {
471 mapsByApplication.remove(map.applicationId(), map);
472 }
Madan Jampani63c659f2015-06-11 00:52:58 -0700473 }
474
475 protected <E> void registerQueue(DefaultDistributedQueue<E> queue) {
476 // TODO: Support multiple local instances of the same queue.
477 if (queues.putIfAbsent(queue.name(), queue) != null) {
478 throw new IllegalStateException("Queue by name " + queue.name() + " already exists");
479 }
480 }
481
Madan Jampani50589ac2015-06-08 11:38:46 -0700482 protected static MessageSubject mapUpdatesSubject(String mapName) {
483 return new MessageSubject(mapName + "-map-updates");
484 }
Madan Jampanie8af1cc2015-06-23 14:23:31 -0700485
486 private class InternalApplicationListener implements ApplicationListener {
487 @Override
488 public void event(ApplicationEvent event) {
489 if (event.type() == APP_UNINSTALLED || event.type() == APP_DEACTIVATED) {
490 ApplicationId appId = event.subject().id();
491 List<DefaultAsyncConsistentMap> mapsToRemove = ImmutableList.copyOf(mapsByApplication.get(appId));
492 mapsToRemove.forEach(DatabaseManager.this::unregisterMap);
493 if (event.type() == APP_UNINSTALLED) {
494 mapsToRemove.stream().filter(map -> map.purgeOnUninstall()).forEach(map -> map.clear());
495 }
496 }
497 }
498 }
Madan Jampanie9b781a2015-07-07 16:23:20 -0700499}