blob: 6ea7c22064795cb5ea25faab61071be4db07317e [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 Jampanie8af1cc2015-06-23 14:23:31 -070019import com.google.common.collect.ArrayListMultimap;
20import com.google.common.collect.ImmutableList;
Thomas Vachuska8dc1a692015-03-31 01:01:37 -070021import com.google.common.collect.ImmutableSet;
Madan Jampanif1b8e172015-03-23 11:42:02 -070022import com.google.common.collect.Lists;
Madan Jampanib5d72d52015-04-03 16:53:50 -070023import com.google.common.collect.Maps;
Madan Jampani85668632015-08-10 10:46:40 -070024import com.google.common.collect.Multimap;
Madan Jampani15687d12015-08-11 07:19:35 -070025import com.google.common.collect.Multimaps;
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 Jampanie8af1cc2015-06-23 14:23:31 -070050import org.onosproject.app.ApplicationEvent;
51import org.onosproject.app.ApplicationListener;
52import org.onosproject.app.ApplicationService;
Madan Jampani09342702015-02-05 23:32:40 -080053import org.onosproject.cluster.ClusterService;
Madan Jampani63c659f2015-06-11 00:52:58 -070054import org.onosproject.cluster.NodeId;
Madan Jampanie8af1cc2015-06-23 14:23:31 -070055import org.onosproject.core.ApplicationId;
Madan Jampanibff6d8f2015-03-31 16:53:47 -070056import org.onosproject.core.IdGenerator;
Madan Jampaniafeebbd2015-05-19 15:26:01 -070057import org.onosproject.store.cluster.impl.ClusterDefinitionManager;
Madan Jampani0cb00672015-02-27 00:27:22 -080058import org.onosproject.store.cluster.impl.NodeInfo;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070059import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
60import org.onosproject.store.ecmap.EventuallyConsistentMapBuilderImpl;
Madan Jampanib5d72d52015-04-03 16:53:50 -070061import org.onosproject.store.service.AtomicCounterBuilder;
Madan Jampani762246d2015-07-21 15:40:59 -070062import org.onosproject.store.service.AtomicValueBuilder;
Madan Jampanif1b8e172015-03-23 11:42:02 -070063import org.onosproject.store.service.ConsistentMapBuilder;
Madan Jampania89f8f92015-04-01 14:39:54 -070064import org.onosproject.store.service.ConsistentMapException;
Madan Jampani63c659f2015-06-11 00:52:58 -070065import org.onosproject.store.service.DistributedQueueBuilder;
Jonathan Hart6ec029a2015-03-24 17:12:35 -070066import org.onosproject.store.service.EventuallyConsistentMapBuilder;
Madan Jampania89f8f92015-04-01 14:39:54 -070067import org.onosproject.store.service.MapInfo;
Jonathan Hart054da972015-02-18 17:30:28 -080068import org.onosproject.store.service.PartitionInfo;
Madan Jampani50589ac2015-06-08 11:38:46 -070069import org.onosproject.store.service.DistributedSetBuilder;
Jonathan Hart054da972015-02-18 17:30:28 -080070import org.onosproject.store.service.StorageAdminService;
Madan Jampani393e0f02015-02-12 07:35:39 +053071import org.onosproject.store.service.StorageService;
Madan Jampanibff6d8f2015-03-31 16:53:47 -070072import org.onosproject.store.service.Transaction;
Ayaka Koshibe3a321562015-04-29 13:24:07 -070073import org.onosproject.store.service.TransactionContextBuilder;
Madan Jampani09342702015-02-05 23:32:40 -080074import org.slf4j.Logger;
75
Jonathan Hart054da972015-02-18 17:30:28 -080076import java.io.File;
77import java.io.IOException;
Madan Jampanibff6d8f2015-03-31 16:53:47 -070078import java.util.Collection;
Jonathan Hart054da972015-02-18 17:30:28 -080079import java.util.List;
80import java.util.Map;
81import java.util.Set;
Madan Jampanif1b8e172015-03-23 11:42:02 -070082import java.util.concurrent.CompletableFuture;
Madan Jampania89f8f92015-04-01 14:39:54 -070083import java.util.concurrent.ExecutionException;
Madan Jampanif1b8e172015-03-23 11:42:02 -070084import java.util.concurrent.Executors;
Madan Jampanid14166a2015-02-24 17:37:51 -080085import java.util.concurrent.TimeUnit;
Madan Jampania89f8f92015-04-01 14:39:54 -070086import java.util.concurrent.TimeoutException;
Jonathan Hart054da972015-02-18 17:30:28 -080087import java.util.stream.Collectors;
88
89import static org.slf4j.LoggerFactory.getLogger;
Madan Jampanie8af1cc2015-06-23 14:23:31 -070090import static org.onosproject.app.ApplicationEvent.Type.APP_UNINSTALLED;
91import static org.onosproject.app.ApplicationEvent.Type.APP_DEACTIVATED;
Madan Jampani09342702015-02-05 23:32:40 -080092
93/**
94 * Database manager.
95 */
96@Component(immediate = true, enabled = true)
97@Service
Jonathan Hart054da972015-02-18 17:30:28 -080098public class DatabaseManager implements StorageService, StorageAdminService {
Madan Jampani09342702015-02-05 23:32:40 -080099
100 private final Logger log = getLogger(getClass());
Thomas Vachuskade563cf2015-04-01 00:28:50 -0700101
Thomas Vachuska06f44e12015-06-01 16:38:05 -0700102 public static final int COPYCAT_TCP_PORT = 9876;
Thomas Vachuskade563cf2015-04-01 00:28:50 -0700103 public static final String PARTITION_DEFINITION_FILE = "../config/tablets.json";
104 public static final String BASE_PARTITION_NAME = "p0";
105
Madan Jampani1a4eca02015-04-02 15:29:26 -0700106 private static final int RAFT_ELECTION_TIMEOUT_MILLIS = 3000;
Madan Jampania89f8f92015-04-01 14:39:54 -0700107 private static final int DATABASE_OPERATION_TIMEOUT_MILLIS = 5000;
Thomas Vachuskade563cf2015-04-01 00:28:50 -0700108
Madan Jampanif1b8e172015-03-23 11:42:02 -0700109 private ClusterCoordinator coordinator;
Madan Jampani50589ac2015-06-08 11:38:46 -0700110 protected PartitionedDatabase partitionedDatabase;
111 protected Database inMemoryDatabase;
Madan Jampani63c659f2015-06-11 00:52:58 -0700112 protected NodeId localNodeId;
Madan Jampanife3a9a72015-03-13 16:32:26 -0700113
Madan Jampanibff6d8f2015-03-31 16:53:47 -0700114 private TransactionManager transactionManager;
115 private final IdGenerator transactionIdGenerator = () -> RandomUtils.nextLong();
116
Madan Jampanie8af1cc2015-06-23 14:23:31 -0700117 private ApplicationListener appListener = new InternalApplicationListener();
Madan Jampani50589ac2015-06-08 11:38:46 -0700118
Madan Jampani15687d12015-08-11 07:19:35 -0700119 private final Multimap<String, DefaultAsyncConsistentMap> maps =
120 Multimaps.synchronizedMultimap(ArrayListMultimap.create());
121 private final Multimap<ApplicationId, DefaultAsyncConsistentMap> mapsByApplication =
122 Multimaps.synchronizedMultimap(ArrayListMultimap.create());
Madan Jampani50589ac2015-06-08 11:38:46 -0700123
Madan Jampani09342702015-02-05 23:32:40 -0800124 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
125 protected ClusterService clusterService;
126
Madan Jampanie8af1cc2015-06-23 14:23:31 -0700127 @Reference(cardinality = ReferenceCardinality.OPTIONAL_UNARY, policy = ReferencePolicy.DYNAMIC)
128 protected ApplicationService applicationService;
129
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700130 @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
131 protected ClusterCommunicationService clusterCommunicator;
132
Madan Jampani0cb00672015-02-27 00:27:22 -0800133 protected String nodeToUri(NodeInfo node) {
Madan Jampani27b69c62015-05-15 15:49:02 -0700134 return String.format("onos://%s:%d", node.getIp(), node.getTcpPort());
Madan Jampani09342702015-02-05 23:32:40 -0800135 }
136
Madan Jampanie8af1cc2015-06-23 14:23:31 -0700137 protected void bindApplicationService(ApplicationService service) {
138 applicationService = service;
139 applicationService.addListener(appListener);
140 }
141
142 protected void unbindApplicationService(ApplicationService service) {
143 applicationService.removeListener(appListener);
144 this.applicationService = null;
145 }
146
Madan Jampani09342702015-02-05 23:32:40 -0800147 @Activate
148 public void activate() {
Madan Jampani63c659f2015-06-11 00:52:58 -0700149 localNodeId = clusterService.getLocalNode().id();
Madan Jampani09342702015-02-05 23:32:40 -0800150 // load database configuration
Thomas Vachuskade563cf2015-04-01 00:28:50 -0700151 File databaseDefFile = new File(PARTITION_DEFINITION_FILE);
152 log.info("Loading database definition: {}", databaseDefFile.getAbsolutePath());
Madan Jampani09342702015-02-05 23:32:40 -0800153
Madan Jampani0cb00672015-02-27 00:27:22 -0800154 Map<String, Set<NodeInfo>> partitionMap;
Madan Jampani09342702015-02-05 23:32:40 -0800155 try {
Thomas Vachuskade563cf2015-04-01 00:28:50 -0700156 DatabaseDefinitionStore databaseDefStore = new DatabaseDefinitionStore(databaseDefFile);
157 if (!databaseDefFile.exists()) {
Thomas Vachuska8dc1a692015-03-31 01:01:37 -0700158 createDefaultDatabaseDefinition(databaseDefStore);
159 }
160 partitionMap = databaseDefStore.read().getPartitions();
Madan Jampani09342702015-02-05 23:32:40 -0800161 } catch (IOException e) {
Madan Jampani09342702015-02-05 23:32:40 -0800162 throw new IllegalStateException("Failed to load database config", e);
163 }
164
165 String[] activeNodeUris = partitionMap.values()
166 .stream()
167 .reduce((s1, s2) -> Sets.union(s1, s2))
168 .get()
169 .stream()
170 .map(this::nodeToUri)
171 .toArray(String[]::new);
172
Madan Jampani0cb00672015-02-27 00:27:22 -0800173 String localNodeUri = nodeToUri(NodeInfo.of(clusterService.getLocalNode()));
Madan Jampani27b69c62015-05-15 15:49:02 -0700174 Protocol protocol = new CopycatCommunicationProtocol(clusterService, clusterCommunicator);
Madan Jampani09342702015-02-05 23:32:40 -0800175
176 ClusterConfig clusterConfig = new ClusterConfig()
Madan Jampani27b69c62015-05-15 15:49:02 -0700177 .withProtocol(protocol)
Madan Jampani1a4eca02015-04-02 15:29:26 -0700178 .withElectionTimeout(electionTimeoutMillis(activeNodeUris))
179 .withHeartbeatInterval(heartbeatTimeoutMillis(activeNodeUris))
Madan Jampani09342702015-02-05 23:32:40 -0800180 .withMembers(activeNodeUris)
181 .withLocalMember(localNodeUri);
182
Madan Jampanif1b8e172015-03-23 11:42:02 -0700183 CopycatConfig copycatConfig = new CopycatConfig()
184 .withName("onos")
185 .withClusterConfig(clusterConfig)
186 .withDefaultSerializer(new DatabaseSerializer())
187 .withDefaultExecutor(Executors.newSingleThreadExecutor(new NamedThreadFactory("copycat-coordinator-%d")));
Madan Jampani09342702015-02-05 23:32:40 -0800188
Madan Jampanif1b8e172015-03-23 11:42:02 -0700189 coordinator = new DefaultClusterCoordinator(copycatConfig.resolve());
190
191 DatabaseConfig inMemoryDatabaseConfig =
192 newDatabaseConfig(BASE_PARTITION_NAME, newInMemoryLog(), activeNodeUris);
193 inMemoryDatabase = coordinator
194 .getResource(inMemoryDatabaseConfig.getName(), inMemoryDatabaseConfig.resolve(clusterConfig)
195 .withSerializer(copycatConfig.getDefaultSerializer())
196 .withDefaultExecutor(copycatConfig.getDefaultExecutor()));
197
198 List<Database> partitions = partitionMap.entrySet()
199 .stream()
200 .map(entry -> {
201 String[] replicas = entry.getValue().stream().map(this::nodeToUri).toArray(String[]::new);
202 return newDatabaseConfig(entry.getKey(), newPersistentLog(), replicas);
203 })
204 .map(config -> {
205 Database db = coordinator.getResource(config.getName(), config.resolve(clusterConfig)
206 .withSerializer(copycatConfig.getDefaultSerializer())
207 .withDefaultExecutor(copycatConfig.getDefaultExecutor()));
208 return db;
209 })
210 .collect(Collectors.toList());
211
212 partitionedDatabase = new PartitionedDatabase("onos-store", partitions);
Madan Jampani09342702015-02-05 23:32:40 -0800213
Madan Jampani0d03d602015-04-20 12:03:56 -0700214 CompletableFuture<Void> status = coordinator.open()
Madan Jampanif1b8e172015-03-23 11:42:02 -0700215 .thenCompose(v -> CompletableFuture.allOf(inMemoryDatabase.open(), partitionedDatabase.open())
216 .whenComplete((db, error) -> {
217 if (error != null) {
Madan Jampani0d03d602015-04-20 12:03:56 -0700218 log.error("Failed to initialize database.", error);
Madan Jampanif1b8e172015-03-23 11:42:02 -0700219 } else {
Madan Jampani0d03d602015-04-20 12:03:56 -0700220 log.info("Successfully initialized database.");
Madan Jampanif1b8e172015-03-23 11:42:02 -0700221 }
222 }));
223
Madan Jampani0d03d602015-04-20 12:03:56 -0700224 Futures.getUnchecked(status);
225
Madan Jampani50589ac2015-06-08 11:38:46 -0700226 transactionManager = new TransactionManager(partitionedDatabase, consistentMapBuilder());
Ayaka Koshibe94cc01b2015-06-26 15:39:11 -0700227 partitionedDatabase.setTransactionManager(transactionManager);
Madan Jampani50589ac2015-06-08 11:38:46 -0700228
Madan Jampani09342702015-02-05 23:32:40 -0800229 log.info("Started");
230 }
231
Thomas Vachuska8dc1a692015-03-31 01:01:37 -0700232 private void createDefaultDatabaseDefinition(DatabaseDefinitionStore store) {
233 // Assumes IPv4 is returned.
Madan Jampaniafeebbd2015-05-19 15:26:01 -0700234 String ip = ClusterDefinitionManager.getSiteLocalAddress();
Thomas Vachuskade563cf2015-04-01 00:28:50 -0700235 NodeInfo node = NodeInfo.from(ip, ip, COPYCAT_TCP_PORT);
Thomas Vachuska8dc1a692015-03-31 01:01:37 -0700236 try {
Thomas Vachuskade563cf2015-04-01 00:28:50 -0700237 store.write(DatabaseDefinition.from(ImmutableSet.of(node)));
Thomas Vachuska8dc1a692015-03-31 01:01:37 -0700238 } catch (IOException e) {
239 log.warn("Unable to write default cluster definition", e);
240 }
241 }
242
Madan Jampani09342702015-02-05 23:32:40 -0800243 @Deactivate
244 public void deactivate() {
Madan Jampanif1b8e172015-03-23 11:42:02 -0700245 CompletableFuture.allOf(inMemoryDatabase.close(), partitionedDatabase.close())
246 .thenCompose(v -> coordinator.close())
247 .whenComplete((result, error) -> {
248 if (error != null) {
249 log.warn("Failed to cleanly close databases.", error);
250 } else {
251 log.info("Successfully closed databases.");
252 }
253 });
Madan Jampanie81f1cf2015-09-15 11:00:41 -0700254 ImmutableList.copyOf(maps.values()).forEach(this::unregisterMap);
Madan Jampanie8af1cc2015-06-23 14:23:31 -0700255 if (applicationService != null) {
256 applicationService.removeListener(appListener);
257 }
Madan Jampani09342702015-02-05 23:32:40 -0800258 log.info("Stopped");
259 }
260
261 @Override
Ayaka Koshibe3a321562015-04-29 13:24:07 -0700262 public TransactionContextBuilder transactionContextBuilder() {
Madan Jampani50589ac2015-06-08 11:38:46 -0700263 return new DefaultTransactionContextBuilder(this, transactionIdGenerator.getNewId());
Madan Jampani64689552015-02-17 10:00:27 -0800264 }
Jonathan Hart054da972015-02-18 17:30:28 -0800265
266 @Override
267 public List<PartitionInfo> getPartitionInfo() {
Madan Jampanif1b8e172015-03-23 11:42:02 -0700268 return Lists.asList(
269 inMemoryDatabase,
270 partitionedDatabase.getPartitions().toArray(new Database[]{}))
Jonathan Hart054da972015-02-18 17:30:28 -0800271 .stream()
Madan Jampanif1b8e172015-03-23 11:42:02 -0700272 .map(DatabaseManager::toPartitionInfo)
Jonathan Hart054da972015-02-18 17:30:28 -0800273 .collect(Collectors.toList());
274 }
275
Madan Jampanif1b8e172015-03-23 11:42:02 -0700276 private Log newPersistentLog() {
277 String logDir = System.getProperty("karaf.data", "./data");
278 return new FileLog()
279 .withDirectory(logDir)
280 .withSegmentSize(1073741824) // 1GB
281 .withFlushOnWrite(true)
282 .withSegmentInterval(Long.MAX_VALUE);
283 }
284
285 private Log newInMemoryLog() {
286 return new BufferedLog()
287 .withFlushOnWrite(false)
288 .withFlushInterval(Long.MAX_VALUE)
289 .withSegmentSize(10485760) // 10MB
290 .withSegmentInterval(Long.MAX_VALUE);
291 }
292
293 private DatabaseConfig newDatabaseConfig(String name, Log log, String[] replicas) {
294 return new DatabaseConfig()
295 .withName(name)
Madan Jampani1a4eca02015-04-02 15:29:26 -0700296 .withElectionTimeout(electionTimeoutMillis(replicas))
297 .withHeartbeatInterval(heartbeatTimeoutMillis(replicas))
Madan Jampani40537ca2015-07-14 19:50:33 -0700298 .withConsistency(Consistency.DEFAULT)
Madan Jampanif1b8e172015-03-23 11:42:02 -0700299 .withLog(log)
300 .withDefaultSerializer(new DatabaseSerializer())
301 .withReplicas(replicas);
302 }
303
Madan Jampani1a4eca02015-04-02 15:29:26 -0700304 private long electionTimeoutMillis(String[] replicas) {
305 return replicas.length == 1 ? 10L : RAFT_ELECTION_TIMEOUT_MILLIS;
306 }
307
308 private long heartbeatTimeoutMillis(String[] replicas) {
309 return electionTimeoutMillis(replicas) / 2;
310 }
311
Jonathan Hart054da972015-02-18 17:30:28 -0800312 /**
313 * Maps a Raft Database object to a PartitionInfo object.
314 *
315 * @param database database containing input data
316 * @return PartitionInfo object
317 */
Madan Jampanif1b8e172015-03-23 11:42:02 -0700318 private static PartitionInfo toPartitionInfo(Database database) {
Jonathan Hart054da972015-02-18 17:30:28 -0800319 return new PartitionInfo(database.name(),
320 database.cluster().term(),
Madan Jampanif1b8e172015-03-23 11:42:02 -0700321 database.cluster().members()
322 .stream()
323 .filter(member -> Type.ACTIVE.equals(member.type()))
Jonathan Hart054da972015-02-18 17:30:28 -0800324 .map(Member::uri)
Madan Jampanif1b8e172015-03-23 11:42:02 -0700325 .sorted()
Jonathan Hart054da972015-02-18 17:30:28 -0800326 .collect(Collectors.toList()),
327 database.cluster().leader() != null ?
328 database.cluster().leader().uri() : null);
329 }
Jonathan Hart6ec029a2015-03-24 17:12:35 -0700330
331
332 @Override
333 public <K, V> EventuallyConsistentMapBuilder<K, V> eventuallyConsistentMapBuilder() {
334 return new EventuallyConsistentMapBuilderImpl<>(clusterService,
335 clusterCommunicator);
336 }
337
Madan Jampanif1b8e172015-03-23 11:42:02 -0700338 @Override
339 public <K, V> ConsistentMapBuilder<K, V> consistentMapBuilder() {
Madan Jampani50589ac2015-06-08 11:38:46 -0700340 return new DefaultConsistentMapBuilder<>(this);
Madan Jampanif1b8e172015-03-23 11:42:02 -0700341 }
Madan Jampania89f8f92015-04-01 14:39:54 -0700342
343 @Override
Madan Jampani50589ac2015-06-08 11:38:46 -0700344 public <E> DistributedSetBuilder<E> setBuilder() {
345 return new DefaultDistributedSetBuilder<>(this);
Madan Jampani08706ce2015-04-01 14:49:28 -0700346 }
347
Madan Jampani63c659f2015-06-11 00:52:58 -0700348
349 @Override
350 public <E> DistributedQueueBuilder<E> queueBuilder() {
351 return new DefaultDistributedQueueBuilder<>(this);
352 }
353
Madan Jampani08706ce2015-04-01 14:49:28 -0700354 @Override
Madan Jampanib5d72d52015-04-03 16:53:50 -0700355 public AtomicCounterBuilder atomicCounterBuilder() {
356 return new DefaultAtomicCounterBuilder(inMemoryDatabase, partitionedDatabase);
357 }
358
359 @Override
Madan Jampani762246d2015-07-21 15:40:59 -0700360 public <V> AtomicValueBuilder<V> atomicValueBuilder() {
361 return new DefaultAtomicValueBuilder<>(this);
362 }
363
364 @Override
Madan Jampania89f8f92015-04-01 14:39:54 -0700365 public List<MapInfo> getMapInfo() {
366 List<MapInfo> maps = Lists.newArrayList();
367 maps.addAll(getMapInfo(inMemoryDatabase));
368 maps.addAll(getMapInfo(partitionedDatabase));
369 return maps;
370 }
371
372 private List<MapInfo> getMapInfo(Database database) {
Madan Jampani7804c992015-07-20 13:20:19 -0700373 return complete(database.maps())
Madan Jampania89f8f92015-04-01 14:39:54 -0700374 .stream()
Madan Jampani7804c992015-07-20 13:20:19 -0700375 .map(name -> new MapInfo(name, complete(database.mapSize(name))))
Madan Jampania89f8f92015-04-01 14:39:54 -0700376 .filter(info -> info.size() > 0)
377 .collect(Collectors.toList());
378 }
379
Madan Jampanib5d72d52015-04-03 16:53:50 -0700380
381 @Override
382 public Map<String, Long> getCounters() {
383 Map<String, Long> counters = Maps.newHashMap();
384 counters.putAll(complete(inMemoryDatabase.counters()));
385 counters.putAll(complete(partitionedDatabase.counters()));
386 return counters;
387 }
388
Madan Jampanibff6d8f2015-03-31 16:53:47 -0700389 @Override
Kaouther Abrouguid8b565a2015-05-20 16:07:20 -0700390 public Map<String, Long> getPartitionedDatabaseCounters() {
391 Map<String, Long> counters = Maps.newHashMap();
392 counters.putAll(complete(partitionedDatabase.counters()));
393 return counters;
394 }
395
396 @Override
397 public Map<String, Long> getInMemoryDatabaseCounters() {
398 Map<String, Long> counters = Maps.newHashMap();
399 counters.putAll(complete(inMemoryDatabase.counters()));
400 return counters;
401 }
402
403 @Override
Madan Jampanibff6d8f2015-03-31 16:53:47 -0700404 public Collection<Transaction> getTransactions() {
405 return complete(transactionManager.getTransactions());
406 }
407
Madan Jampania89f8f92015-04-01 14:39:54 -0700408 private static <T> T complete(CompletableFuture<T> future) {
409 try {
410 return future.get(DATABASE_OPERATION_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS);
411 } catch (InterruptedException e) {
412 Thread.currentThread().interrupt();
413 throw new ConsistentMapException.Interrupted();
414 } catch (TimeoutException e) {
415 throw new ConsistentMapException.Timeout();
416 } catch (ExecutionException e) {
417 throw new ConsistentMapException(e.getCause());
418 }
419 }
Madan Jampanibff6d8f2015-03-31 16:53:47 -0700420
421 @Override
422 public void redriveTransactions() {
423 getTransactions().stream().forEach(transactionManager::execute);
424 }
Madan Jampani50589ac2015-06-08 11:38:46 -0700425
Brian O'Connor24cf6562015-06-11 18:42:30 -0700426 protected <K, V> DefaultAsyncConsistentMap<K, V> registerMap(DefaultAsyncConsistentMap<K, V> map) {
Madan Jampani85668632015-08-10 10:46:40 -0700427 maps.put(map.name(), map);
428 if (map.applicationId() != null) {
429 mapsByApplication.put(map.applicationId(), map);
Madan Jampani50589ac2015-06-08 11:38:46 -0700430 }
Brian O'Connor24cf6562015-06-11 18:42:30 -0700431 return map;
Madan Jampani50589ac2015-06-08 11:38:46 -0700432 }
433
Madan Jampani63c659f2015-06-11 00:52:58 -0700434 protected <K, V> void unregisterMap(DefaultAsyncConsistentMap<K, V> map) {
Madan Jampani85668632015-08-10 10:46:40 -0700435 maps.remove(map.name(), map);
Madan Jampanie8af1cc2015-06-23 14:23:31 -0700436 if (map.applicationId() != null) {
437 mapsByApplication.remove(map.applicationId(), map);
438 }
Madan Jampani63c659f2015-06-11 00:52:58 -0700439 }
440
Madan Jampanie8af1cc2015-06-23 14:23:31 -0700441 private class InternalApplicationListener implements ApplicationListener {
442 @Override
443 public void event(ApplicationEvent event) {
444 if (event.type() == APP_UNINSTALLED || event.type() == APP_DEACTIVATED) {
445 ApplicationId appId = event.subject().id();
HIGUCHI Yuta99b7b342015-09-29 16:54:21 -0700446 List<DefaultAsyncConsistentMap> mapsToRemove;
447 synchronized (mapsByApplication) {
448 mapsToRemove = ImmutableList.copyOf(mapsByApplication.get(appId));
449 }
Madan Jampanie8af1cc2015-06-23 14:23:31 -0700450 mapsToRemove.forEach(DatabaseManager.this::unregisterMap);
451 if (event.type() == APP_UNINSTALLED) {
452 mapsToRemove.stream().filter(map -> map.purgeOnUninstall()).forEach(map -> map.clear());
453 }
454 }
455 }
456 }
Madan Jampanie9b781a2015-07-07 16:23:20 -0700457}