StorageManager with associated new primitive builders

Change-Id: Ic5d7048cda5ea5e7e86df2d2390c16eea850cd83
diff --git a/core/store/dist/src/main/java/org/onosproject/store/cluster/impl/NewDistributedLeadershipStore.java b/core/store/dist/src/main/java/org/onosproject/store/cluster/impl/NewDistributedLeadershipStore.java
new file mode 100644
index 0000000..ad1d0a2
--- /dev/null
+++ b/core/store/dist/src/main/java/org/onosproject/store/cluster/impl/NewDistributedLeadershipStore.java
@@ -0,0 +1,155 @@
+/*
+ * Copyright 2016 Open Networking Laboratory
+ *
+ * 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 org.onosproject.store.cluster.impl;
+
+import static org.slf4j.LoggerFactory.getLogger;
+
+import java.util.Map;
+import java.util.Objects;
+import java.util.function.Consumer;
+
+import org.apache.felix.scr.annotations.Activate;
+import org.apache.felix.scr.annotations.Component;
+import org.apache.felix.scr.annotations.Deactivate;
+import org.apache.felix.scr.annotations.Reference;
+import org.apache.felix.scr.annotations.ReferenceCardinality;
+import org.apache.felix.scr.annotations.Service;
+import org.onlab.util.Tools;
+import org.onosproject.cluster.ClusterService;
+import org.onosproject.cluster.Leadership;
+import org.onosproject.cluster.LeadershipEvent;
+import org.onosproject.cluster.LeadershipStore;
+import org.onosproject.cluster.LeadershipStoreDelegate;
+import org.onosproject.cluster.NodeId;
+import org.onosproject.event.Change;
+import org.onosproject.store.AbstractStore;
+import org.onosproject.store.service.LeaderElector;
+import org.onosproject.store.service.StorageException;
+import org.onosproject.store.service.StorageService;
+import org.slf4j.Logger;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.Maps;
+
+/**
+ * Implementation of {@code LeadershipStore} that makes use of a {@link LeaderElector}
+ * primitive.
+ */
+@Service
+@Component(immediate = true, enabled = false)
+public class NewDistributedLeadershipStore
+    extends AbstractStore<LeadershipEvent, LeadershipStoreDelegate>
+    implements LeadershipStore {
+
+    private final Logger log = getLogger(getClass());
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected ClusterService clusterService;
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected StorageService storageService;
+
+    private NodeId localNodeId;
+    private LeaderElector leaderElector;
+    private final Map<String, Leadership> leaderBoard = Maps.newConcurrentMap();
+
+    private static final int MAX_RETRIES = 10;
+    private static final int MAX_DELAY_MILLIS_BETWEEN_RETRIES = 100;
+
+    private final Consumer<Change<Leadership>> leadershipChangeListener =
+            change -> {
+                Leadership oldValue = change.oldValue();
+                Leadership newValue = change.newValue();
+                leaderBoard.put(newValue.topic(), newValue);
+                boolean leaderChanged = !Objects.equals(oldValue.leader(), newValue.leader());
+                boolean candidatesChanged = !Objects.equals(oldValue.candidates(), newValue.candidates());
+                LeadershipEvent.Type eventType = null;
+                if (leaderChanged && candidatesChanged) {
+                    eventType = LeadershipEvent.Type.LEADER_AND_CANDIDATES_CHANGED;
+                }
+                if (leaderChanged && !candidatesChanged) {
+                    eventType = LeadershipEvent.Type.LEADER_CHANGED;
+                }
+                if (!leaderChanged && candidatesChanged) {
+                    eventType = LeadershipEvent.Type.CANDIDATES_CHANGED;
+                }
+                notifyDelegate(new LeadershipEvent(eventType, change.newValue()));
+            };
+
+    @Activate
+    public void activate() {
+        localNodeId = clusterService.getLocalNode().id();
+        leaderElector = storageService.leaderElectorBuilder()
+                      .withName("onos-leadership-elections")
+                      .build()
+                      .asLeaderElector();
+        leaderElector.addChangeListener(leadershipChangeListener);
+        leaderBoard.putAll(getLeaderships());
+        log.info("Started");
+    }
+
+    @Deactivate
+    public void deactivate() {
+        leaderElector.removeChangeListener(leadershipChangeListener);
+        log.info("Stopped");
+    }
+
+    @Override
+    public Leadership addRegistration(String topic) {
+        return Tools.retryable(() -> leaderElector.run(topic, localNodeId),
+                               StorageException.class,
+                               MAX_RETRIES,
+                               MAX_DELAY_MILLIS_BETWEEN_RETRIES).get();
+    }
+
+    @Override
+    public void removeRegistration(String topic) {
+        Tools.retryable(() -> {
+                            leaderElector.withdraw(topic);
+                            return null;
+                        },
+                        StorageException.class,
+                        MAX_RETRIES,
+                        MAX_DELAY_MILLIS_BETWEEN_RETRIES).get();
+    }
+
+    @Override
+    public void removeRegistration(NodeId nodeId) {
+        // TODO
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public boolean moveLeadership(String topic, NodeId toNodeId) {
+        return leaderElector.anoint(topic, toNodeId);
+    }
+
+    @Override
+    public boolean makeTopCandidate(String topic, NodeId nodeId) {
+        // TODO
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public Leadership getLeadership(String topic) {
+        return leaderBoard.get(topic);
+    }
+
+    @Override
+    public Map<String, Leadership> getLeaderships() {
+        return ImmutableMap.copyOf(leaderBoard);
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/NewDefaultAtomicCounterBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/NewDefaultAtomicCounterBuilder.java
new file mode 100644
index 0000000..42ef659
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/NewDefaultAtomicCounterBuilder.java
@@ -0,0 +1,40 @@
+/*
+ * Copyright 2016 Open Networking Laboratory
+ *
+ * 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 org.onosproject.store.primitives.impl;
+
+import org.onosproject.store.primitives.DistributedPrimitiveCreator;
+import org.onosproject.store.service.AsyncAtomicCounter;
+import org.onosproject.store.service.AtomicCounterBuilder;
+
+/**
+ * Default implementation of AtomicCounterBuilder.
+ */
+public class NewDefaultAtomicCounterBuilder extends AtomicCounterBuilder {
+
+    private final DistributedPrimitiveCreator base;
+    private final DistributedPrimitiveCreator federated;
+
+    public NewDefaultAtomicCounterBuilder(DistributedPrimitiveCreator base, DistributedPrimitiveCreator federated) {
+        this.base = base;
+        this.federated = federated;
+    }
+
+    @Override
+    public AsyncAtomicCounter build() {
+        DistributedPrimitiveCreator creator = partitionsDisabled() ? base : federated;
+        return creator.newAsyncCounter(name());
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/NewDefaultConsistentMapBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/NewDefaultConsistentMapBuilder.java
new file mode 100644
index 0000000..50b3fea
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/NewDefaultConsistentMapBuilder.java
@@ -0,0 +1,52 @@
+/*
+ * Copyright 2016 Open Networking Laboratory
+ *
+ * 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 org.onosproject.store.primitives.impl;
+
+import org.onosproject.store.primitives.DistributedPrimitiveCreator;
+import org.onosproject.store.service.AsyncConsistentMap;
+import org.onosproject.store.service.ConsistentMap;
+import org.onosproject.store.service.ConsistentMapBuilder;
+
+/**
+ * Default {@link AsyncConsistentMap} builder.
+ *
+ * @param <K> type for map key
+ * @param <V> type for map value
+ */
+public class NewDefaultConsistentMapBuilder<K, V> extends ConsistentMapBuilder<K, V> {
+
+    private final DistributedPrimitiveCreator base;
+    private final DistributedPrimitiveCreator federated;
+
+    public NewDefaultConsistentMapBuilder(DistributedPrimitiveCreator base, DistributedPrimitiveCreator federated) {
+        this.base = base;
+        this.federated = federated;
+    }
+
+    @Override
+    public ConsistentMap<K, V> build() {
+        return buildAsyncMap().asConsistentMap();
+    }
+
+    @Override
+    public AsyncConsistentMap<K, V> buildAsyncMap() {
+        DistributedPrimitiveCreator creator = partitionsDisabled() ? base : federated;
+        AsyncConsistentMap<K, V> map = creator.newAsyncConsistentMap(name(), serializer());
+        map = relaxedReadConsistency() ? DistributedPrimitives.newCachingMap(map) : map;
+        map = readOnly() ? DistributedPrimitives.newUnmodifiableMap(map) : map;
+        return meteringEnabled() ? DistributedPrimitives.newMeteredMap(map) : map;
+    }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/NewDefaultTransactionContext.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/NewDefaultTransactionContext.java
new file mode 100644
index 0000000..39d24cf
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/NewDefaultTransactionContext.java
@@ -0,0 +1,92 @@
+/*
+ * Copyright 2016 Open Networking Laboratory
+ *
+ * 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 org.onosproject.store.primitives.impl;
+
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.onosproject.store.primitives.DistributedPrimitiveCreator;
+import org.onosproject.store.primitives.TransactionId;
+import org.onosproject.store.service.Serializer;
+import org.onosproject.store.service.TransactionContext;
+import org.onosproject.store.service.TransactionalMap;
+
+import com.google.common.collect.Sets;
+
+/**
+ * Default implementation of transaction context.
+ */
+public class NewDefaultTransactionContext implements TransactionContext {
+
+    private final AtomicBoolean isOpen = new AtomicBoolean(false);
+    private final DistributedPrimitiveCreator creator;
+    private final TransactionId transactionId;
+    private final TransactionCoordinator transactionCoordinator;
+    private final Set<TransactionParticipant> txParticipants = Sets.newConcurrentHashSet();
+
+    public NewDefaultTransactionContext(TransactionId transactionId,
+            DistributedPrimitiveCreator creator,
+            TransactionCoordinator transactionCoordinator) {
+        this.transactionId = transactionId;
+        this.creator = creator;
+        this.transactionCoordinator = transactionCoordinator;
+    }
+
+    @Override
+    public String name() {
+        return transactionId.toString();
+    }
+
+    @Override
+    public TransactionId transactionId() {
+        return transactionId;
+    }
+
+    @Override
+    public boolean isOpen() {
+        return isOpen.get();
+    }
+
+    @Override
+    public void begin() {
+        if (!isOpen.compareAndSet(false, true)) {
+            throw new IllegalStateException("TransactionContext is already open");
+        }
+    }
+
+    @Override
+    public boolean commit() {
+        transactionCoordinator.commit(transactionId, txParticipants).getNow(null);
+        return true;
+    }
+
+    @Override
+    public void abort() {
+        isOpen.set(false);
+    }
+
+    @Override
+    public <K, V> TransactionalMap<K, V> getTransactionalMap(String mapName,
+            Serializer serializer) {
+        // FIXME: Do not create duplicates.
+        DefaultTransactionalMap<K, V> txMap = new DefaultTransactionalMap<K, V>(mapName,
+                creator.<K, V>newAsyncConsistentMap(mapName, serializer),
+                this,
+                serializer);
+        txParticipants.add(txMap);
+        return txMap;
+    }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/NewDefaultTransactionContextBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/NewDefaultTransactionContextBuilder.java
new file mode 100644
index 0000000..3a27122
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/NewDefaultTransactionContextBuilder.java
@@ -0,0 +1,48 @@
+/*
+ * Copyright 2016 Open Networking Laboratory
+ *
+ * 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 org.onosproject.store.primitives.impl;
+
+import org.onosproject.store.primitives.DistributedPrimitiveCreator;
+import org.onosproject.store.primitives.TransactionId;
+import org.onosproject.store.service.TransactionContext;
+import org.onosproject.store.service.TransactionContextBuilder;
+/**
+ * Default Transaction Context Builder.
+ */
+public class NewDefaultTransactionContextBuilder extends TransactionContextBuilder {
+
+    private final TransactionId transactionId;
+    private final DistributedPrimitiveCreator base;
+    private final DistributedPrimitiveCreator federated;
+    private final TransactionCoordinator transactionCoordinator;
+
+    public NewDefaultTransactionContextBuilder(TransactionId transactionId,
+            DistributedPrimitiveCreator base,
+            DistributedPrimitiveCreator federated,
+            TransactionCoordinator transactionCoordinator) {
+        this.transactionId = transactionId;
+        this.base = base;
+        this.federated = federated;
+        this.transactionCoordinator = transactionCoordinator;
+    }
+
+    @Override
+    public TransactionContext build() {
+        return new NewDefaultTransactionContext(transactionId,
+                this.partitionsDisabled() ? base : federated,
+                transactionCoordinator);
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StorageManager.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StorageManager.java
new file mode 100644
index 0000000..f1eb4ce
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StorageManager.java
@@ -0,0 +1,220 @@
+/*
+ * Copyright 2016 Open Networking Laboratory
+ *
+ * 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 org.onosproject.store.primitives.impl;
+
+import static org.slf4j.LoggerFactory.getLogger;
+
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+import java.util.UUID;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
+import org.apache.commons.collections.ListUtils;
+import org.apache.felix.scr.annotations.Activate;
+import org.apache.felix.scr.annotations.Component;
+import org.apache.felix.scr.annotations.Deactivate;
+import org.apache.felix.scr.annotations.Reference;
+import org.apache.felix.scr.annotations.ReferenceCardinality;
+import org.apache.felix.scr.annotations.Service;
+import org.onosproject.cluster.ClusterService;
+import org.onosproject.cluster.PartitionId;
+import org.onosproject.persistence.PersistenceService;
+import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
+import org.onosproject.store.primitives.DistributedPrimitiveCreator;
+import org.onosproject.store.primitives.MapUpdate;
+import org.onosproject.store.primitives.PartitionService;
+import org.onosproject.store.primitives.TransactionId;
+import org.onosproject.store.serializers.KryoNamespaces;
+import org.onosproject.store.service.AsyncConsistentMap;
+import org.onosproject.store.service.AtomicCounterBuilder;
+import org.onosproject.store.service.AtomicValueBuilder;
+import org.onosproject.store.service.ConsistentMap;
+import org.onosproject.store.service.ConsistentMapBuilder;
+import org.onosproject.store.service.DistributedQueueBuilder;
+import org.onosproject.store.service.DistributedSetBuilder;
+import org.onosproject.store.service.EventuallyConsistentMapBuilder;
+import org.onosproject.store.service.LeaderElectorBuilder;
+import org.onosproject.store.service.MapInfo;
+import org.onosproject.store.service.PartitionInfo;
+import org.onosproject.store.service.Serializer;
+import org.onosproject.store.service.StorageAdminService;
+import org.onosproject.store.service.StorageService;
+import org.onosproject.store.service.TransactionContextBuilder;
+import org.slf4j.Logger;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.Futures;
+
+/**
+ * Implementation for {@code StorageService} and {@code StorageAdminService}.
+ */
+@Service
+@Component(immediate = true, enabled = false)
+public class StorageManager implements StorageService, StorageAdminService {
+
+    private final Logger log = getLogger(getClass());
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected ClusterService clusterService;
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected ClusterCommunicationService clusterCommunicator;
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected PersistenceService persistenceService;
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected PartitionService partitionService;
+
+    private final Supplier<TransactionId> transactionIdGenerator =
+            () -> TransactionId.from(UUID.randomUUID().toString());
+    private DistributedPrimitiveCreator basePrimitiveCreator;
+    private DistributedPrimitiveCreator federatedPrimitiveCreator;
+    private AsyncConsistentMap<TransactionId, Transaction.State> transactions;
+    private TransactionCoordinator transactionCoordinator;
+
+    @Activate
+    public void actiavte() {
+        basePrimitiveCreator = partitionService.getDistributedPrimitiveCreator(PartitionId.from(0));
+        Map<PartitionId, DistributedPrimitiveCreator> partitionMap = Maps.newHashMap();
+        partitionService.getAllPartitionIds().stream()
+            .filter(id -> !id.equals(PartitionId.from(0)))
+            .forEach(id -> partitionMap.put(id, partitionService.getDistributedPrimitiveCreator(id)));
+        federatedPrimitiveCreator = new FederatedDistributedPrimitiveCreator(partitionMap);
+        transactions = this.<TransactionId, Transaction.State>consistentMapBuilder()
+                    .withName("onos-transactions")
+                    .withSerializer(Serializer.using(KryoNamespaces.API,
+                            MapUpdate.class,
+                            MapUpdate.Type.class,
+                            Transaction.class,
+                            Transaction.State.class))
+                    .buildAsyncMap();
+        transactionCoordinator = new TransactionCoordinator(transactions);
+        log.info("Started");
+    }
+
+    @Deactivate
+    public void deactivate() {
+        log.info("Stopped");
+    }
+
+    @Override
+    public <K, V> EventuallyConsistentMapBuilder<K, V> eventuallyConsistentMapBuilder() {
+        return new EventuallyConsistentMapBuilderImpl<>(clusterService,
+                clusterCommunicator,
+                persistenceService);
+    }
+
+    @Override
+    public <K, V> ConsistentMapBuilder<K, V> consistentMapBuilder() {
+        return new NewDefaultConsistentMapBuilder<>(basePrimitiveCreator, federatedPrimitiveCreator);
+    }
+
+    @Override
+    public <E> DistributedSetBuilder<E> setBuilder() {
+        return new DefaultDistributedSetBuilder<>(() -> this.<E, Boolean>consistentMapBuilder());
+    }
+
+    @Override
+    public <E> DistributedQueueBuilder<E> queueBuilder() {
+        // TODO: implement
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public AtomicCounterBuilder atomicCounterBuilder() {
+        return new NewDefaultAtomicCounterBuilder(basePrimitiveCreator, federatedPrimitiveCreator);
+    }
+
+    @Override
+    public <V> AtomicValueBuilder<V> atomicValueBuilder() {
+        Supplier<ConsistentMapBuilder<String, byte[]>> mapBuilderSupplier =
+                () -> this.<String, byte[]>consistentMapBuilder()
+                          .withName("onos-atomic-values")
+                          .withMeteringDisabled()
+                          .withSerializer(Serializer.using(KryoNamespaces.BASIC));
+        return new DefaultAtomicValueBuilder<>(mapBuilderSupplier);
+    }
+
+    @Override
+    public TransactionContextBuilder transactionContextBuilder() {
+        return new NewDefaultTransactionContextBuilder(transactionIdGenerator.get(),
+                basePrimitiveCreator,
+                federatedPrimitiveCreator,
+                transactionCoordinator);
+    }
+
+    @Override
+    public LeaderElectorBuilder leaderElectorBuilder() {
+        return new DefaultLeaderElectorBuilder(basePrimitiveCreator,
+                federatedPrimitiveCreator);
+    }
+
+    @Override
+    public List<MapInfo> getMapInfo() {
+        return ListUtils.union(listMapInfo(basePrimitiveCreator), listMapInfo(federatedPrimitiveCreator));
+    }
+
+    @Override
+    public Map<String, Long> getCounters() {
+        Map<String, Long> result = Maps.newHashMap();
+        result.putAll(getInMemoryDatabaseCounters());
+        result.putAll(getPartitionedDatabaseCounters());
+        return result;
+    }
+
+    @Override
+    public Map<String, Long> getInMemoryDatabaseCounters() {
+        return getCounters(basePrimitiveCreator);
+    }
+
+    @Override
+    public Map<String, Long> getPartitionedDatabaseCounters() {
+        return getCounters(federatedPrimitiveCreator);
+    }
+
+    public Map<String, Long> getCounters(DistributedPrimitiveCreator creator) {
+        Map<String, Long> counters = Maps.newConcurrentMap();
+        creator.getAsyncAtomicCounterNames()
+               .forEach(name -> counters.put(name, creator.newAsyncCounter(name).asAtomicCounter().get()));
+        return counters;
+    }
+
+    @Override
+    public List<PartitionInfo> getPartitionInfo() {
+        return Lists.newArrayList();
+    }
+
+    @Override
+    public Collection<TransactionId> getPendingTransactions() {
+        return Futures.getUnchecked(transactions.keySet());
+    }
+
+    private List<MapInfo> listMapInfo(DistributedPrimitiveCreator creator) {
+        Serializer serializer = Serializer.using(KryoNamespaces.BASIC);
+        return creator.getAsyncConsistentMapNames()
+        .stream()
+        .map(name -> {
+            ConsistentMap<String, byte[]> map =
+                    creator.<String, byte[]>newAsyncConsistentMap(name, serializer)
+                                             .asConsistentMap();
+                    return new MapInfo(name, map.size());
+        }).collect(Collectors.toList());
+    }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionCoordinator.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionCoordinator.java
new file mode 100644
index 0000000..cd2768f
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionCoordinator.java
@@ -0,0 +1,79 @@
+/*
+ * Copyright 2016 Open Networking Laboratory
+ *
+ * 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 org.onosproject.store.primitives.impl;
+
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import org.onlab.util.Tools;
+import org.onosproject.store.primitives.TransactionId;
+import org.onosproject.store.service.AsyncConsistentMap;
+
+/**
+ * Coordinator for a two-phase commit protocol.
+ */
+public class TransactionCoordinator {
+
+    private final AsyncConsistentMap<TransactionId, Transaction.State> transactions;
+
+    public TransactionCoordinator(AsyncConsistentMap<TransactionId, Transaction.State> transactions) {
+        this.transactions = transactions;
+    }
+
+    /**
+     * Commits a transaction.
+     * @param transactionId transaction
+     * @return future for commit result
+     */
+    CompletableFuture<Void> commit(TransactionId transactionId, Set<TransactionParticipant> transactionParticipants) {
+        if (!transactionParticipants.stream().anyMatch(t -> t.hasPendingUpdates())) {
+            return CompletableFuture.completedFuture(null);
+        }
+
+       return  transactions.put(transactionId, Transaction.State.PREPARING)
+                    .thenCompose(v -> this.doPrepare(transactionParticipants))
+                    .thenCompose(result -> result
+                           ? transactions.put(transactionId, Transaction.State.COMMITTING)
+                                         .thenCompose(v -> doCommit(transactionParticipants))
+                                         .thenApply(v -> null)
+                           : transactions.put(transactionId, Transaction.State.ROLLINGBACK)
+                                         .thenCompose(v -> doRollback(transactionParticipants))
+                                         .thenApply(v -> null))
+                    .thenCompose(v -> transactions.remove(transactionId).thenApply(u -> null))
+                    .thenApply(v -> null);
+    }
+
+    private CompletableFuture<Boolean> doPrepare(Set<TransactionParticipant> transactionParticipants) {
+        return Tools.allOf(transactionParticipants
+                                           .stream()
+                                           .map(TransactionParticipant::prepare)
+                                           .collect(Collectors.toList()))
+                    .thenApply(list -> list.stream().reduce(Boolean::logicalAnd).orElse(true));
+    }
+
+    private CompletableFuture<Void> doCommit(Set<TransactionParticipant> transactionParticipants) {
+        return CompletableFuture.allOf(transactionParticipants.stream()
+                                                              .map(p -> p.commit())
+                                                              .toArray(CompletableFuture[]::new));
+    }
+
+    private CompletableFuture<Void> doRollback(Set<TransactionParticipant> transactionParticipants) {
+        return CompletableFuture.allOf(transactionParticipants.stream()
+                                                              .map(p -> p.rollback())
+                                                              .toArray(CompletableFuture[]::new));
+    }
+}
\ No newline at end of file