Upgrade to Atomix 3.0-rc5
* Upgrade Raft primitives to Atomix 3.0
* Replace cluster store and messaging implementations with Atomix cluster management/messaging
* Add test scripts for installing/starting Atomix cluster
* Replace core primitives with Atomix primitives.

Change-Id: I7623653c81292a34f21b01f5f38ca11b5ef15cad
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/cluster/impl/AtomixClusterStore.java b/core/store/primitives/src/main/java/org/onosproject/store/cluster/impl/AtomixClusterStore.java
new file mode 100644
index 0000000..0afc462
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/cluster/impl/AtomixClusterStore.java
@@ -0,0 +1,241 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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 java.time.Instant;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Maps;
+import io.atomix.cluster.ClusterMembershipEvent;
+import io.atomix.cluster.ClusterMembershipEventListener;
+import io.atomix.cluster.ClusterMembershipService;
+import io.atomix.cluster.Member;
+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.packet.IpAddress;
+import org.onosproject.cluster.ClusterEvent;
+import org.onosproject.cluster.ClusterStore;
+import org.onosproject.cluster.ClusterStoreDelegate;
+import org.onosproject.cluster.ControllerNode;
+import org.onosproject.cluster.DefaultControllerNode;
+import org.onosproject.cluster.Node;
+import org.onosproject.cluster.NodeId;
+import org.onosproject.core.Version;
+import org.onosproject.core.VersionService;
+import org.onosproject.store.AbstractStore;
+import org.onosproject.store.impl.AtomixManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Atomix cluster store.
+ */
+@Component(immediate = true)
+@Service
+public class AtomixClusterStore extends AbstractStore<ClusterEvent, ClusterStoreDelegate> implements ClusterStore {
+    private static final String INSTANCE_ID_NULL = "Instance ID cannot be null";
+
+    private static final String STATE_KEY = "state";
+    private static final String VERSION_KEY = "version";
+
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected AtomixManager atomixManager;
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected VersionService versionService;
+
+    private ClusterMembershipService membershipService;
+    private ControllerNode localNode;
+    private final Map<NodeId, ControllerNode> nodes = Maps.newConcurrentMap();
+    private final Map<NodeId, ControllerNode.State> states = Maps.newConcurrentMap();
+    private final Map<NodeId, Version> versions = Maps.newConcurrentMap();
+    private final Map<NodeId, Instant> updates = Maps.newConcurrentMap();
+    private final ClusterMembershipEventListener membershipEventListener = this::changeMembership;
+
+    @Activate
+    public void activate() {
+        membershipService = atomixManager.getAtomix().getMembershipService();
+        membershipService.addListener(membershipEventListener);
+        membershipService.getMembers().forEach(member -> {
+            ControllerNode node = toControllerNode(member);
+            nodes.put(node.id(), node);
+            updateState(node, member);
+            updateVersion(node, member);
+        });
+        membershipService.getLocalMember().properties().put(STATE_KEY, ControllerNode.State.ACTIVE.name());
+        membershipService.getLocalMember().properties().put(VERSION_KEY, versionService.version().toString());
+        localNode = toControllerNode(membershipService.getLocalMember());
+        states.put(localNode.id(), ControllerNode.State.ACTIVE);
+        versions.put(localNode.id(), versionService.version());
+        log.info("Started");
+    }
+
+    @Deactivate
+    public void deactivate() {
+        membershipService.removeListener(membershipEventListener);
+        log.info("Stopped");
+    }
+
+    private void changeMembership(ClusterMembershipEvent event) {
+        ControllerNode node = nodes.get(NodeId.nodeId(event.subject().id().id()));
+        switch (event.type()) {
+            case MEMBER_ADDED:
+            case METADATA_CHANGED:
+                if (node == null) {
+                    node = toControllerNode(event.subject());
+                    nodes.put(node.id(), node);
+                    notifyDelegate(new ClusterEvent(ClusterEvent.Type.INSTANCE_ADDED, node));
+                }
+                updateVersion(node, event.subject());
+                updateState(node, event.subject());
+                break;
+            case MEMBER_REMOVED:
+                if (node != null
+                    && states.put(node.id(), ControllerNode.State.INACTIVE) != ControllerNode.State.INACTIVE) {
+                    notifyDelegate(new ClusterEvent(ClusterEvent.Type.INSTANCE_DEACTIVATED, node));
+                    notifyDelegate(new ClusterEvent(ClusterEvent.Type.INSTANCE_REMOVED, node));
+                }
+                break;
+            default:
+                break;
+        }
+    }
+
+    private void updateState(ControllerNode node, Member member) {
+        String state = member.properties().getProperty(STATE_KEY);
+        if (state == null || !state.equals(ControllerNode.State.READY.name())) {
+            if (states.put(node.id(), ControllerNode.State.ACTIVE) != ControllerNode.State.ACTIVE) {
+                log.info("Updated node {} state to {}", node.id(), ControllerNode.State.ACTIVE);
+                markUpdated(node.id());
+                notifyDelegate(new ClusterEvent(ClusterEvent.Type.INSTANCE_ACTIVATED, node));
+            }
+        } else {
+            if (states.put(node.id(), ControllerNode.State.READY) != ControllerNode.State.READY) {
+                log.info("Updated node {} state to {}", node.id(), ControllerNode.State.READY);
+                markUpdated(node.id());
+                notifyDelegate(new ClusterEvent(ClusterEvent.Type.INSTANCE_READY, node));
+            }
+        }
+    }
+
+    private void updateVersion(ControllerNode node, Member member) {
+        String versionString = member.properties().getProperty(VERSION_KEY);
+        if (versionString != null) {
+            Version version = Version.version(versionString);
+            if (!Objects.equals(versions.put(node.id(), version), version)) {
+                log.info("Updated node {} version to {}", node.id(), version);
+            }
+        }
+    }
+
+    private void markUpdated(NodeId nodeId) {
+        updates.put(nodeId, Instant.now());
+    }
+
+    private ControllerNode toControllerNode(Member member) {
+        return new DefaultControllerNode(
+            NodeId.nodeId(member.id().id()),
+            IpAddress.valueOf(member.address().address()),
+            member.address().port());
+    }
+
+    @Override
+    public ControllerNode getLocalNode() {
+        return toControllerNode(membershipService.getLocalMember());
+    }
+
+    @Override
+    public Set<Node> getStorageNodes() {
+        return membershipService.getMembers()
+            .stream()
+            .filter(member -> !Objects.equals(member.properties().getProperty("type"), "onos"))
+            .map(this::toControllerNode)
+            .collect(Collectors.toSet());
+    }
+
+    @Override
+    public Set<ControllerNode> getNodes() {
+        return membershipService.getMembers()
+            .stream()
+            .filter(member -> Objects.equals(member.properties().getProperty("type"), "onos"))
+            .map(this::toControllerNode)
+            .collect(Collectors.toSet());
+    }
+
+    @Override
+    public ControllerNode getNode(NodeId nodeId) {
+        Member member = membershipService.getMember(nodeId.id());
+        return member != null ? toControllerNode(member) : null;
+    }
+
+    @Override
+    public ControllerNode.State getState(NodeId nodeId) {
+        checkNotNull(nodeId, INSTANCE_ID_NULL);
+        return states.get(nodeId);
+    }
+
+    @Override
+    public Version getVersion(NodeId nodeId) {
+        checkNotNull(nodeId, INSTANCE_ID_NULL);
+        return versions.get(nodeId);
+    }
+
+    @Override
+    public Instant getLastUpdatedInstant(NodeId nodeId) {
+        checkNotNull(nodeId, INSTANCE_ID_NULL);
+        return updates.get(nodeId);
+    }
+
+    @Override
+    public void markFullyStarted(boolean started) {
+        ControllerNode.State state = started ? ControllerNode.State.READY : ControllerNode.State.ACTIVE;
+        states.put(localNode.id(), state);
+        membershipService.getLocalMember().properties().setProperty(STATE_KEY, state.name());
+    }
+
+    @Override
+    public ControllerNode addNode(NodeId nodeId, IpAddress ip, int tcpPort) {
+        checkNotNull(nodeId, INSTANCE_ID_NULL);
+        ControllerNode node = new DefaultControllerNode(nodeId, ip, tcpPort);
+        nodes.put(node.id(), node);
+        ControllerNode.State state = node.equals(localNode)
+            ? ControllerNode.State.ACTIVE : ControllerNode.State.INACTIVE;
+        membershipService.getMember(node.id().id()).properties().setProperty(STATE_KEY, state.name());
+        notifyDelegate(new ClusterEvent(ClusterEvent.Type.INSTANCE_ADDED, node));
+        return node;
+    }
+
+    @Override
+    public void removeNode(NodeId nodeId) {
+        checkNotNull(nodeId, INSTANCE_ID_NULL);
+        ControllerNode node = nodes.remove(nodeId);
+        if (node != null) {
+            states.remove(nodeId);
+            notifyDelegate(new ClusterEvent(ClusterEvent.Type.INSTANCE_REMOVED, node));
+        }
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/package-info.java b/core/store/primitives/src/main/java/org/onosproject/store/cluster/impl/package-info.java
similarity index 76%
copy from core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/package-info.java
copy to core/store/primitives/src/main/java/org/onosproject/store/cluster/impl/package-info.java
index aa11664..23bd9c4 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/package-info.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/cluster/impl/package-info.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2016-present Open Networking Foundation
+ * Copyright 2018-present Open Networking Foundation
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
@@ -15,6 +15,6 @@
  */
 
 /**
- * State machine implementation classes for distributed primitives.
+ * Implementation classes for cluster stores.
  */
-package org.onosproject.store.primitives.resources.impl;
+package org.onosproject.store.cluster.impl;
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/cluster/messaging/impl/AtomixMessagingManager.java b/core/store/primitives/src/main/java/org/onosproject/store/cluster/messaging/impl/AtomixMessagingManager.java
new file mode 100644
index 0000000..46b27bf
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/cluster/messaging/impl/AtomixMessagingManager.java
@@ -0,0 +1,109 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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.messaging.impl;
+
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.function.BiConsumer;
+import java.util.function.BiFunction;
+
+import io.atomix.utils.net.Address;
+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.packet.IpAddress;
+import org.onosproject.store.cluster.messaging.Endpoint;
+import org.onosproject.store.cluster.messaging.MessagingService;
+import org.onosproject.store.impl.AtomixManager;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Atomix messaging manager.
+ */
+@Component(immediate = true)
+@Service
+public class AtomixMessagingManager implements MessagingService {
+
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected AtomixManager atomixManager;
+
+    private io.atomix.cluster.messaging.MessagingService messagingService;
+
+    @Activate
+    public void activate() {
+        messagingService = atomixManager.getAtomix().getMessagingService();
+        log.info("Started");
+    }
+
+    @Deactivate
+    public void deactivate() {
+        log.info("Stopped");
+    }
+
+    private Address toAddress(Endpoint ep) {
+        return new Address(ep.host().toString(), ep.port(), ep.host().toInetAddress());
+    }
+
+    private Endpoint toEndpoint(Address address) {
+        return new Endpoint(IpAddress.valueOf(address.address()), address.port());
+    }
+
+    @Override
+    public CompletableFuture<Void> sendAsync(Endpoint ep, String type, byte[] payload) {
+        return messagingService.sendAsync(toAddress(ep), type, payload);
+    }
+
+    @Override
+    public CompletableFuture<byte[]> sendAndReceive(Endpoint ep, String type, byte[] payload) {
+        return messagingService.sendAndReceive(toAddress(ep), type, payload);
+    }
+
+    @Override
+    public CompletableFuture<byte[]> sendAndReceive(Endpoint ep, String type, byte[] payload, Executor executor) {
+        return messagingService.sendAndReceive(toAddress(ep), type, payload, executor);
+    }
+
+    @Override
+    public void registerHandler(String type, BiConsumer<Endpoint, byte[]> handler, Executor executor) {
+        BiConsumer<Address, byte[]> consumer = (address, payload) -> handler.accept(toEndpoint(address), payload);
+        messagingService.registerHandler(type, consumer, executor);
+    }
+
+    @Override
+    public void registerHandler(String type, BiFunction<Endpoint, byte[], byte[]> handler, Executor executor) {
+        BiFunction<Address, byte[], byte[]> function = (address, payload) ->
+            handler.apply(toEndpoint(address), payload);
+        messagingService.registerHandler(type, function, executor);
+    }
+
+    @Override
+    public void registerHandler(String type, BiFunction<Endpoint, byte[], CompletableFuture<byte[]>> handler) {
+        BiFunction<Address, byte[], CompletableFuture<byte[]>> function = (address, payload) ->
+            handler.apply(toEndpoint(address), payload);
+        messagingService.registerHandler(type, function);
+    }
+
+    @Override
+    public void unregisterHandler(String type) {
+        messagingService.unregisterHandler(type);
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/package-info.java b/core/store/primitives/src/main/java/org/onosproject/store/cluster/messaging/impl/package-info.java
similarity index 76%
rename from core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/package-info.java
rename to core/store/primitives/src/main/java/org/onosproject/store/cluster/messaging/impl/package-info.java
index aa11664..b88387a 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/package-info.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/cluster/messaging/impl/package-info.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2016-present Open Networking Foundation
+ * Copyright 2018-present Open Networking Foundation
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
@@ -15,6 +15,6 @@
  */
 
 /**
- * State machine implementation classes for distributed primitives.
+ * Implementation classes for cluster messaging.
  */
-package org.onosproject.store.primitives.resources.impl;
+package org.onosproject.store.cluster.messaging.impl;
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/impl/AtomixManager.java b/core/store/primitives/src/main/java/org/onosproject/store/impl/AtomixManager.java
new file mode 100644
index 0000000..43cde78
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/impl/AtomixManager.java
@@ -0,0 +1,84 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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.impl;
+
+import java.util.stream.Collectors;
+
+import io.atomix.cluster.discovery.BootstrapDiscoveryProvider;
+import io.atomix.core.Atomix;
+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.ClusterMetadataService;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Atomix manager.
+ */
+@Component(immediate = true)
+@Service(value = AtomixManager.class)
+public class AtomixManager {
+
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected ClusterMetadataService metadataService;
+
+    private Atomix atomix;
+
+    /**
+     * Returns the Atomix instance.
+     *
+     * @return the Atomix instance
+     */
+    public Atomix getAtomix() {
+        return atomix;
+    }
+
+    @Activate
+    public void activate() {
+        atomix = createAtomix();
+        atomix.start().join();
+        log.info("Started");
+    }
+
+    @Deactivate
+    public void deactivate() {
+        atomix.stop().join();
+        log.info("Stopped");
+    }
+
+    private Atomix createAtomix() {
+        return Atomix.builder(getClass().getClassLoader())
+            .withClusterId(metadataService.getClusterMetadata().getName())
+            .withMemberId(metadataService.getLocalNode().id().id())
+            .withAddress(metadataService.getLocalNode().ip().toString(), metadataService.getLocalNode().tcpPort())
+            .withProperty("type", "onos")
+            .withMembershipProvider(BootstrapDiscoveryProvider.builder()
+                .withNodes(metadataService.getClusterMetadata().getStorageNodes().stream()
+                    .map(node -> io.atomix.cluster.Node.builder()
+                        .withId(node.id().id())
+                        .withAddress(node.ip().toString(), node.tcpPort())
+                        .build())
+                    .collect(Collectors.toList()))
+                .build())
+            .build();
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/package-info.java b/core/store/primitives/src/main/java/org/onosproject/store/impl/package-info.java
similarity index 76%
copy from core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/package-info.java
copy to core/store/primitives/src/main/java/org/onosproject/store/impl/package-info.java
index aa11664..6f34a50 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/package-info.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/impl/package-info.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2016-present Open Networking Foundation
+ * Copyright 2018-present Open Networking Foundation
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
@@ -15,6 +15,6 @@
  */
 
 /**
- * State machine implementation classes for distributed primitives.
+ * Implementation classes for various Distributed primitives.
  */
-package org.onosproject.store.primitives.resources.impl;
+package org.onosproject.store.impl;
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AntiEntropyAdvertisement.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AntiEntropyAdvertisement.java
index 373273c..0004fe1 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AntiEntropyAdvertisement.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AntiEntropyAdvertisement.java
@@ -15,13 +15,12 @@
  */
 package org.onosproject.store.primitives.impl;
 
+import java.util.Map;
+
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ImmutableMap;
-
 import org.onosproject.cluster.NodeId;
 
-import java.util.Map;
-
 import static com.google.common.base.Preconditions.checkNotNull;
 
 /**
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounter.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounter.java
new file mode 100644
index 0000000..ce5c248
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounter.java
@@ -0,0 +1,71 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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.concurrent.CompletableFuture;
+
+import org.onosproject.store.service.AsyncAtomicCounter;
+
+/**
+ * Atomix atomic counter.
+ */
+public class AtomixAtomicCounter implements AsyncAtomicCounter {
+    private final io.atomix.core.counter.AsyncAtomicCounter atomixCounter;
+
+    public AtomixAtomicCounter(io.atomix.core.counter.AsyncAtomicCounter atomixCounter) {
+        this.atomixCounter = atomixCounter;
+    }
+
+    @Override
+    public String name() {
+        return atomixCounter.name();
+    }
+
+    @Override
+    public CompletableFuture<Long> incrementAndGet() {
+        return atomixCounter.incrementAndGet();
+    }
+
+    @Override
+    public CompletableFuture<Long> getAndIncrement() {
+        return atomixCounter.getAndIncrement();
+    }
+
+    @Override
+    public CompletableFuture<Long> getAndAdd(long delta) {
+        return atomixCounter.getAndAdd(delta);
+    }
+
+    @Override
+    public CompletableFuture<Long> addAndGet(long delta) {
+        return atomixCounter.addAndGet(delta);
+    }
+
+    @Override
+    public CompletableFuture<Long> get() {
+        return atomixCounter.get();
+    }
+
+    @Override
+    public CompletableFuture<Void> set(long value) {
+        return atomixCounter.set(value);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> compareAndSet(long expectedValue, long updateValue) {
+        return atomixCounter.compareAndSet(expectedValue, updateValue);
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounterBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounterBuilder.java
new file mode 100644
index 0000000..b18c128
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounterBuilder.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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 io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.AsyncAtomicCounter;
+import org.onosproject.store.service.AtomicCounterBuilder;
+
+/**
+ * Default {@link org.onosproject.store.service.AsyncAtomicCounter} builder.
+ */
+public class AtomixAtomicCounterBuilder extends AtomicCounterBuilder {
+    private static final int MAX_RETRIES = 5;
+    private final Atomix atomix;
+    private final String group;
+
+    public AtomixAtomicCounterBuilder(Atomix atomix, String group) {
+        this.atomix = atomix;
+        this.group = group;
+    }
+
+    @Override
+    public AsyncAtomicCounter build() {
+        return new AtomixAtomicCounter(atomix.atomicCounterBuilder(name())
+            .withProtocol(MultiRaftProtocol.builder(group)
+                .withRecoveryStrategy(Recovery.RECOVER)
+                .withMaxRetries(MAX_RETRIES)
+                .build())
+            .withReadOnly(readOnly())
+            .withSerializer(new AtomixSerializerAdapter(serializer()))
+            .build()
+            .async());
+    }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounterMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounterMap.java
new file mode 100644
index 0000000..fc0aca1
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounterMap.java
@@ -0,0 +1,111 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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.concurrent.CompletableFuture;
+
+import org.onosproject.store.service.AsyncAtomicCounterMap;
+
+/**
+ * Atomix atomic counter map.
+ */
+public class AtomixAtomicCounterMap<K> implements AsyncAtomicCounterMap<K> {
+    private final io.atomix.core.map.AsyncAtomicCounterMap<K> atomixMap;
+
+    public AtomixAtomicCounterMap(io.atomix.core.map.AsyncAtomicCounterMap<K> atomixMap) {
+        this.atomixMap = atomixMap;
+    }
+
+    @Override
+    public String name() {
+        return atomixMap.name();
+    }
+
+    @Override
+    public CompletableFuture<Long> incrementAndGet(K key) {
+        return atomixMap.incrementAndGet(key);
+    }
+
+    @Override
+    public CompletableFuture<Long> decrementAndGet(K key) {
+        return atomixMap.decrementAndGet(key);
+    }
+
+    @Override
+    public CompletableFuture<Long> getAndIncrement(K key) {
+        return atomixMap.getAndIncrement(key);
+    }
+
+    @Override
+    public CompletableFuture<Long> getAndDecrement(K key) {
+        return atomixMap.getAndDecrement(key);
+    }
+
+    @Override
+    public CompletableFuture<Long> addAndGet(K key, long delta) {
+        return atomixMap.addAndGet(key, delta);
+    }
+
+    @Override
+    public CompletableFuture<Long> getAndAdd(K key, long delta) {
+        return atomixMap.getAndAdd(key, delta);
+    }
+
+    @Override
+    public CompletableFuture<Long> get(K key) {
+        return atomixMap.get(key);
+    }
+
+    @Override
+    public CompletableFuture<Long> put(K key, long newValue) {
+        return atomixMap.put(key, newValue);
+    }
+
+    @Override
+    public CompletableFuture<Long> putIfAbsent(K key, long newValue) {
+        return atomixMap.putIfAbsent(key, newValue);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> replace(K key, long expectedOldValue, long newValue) {
+        return atomixMap.replace(key, expectedOldValue, newValue);
+    }
+
+    @Override
+    public CompletableFuture<Long> remove(K key) {
+        return atomixMap.remove(key);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> remove(K key, long value) {
+        return atomixMap.remove(key, value);
+    }
+
+    @Override
+    public CompletableFuture<Integer> size() {
+        return atomixMap.size();
+    }
+
+    @Override
+    public CompletableFuture<Boolean> isEmpty() {
+        return atomixMap.isEmpty();
+    }
+
+    @Override
+    public CompletableFuture<Void> clear() {
+        return atomixMap.clear();
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounterMapBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounterMapBuilder.java
new file mode 100644
index 0000000..03177bf
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicCounterMapBuilder.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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 io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.AsyncAtomicCounterMap;
+import org.onosproject.store.service.AtomicCounterMap;
+import org.onosproject.store.service.AtomicCounterMapBuilder;
+
+/**
+ * Default {@link org.onosproject.store.service.AsyncAtomicCounterMap} builder.
+ *
+ * @param <K> type for tree value
+ */
+public class AtomixAtomicCounterMapBuilder<K> extends AtomicCounterMapBuilder<K> {
+    private static final int MAX_RETRIES = 5;
+    private final Atomix atomix;
+    private final String group;
+
+    public AtomixAtomicCounterMapBuilder(Atomix atomix, String group) {
+        this.atomix = atomix;
+        this.group = group;
+    }
+
+    @Override
+    public AtomicCounterMap<K> build() {
+        return buildAsyncMap().asAtomicCounterMap();
+    }
+
+    @Override
+    public AsyncAtomicCounterMap<K> buildAsyncMap() {
+        return new AtomixAtomicCounterMap<K>(atomix.<K>atomicCounterMapBuilder(name())
+            .withProtocol(MultiRaftProtocol.builder(group)
+                .withRecoveryStrategy(Recovery.RECOVER)
+                .withMaxRetries(MAX_RETRIES)
+                .build())
+            .withReadOnly(readOnly())
+            .withSerializer(new AtomixSerializerAdapter(serializer()))
+            .build()
+            .async());
+    }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicIdGenerator.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicIdGenerator.java
new file mode 100644
index 0000000..2bafa0b
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicIdGenerator.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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.concurrent.CompletableFuture;
+
+import org.onosproject.store.service.AsyncAtomicIdGenerator;
+
+/**
+ * Atomix atomic ID generator.
+ */
+public class AtomixAtomicIdGenerator implements AsyncAtomicIdGenerator {
+    private final io.atomix.core.idgenerator.AsyncAtomicIdGenerator atomixIdGenerator;
+
+    public AtomixAtomicIdGenerator(io.atomix.core.idgenerator.AsyncAtomicIdGenerator atomixIdGenerator) {
+        this.atomixIdGenerator = atomixIdGenerator;
+    }
+
+    @Override
+    public String name() {
+        return atomixIdGenerator.name();
+    }
+
+    @Override
+    public CompletableFuture<Long> nextId() {
+        return atomixIdGenerator.nextId();
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicIdGeneratorBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicIdGeneratorBuilder.java
new file mode 100644
index 0000000..2b7bd85
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicIdGeneratorBuilder.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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 io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.AsyncAtomicIdGenerator;
+import org.onosproject.store.service.AtomicIdGeneratorBuilder;
+
+/**
+ * Default {@link org.onosproject.store.service.AsyncAtomicIdGenerator} builder.
+ */
+public class AtomixAtomicIdGeneratorBuilder extends AtomicIdGeneratorBuilder {
+    private static final int MAX_RETRIES = 5;
+    private final Atomix atomix;
+    private final String group;
+
+    public AtomixAtomicIdGeneratorBuilder(Atomix atomix, String group) {
+        this.atomix = atomix;
+        this.group = group;
+    }
+
+    @Override
+    public AsyncAtomicIdGenerator build() {
+        return new AtomixAtomicIdGenerator(atomix.atomicIdGeneratorBuilder(name())
+            .withProtocol(MultiRaftProtocol.builder(group)
+                .withRecoveryStrategy(Recovery.RECOVER)
+                .withMaxRetries(MAX_RETRIES)
+                .build())
+            .withReadOnly(readOnly())
+            .withSerializer(new AtomixSerializerAdapter(serializer()))
+            .build()
+            .async());
+    }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicValue.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicValue.java
new file mode 100644
index 0000000..977b206
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicValue.java
@@ -0,0 +1,82 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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.Map;
+import java.util.concurrent.CompletableFuture;
+
+import com.google.common.collect.Maps;
+import org.onosproject.store.service.AsyncAtomicValue;
+import org.onosproject.store.service.AtomicValueEvent;
+import org.onosproject.store.service.AtomicValueEventListener;
+
+/**
+ * Atomix atomic value.
+ */
+public class AtomixAtomicValue<V> implements AsyncAtomicValue<V> {
+    private final io.atomix.core.value.AsyncAtomicValue<V> atomixValue;
+    private final Map<AtomicValueEventListener<V>, io.atomix.core.value.AtomicValueEventListener<V>> listenerMap =
+        Maps.newIdentityHashMap();
+
+    public AtomixAtomicValue(io.atomix.core.value.AsyncAtomicValue<V> atomixValue) {
+        this.atomixValue = atomixValue;
+    }
+
+    @Override
+    public String name() {
+        return atomixValue.name();
+    }
+
+    @Override
+    public CompletableFuture<Boolean> compareAndSet(V expect, V update) {
+        return atomixValue.compareAndSet(expect, update);
+    }
+
+    @Override
+    public CompletableFuture<V> get() {
+        return atomixValue.get();
+    }
+
+    @Override
+    public CompletableFuture<V> getAndSet(V value) {
+        return atomixValue.getAndSet(value);
+    }
+
+    @Override
+    public CompletableFuture<Void> set(V value) {
+        return atomixValue.set(value);
+    }
+
+    @Override
+    public synchronized CompletableFuture<Void> addListener(AtomicValueEventListener<V> listener) {
+        io.atomix.core.value.AtomicValueEventListener<V> atomixListener = event ->
+            listener.event(new AtomicValueEvent<V>(
+                name(),
+                event.newValue(),
+                event.oldValue()));
+        listenerMap.put(listener, atomixListener);
+        return atomixValue.addListener(atomixListener);
+    }
+
+    @Override
+    public synchronized CompletableFuture<Void> removeListener(AtomicValueEventListener<V> listener) {
+        io.atomix.core.value.AtomicValueEventListener<V> atomixListener = listenerMap.remove(listener);
+        if (atomixListener != null) {
+            return atomixValue.removeListener(atomixListener);
+        }
+        return CompletableFuture.completedFuture(null);
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicValueBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicValueBuilder.java
new file mode 100644
index 0000000..6b2e5df
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixAtomicValueBuilder.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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 io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.AsyncAtomicValue;
+import org.onosproject.store.service.AtomicValueBuilder;
+
+/**
+ * Default {@link org.onosproject.store.service.AsyncAtomicValue} builder.
+ *
+ * @param <V> type for set value
+ */
+public class AtomixAtomicValueBuilder<V> extends AtomicValueBuilder<V> {
+    private static final int MAX_RETRIES = 5;
+    private final Atomix atomix;
+    private final String group;
+
+    public AtomixAtomicValueBuilder(Atomix atomix, String group) {
+        this.atomix = atomix;
+        this.group = group;
+    }
+
+    @Override
+    public AsyncAtomicValue<V> build() {
+        return new AtomixAtomicValue<V>(atomix.<V>atomicValueBuilder(name())
+            .withProtocol(MultiRaftProtocol.builder(group)
+                .withRecoveryStrategy(Recovery.RECOVER)
+                .withMaxRetries(MAX_RETRIES)
+                .build())
+            .withReadOnly(readOnly())
+            .withSerializer(new AtomixSerializerAdapter(serializer()))
+            .build()
+            .async());
+    }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMap.java
new file mode 100644
index 0000000..d1a601d
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMap.java
@@ -0,0 +1,259 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.function.BiFunction;
+import java.util.function.Predicate;
+
+import com.google.common.base.Throwables;
+import com.google.common.collect.Maps;
+import io.atomix.core.collection.impl.TranscodingAsyncDistributedCollection;
+import io.atomix.core.set.impl.TranscodingAsyncDistributedSet;
+import org.onosproject.store.primitives.MapUpdate;
+import org.onosproject.store.primitives.TransactionId;
+import org.onosproject.store.service.AsyncConsistentMap;
+import org.onosproject.store.service.AsyncIterator;
+import org.onosproject.store.service.ConsistentMapException;
+import org.onosproject.store.service.MapEvent;
+import org.onosproject.store.service.MapEventListener;
+import org.onosproject.store.service.TransactionLog;
+import org.onosproject.store.service.Version;
+import org.onosproject.store.service.Versioned;
+
+/**
+ * Atomix consistent map.
+ */
+public class AtomixConsistentMap<K, V> implements AsyncConsistentMap<K, V> {
+    private final io.atomix.core.map.AsyncAtomicMap<K, V> atomixMap;
+    private final Map<MapEventListener<K, V>, io.atomix.core.map.AtomicMapEventListener<K, V>> listenerMap =
+        Maps.newIdentityHashMap();
+
+    public AtomixConsistentMap(io.atomix.core.map.AsyncAtomicMap<K, V> atomixMap) {
+        this.atomixMap = atomixMap;
+    }
+
+    @Override
+    public String name() {
+        return atomixMap.name();
+    }
+
+    @Override
+    public CompletableFuture<Integer> size() {
+        return atomixMap.size();
+    }
+
+    @Override
+    public CompletableFuture<Boolean> containsKey(K key) {
+        return atomixMap.containsKey(key);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> containsValue(V value) {
+        return atomixMap.containsValue(value);
+    }
+
+    @Override
+    public CompletableFuture<Versioned<V>> get(K key) {
+        return atomixMap.get(key).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Versioned<V>> getOrDefault(K key, V defaultValue) {
+        return atomixMap.getOrDefault(key, defaultValue).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Versioned<V>> computeIf(
+        K key, Predicate<? super V> condition, BiFunction<? super K, ? super V, ? extends V> remappingFunction) {
+        return adapt(atomixMap.computeIf(key, condition, remappingFunction).thenApply(this::toVersioned));
+    }
+
+    @Override
+    public CompletableFuture<Versioned<V>> put(K key, V value) {
+        return adapt(atomixMap.put(key, value).thenApply(this::toVersioned));
+    }
+
+    @Override
+    public CompletableFuture<Versioned<V>> putAndGet(K key, V value) {
+        return adapt(atomixMap.putAndGet(key, value).thenApply(this::toVersioned));
+    }
+
+    @Override
+    public CompletableFuture<Versioned<V>> remove(K key) {
+        return adapt(atomixMap.remove(key).thenApply(this::toVersioned));
+    }
+
+    @Override
+    public CompletableFuture<Void> clear() {
+        return atomixMap.clear();
+    }
+
+    @Override
+    public CompletableFuture<Set<K>> keySet() {
+        return CompletableFuture.completedFuture(atomixMap.keySet().sync());
+    }
+
+    @Override
+    public CompletableFuture<Collection<Versioned<V>>> values() {
+        return CompletableFuture.completedFuture(
+            new TranscodingAsyncDistributedCollection<Versioned<V>, io.atomix.utils.time.Versioned<V>>(
+                atomixMap.values(),
+                v -> new io.atomix.utils.time.Versioned<>(v.value(), v.version()),
+                v -> new Versioned<>(v.value(), v.version())).sync());
+    }
+
+    @Override
+    public CompletableFuture<Set<Map.Entry<K, Versioned<V>>>> entrySet() {
+        return CompletableFuture.completedFuture(
+            new TranscodingAsyncDistributedSet<Map.Entry<K, Versioned<V>>,
+                Map.Entry<K, io.atomix.utils.time.Versioned<V>>>(
+                atomixMap.entrySet(),
+                e -> Maps.immutableEntry(e.getKey(),
+                    new io.atomix.utils.time.Versioned<>(e.getValue().value(), e.getValue().version())),
+                e -> Maps.immutableEntry(e.getKey(), new Versioned<>(e.getValue().value(), e.getValue().version())))
+                .sync());
+    }
+
+    @Override
+    public CompletableFuture<Versioned<V>> putIfAbsent(K key, V value) {
+        return adapt(atomixMap.putIfAbsent(key, value).thenApply(this::toVersioned));
+    }
+
+    @Override
+    public CompletableFuture<Boolean> remove(K key, V value) {
+        return adapt(atomixMap.remove(key, value));
+    }
+
+    @Override
+    public CompletableFuture<Boolean> remove(K key, long version) {
+        return adapt(atomixMap.remove(key, version));
+    }
+
+    @Override
+    public CompletableFuture<Versioned<V>> replace(K key, V value) {
+        return adapt(atomixMap.replace(key, value).thenApply(this::toVersioned));
+    }
+
+    @Override
+    public CompletableFuture<Boolean> replace(K key, V oldValue, V newValue) {
+        return adapt(atomixMap.replace(key, oldValue, newValue));
+    }
+
+    @Override
+    public CompletableFuture<Boolean> replace(K key, long oldVersion, V newValue) {
+        return adapt(atomixMap.replace(key, oldVersion, newValue));
+    }
+
+    @Override
+    public CompletableFuture<AsyncIterator<Map.Entry<K, Versioned<V>>>> iterator() {
+        io.atomix.core.iterator.AsyncIterator<Map.Entry<K, io.atomix.utils.time.Versioned<V>>> atomixIterator
+            = atomixMap.entrySet().iterator();
+        return CompletableFuture.completedFuture(new AsyncIterator<Map.Entry<K, Versioned<V>>>() {
+            @Override
+            public CompletableFuture<Boolean> hasNext() {
+                return atomixIterator.hasNext();
+            }
+
+            @Override
+            public CompletableFuture<Map.Entry<K, Versioned<V>>> next() {
+                return atomixIterator.next()
+                    .thenApply(entry -> Maps.immutableEntry(entry.getKey(), toVersioned(entry.getValue())));
+            }
+        });
+    }
+
+    @Override
+    public synchronized CompletableFuture<Void> addListener(MapEventListener<K, V> listener, Executor executor) {
+        io.atomix.core.map.AtomicMapEventListener<K, V> atomixListener = event ->
+            listener.event(new MapEvent<K, V>(
+                MapEvent.Type.valueOf(event.type().name()),
+                name(),
+                event.key(),
+                toVersioned(event.newValue()),
+                toVersioned(event.oldValue())));
+        listenerMap.put(listener, atomixListener);
+        return atomixMap.addListener(atomixListener, executor);
+    }
+
+    @Override
+    public CompletableFuture<Void> removeListener(MapEventListener<K, V> listener) {
+        io.atomix.core.map.AtomicMapEventListener<K, V> atomixListener = listenerMap.remove(listener);
+        if (atomixListener != null) {
+            return atomixMap.removeListener(atomixListener);
+        }
+        return CompletableFuture.completedFuture(null);
+    }
+
+    @Override
+    public CompletableFuture<Version> begin(TransactionId transactionId) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public CompletableFuture<Boolean> prepare(TransactionLog<MapUpdate<K, V>> transactionLog) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<MapUpdate<K, V>> transactionLog) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public CompletableFuture<Void> commit(TransactionId transactionId) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public CompletableFuture<Void> rollback(TransactionId transactionId) {
+        throw new UnsupportedOperationException();
+    }
+
+    private <T> CompletableFuture<T> adapt(CompletableFuture<T> future) {
+        CompletableFuture<T> newFuture = new CompletableFuture<>();
+        future.whenComplete((result, error) -> {
+            if (error == null) {
+                newFuture.complete(result);
+            } else {
+                Throwable cause = Throwables.getRootCause(error);
+                if (cause instanceof io.atomix.primitive.PrimitiveException.ConcurrentModification) {
+                    newFuture.completeExceptionally(
+                        new ConsistentMapException.ConcurrentModification(cause.getMessage()));
+                } else if (cause instanceof io.atomix.primitive.PrimitiveException.Timeout) {
+                    newFuture.completeExceptionally(new ConsistentMapException.Timeout(cause.getMessage()));
+                } else if (cause instanceof io.atomix.primitive.PrimitiveException.Interrupted) {
+                    newFuture.completeExceptionally(new ConsistentMapException.Interrupted());
+                } else if (cause instanceof io.atomix.primitive.PrimitiveException.Unavailable) {
+                    newFuture.completeExceptionally(new ConsistentMapException.Unavailable());
+                } else if (cause instanceof io.atomix.primitive.PrimitiveException) {
+                    newFuture.completeExceptionally(new ConsistentMapException(cause.getMessage()));
+                }
+            }
+        });
+        return newFuture;
+    }
+
+    private Versioned<V> toVersioned(io.atomix.utils.time.Versioned<V> versioned) {
+        return versioned != null
+            ? new Versioned<>(versioned.value(), versioned.version(), versioned.creationTime())
+            : null;
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMapBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMapBuilder.java
new file mode 100644
index 0000000..0eadcde
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMapBuilder.java
@@ -0,0 +1,60 @@
+/*
+ * Copyright 2016-present Open Networking Foundation
+ *
+ * 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 io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+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 AtomixConsistentMapBuilder<K, V> extends ConsistentMapBuilder<K, V> {
+    private static final int MAX_RETRIES = 5;
+    private final Atomix atomix;
+    private final String group;
+
+    public AtomixConsistentMapBuilder(Atomix atomix, String group) {
+        this.atomix = atomix;
+        this.group = group;
+    }
+
+    @Override
+    public ConsistentMap<K, V> build() {
+        return buildAsyncMap().asConsistentMap();
+    }
+
+    @Override
+    public AsyncConsistentMap<K, V> buildAsyncMap() {
+        return new AtomixConsistentMap<>(atomix.<K, V>atomicMapBuilder(name())
+            .withRegistrationRequired()
+            .withProtocol(MultiRaftProtocol.builder(group)
+                .withRecoveryStrategy(Recovery.RECOVER)
+                .withMaxRetries(MAX_RETRIES)
+                .build())
+            .withReadOnly(readOnly())
+            .withCacheEnabled(relaxedReadConsistency())
+            .withSerializer(new AtomixSerializerAdapter(serializer()))
+            .build()
+            .async());
+    }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMultimap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMultimap.java
new file mode 100644
index 0000000..6627a81
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMultimap.java
@@ -0,0 +1,191 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.function.Function;
+
+import com.google.common.collect.Maps;
+import com.google.common.collect.Multiset;
+import io.atomix.core.map.impl.TranscodingAsyncDistributedMap;
+import org.onosproject.store.service.AsyncConsistentMultimap;
+import org.onosproject.store.service.AsyncIterator;
+import org.onosproject.store.service.MultimapEvent;
+import org.onosproject.store.service.MultimapEventListener;
+import org.onosproject.store.service.Versioned;
+
+/**
+ * Atomix consistent map.
+ */
+public class AtomixConsistentMultimap<K, V> implements AsyncConsistentMultimap<K, V> {
+    private final io.atomix.core.multimap.AsyncAtomicMultimap<K, V> atomixMultimap;
+    private final Map<MultimapEventListener<K, V>, io.atomix.core.multimap.AtomicMultimapEventListener<K, V>>
+        listenerMap = Maps.newIdentityHashMap();
+
+    public AtomixConsistentMultimap(io.atomix.core.multimap.AsyncAtomicMultimap<K, V> atomixMultimap) {
+        this.atomixMultimap = atomixMultimap;
+    }
+
+    @Override
+    public String name() {
+        return atomixMultimap.name();
+    }
+
+    @Override
+    public CompletableFuture<Integer> size() {
+        return atomixMultimap.size();
+    }
+
+    @Override
+    public CompletableFuture<Boolean> containsKey(K key) {
+        return atomixMultimap.containsKey(key);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> containsValue(V value) {
+        return atomixMultimap.containsValue(value);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> isEmpty() {
+        return atomixMultimap.isEmpty();
+    }
+
+    @Override
+    public CompletableFuture<Boolean> containsEntry(K key, V value) {
+        return atomixMultimap.containsEntry(key, value);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> put(K key, V value) {
+        return atomixMultimap.put(key, value);
+    }
+
+    @Override
+    public CompletableFuture<Versioned<Collection<? extends V>>> putAndGet(K key, V value) {
+        return atomixMultimap.put(key, value).thenCompose(v -> atomixMultimap.get(key)).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> remove(K key, V value) {
+        return atomixMultimap.remove(key, value);
+    }
+
+    @Override
+    public CompletableFuture<Versioned<Collection<? extends V>>> removeAndGet(K key, V value) {
+        return atomixMultimap.remove(key, value).thenCompose(v -> atomixMultimap.get(key)).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> removeAll(K key, Collection<? extends V> values) {
+        return atomixMultimap.removeAll(key, values);
+    }
+
+    @Override
+    public CompletableFuture<Versioned<Collection<? extends V>>> removeAll(K key) {
+        return atomixMultimap.removeAll(key).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> putAll(K key, Collection<? extends V> values) {
+        return atomixMultimap.putAll(key, values);
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public CompletableFuture<Versioned<Collection<? extends V>>> replaceValues(K key, Collection<V> values) {
+        return atomixMultimap.replaceValues(key, values).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Void> clear() {
+        return atomixMultimap.clear();
+    }
+
+    @Override
+    @SuppressWarnings("unchecked")
+    public CompletableFuture<Versioned<Collection<? extends V>>> get(K key) {
+        return atomixMultimap.get(key).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Set<K>> keySet() {
+        return CompletableFuture.completedFuture(atomixMultimap.keySet().sync());
+    }
+
+    @Override
+    public CompletableFuture<Multiset<K>> keys() {
+        return CompletableFuture.completedFuture(atomixMultimap.keys().sync());
+    }
+
+    @Override
+    public CompletableFuture<Multiset<V>> values() {
+        return CompletableFuture.completedFuture(atomixMultimap.values().sync());
+    }
+
+    @Override
+    public CompletableFuture<Collection<Map.Entry<K, V>>> entries() {
+        return CompletableFuture.completedFuture(atomixMultimap.entries().sync());
+    }
+
+    @Override
+    public CompletableFuture<AsyncIterator<Map.Entry<K, V>>> iterator() {
+        return CompletableFuture.completedFuture(new AtomixIterator<>(atomixMultimap.entries().iterator()));
+    }
+
+    @Override
+    public CompletableFuture<Map<K, Collection<V>>> asMap() {
+        return CompletableFuture.completedFuture(
+            new TranscodingAsyncDistributedMap<K, Collection<V>, K, io.atomix.utils.time.Versioned<Collection<V>>>(
+                atomixMultimap.asMap(),
+                Function.identity(),
+                Function.identity(),
+                v -> new io.atomix.utils.time.Versioned<>(v, 0),
+                v -> v.value()).sync());
+    }
+
+    @Override
+    public synchronized CompletableFuture<Void> addListener(MultimapEventListener<K, V> listener, Executor executor) {
+        io.atomix.core.multimap.AtomicMultimapEventListener<K, V> atomixListener = event ->
+            listener.event(new MultimapEvent<K, V>(
+                name(),
+                event.key(),
+                event.newValue(),
+                event.oldValue()));
+        listenerMap.put(listener, atomixListener);
+        return atomixMultimap.addListener(atomixListener, executor);
+    }
+
+    @Override
+    public CompletableFuture<Void> removeListener(MultimapEventListener<K, V> listener) {
+        io.atomix.core.multimap.AtomicMultimapEventListener<K, V> atomixListener = listenerMap.remove(listener);
+        if (atomixListener != null) {
+            return atomixMultimap.removeListener(atomixListener);
+        }
+        return CompletableFuture.completedFuture(null);
+    }
+
+    private Versioned<Collection<? extends V>> toVersioned(
+        io.atomix.utils.time.Versioned<Collection<V>> versioned) {
+        return versioned != null
+            ? new Versioned<>(versioned.value(), versioned.version(), versioned.creationTime())
+            : null;
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMultimapBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMultimapBuilder.java
new file mode 100644
index 0000000..88f6bd2
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentMultimapBuilder.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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 io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.AsyncConsistentMultimap;
+import org.onosproject.store.service.ConsistentMultimap;
+import org.onosproject.store.service.ConsistentMultimapBuilder;
+
+/**
+ * Default {@link AsyncConsistentMultimap} builder.
+ *
+ * @param <K> type for multimap key
+ * @param <V> type for multimap value
+ */
+public class AtomixConsistentMultimapBuilder<K, V> extends ConsistentMultimapBuilder<K, V> {
+    private static final int MAX_RETRIES = 5;
+    private final Atomix atomix;
+    private final String group;
+
+    public AtomixConsistentMultimapBuilder(Atomix atomix, String group) {
+        this.atomix = atomix;
+        this.group = group;
+    }
+
+    @Override
+    public AsyncConsistentMultimap<K, V> buildMultimap() {
+        return new AtomixConsistentMultimap<>(atomix.<K, V>atomicMultimapBuilder(name())
+            .withProtocol(MultiRaftProtocol.builder(group)
+                .withRecoveryStrategy(Recovery.RECOVER)
+                .withMaxRetries(MAX_RETRIES)
+                .build())
+            .withReadOnly(readOnly())
+            .withCacheEnabled(relaxedReadConsistency())
+            .withSerializer(new AtomixSerializerAdapter(serializer()))
+            .build()
+            .async());
+    }
+
+    @Override
+    public ConsistentMultimap<K, V> build() {
+        return buildMultimap().asMultimap();
+    }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentTreeMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentTreeMap.java
new file mode 100644
index 0000000..a7bedbb
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentTreeMap.java
@@ -0,0 +1,345 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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.Collection;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.NavigableSet;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.function.BiFunction;
+import java.util.function.Predicate;
+import java.util.stream.Collectors;
+
+import com.google.common.base.Throwables;
+import com.google.common.collect.Maps;
+import io.atomix.core.collection.impl.TranscodingAsyncDistributedCollection;
+import io.atomix.core.map.impl.DelegatingAsyncDistributedNavigableMap;
+import org.onosproject.store.primitives.MapUpdate;
+import org.onosproject.store.primitives.TransactionId;
+import org.onosproject.store.service.AsyncConsistentTreeMap;
+import org.onosproject.store.service.AsyncIterator;
+import org.onosproject.store.service.ConsistentMapException;
+import org.onosproject.store.service.MapEvent;
+import org.onosproject.store.service.MapEventListener;
+import org.onosproject.store.service.TransactionLog;
+import org.onosproject.store.service.Version;
+import org.onosproject.store.service.Versioned;
+
+/**
+ * Atomix consistent tree map.
+ */
+public class AtomixConsistentTreeMap<V> implements AsyncConsistentTreeMap<V> {
+    private final io.atomix.core.map.AsyncAtomicNavigableMap<String, V> atomixTreeMap;
+    private final Map<MapEventListener<String, V>, io.atomix.core.map.AtomicMapEventListener<String, V>> listenerMap =
+        Maps.newIdentityHashMap();
+
+    public AtomixConsistentTreeMap(io.atomix.core.map.AsyncAtomicNavigableMap<String, V> atomixTreeMap) {
+        this.atomixTreeMap = atomixTreeMap;
+    }
+
+    @Override
+    public String name() {
+        return atomixTreeMap.name();
+    }
+
+    @Override
+    public CompletableFuture<Integer> size() {
+        return atomixTreeMap.size();
+    }
+
+    @Override
+    public CompletableFuture<String> firstKey() {
+        return atomixTreeMap.firstKey();
+    }
+
+    @Override
+    public CompletableFuture<String> lastKey() {
+        return atomixTreeMap.lastKey();
+    }
+
+    @Override
+    public CompletableFuture<Map.Entry<String, Versioned<V>>> ceilingEntry(String key) {
+        return atomixTreeMap.ceilingEntry(key).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Map.Entry<String, Versioned<V>>> floorEntry(String key) {
+        return atomixTreeMap.floorEntry(key).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Map.Entry<String, Versioned<V>>> higherEntry(String key) {
+        return atomixTreeMap.higherEntry(key).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Map.Entry<String, Versioned<V>>> lowerEntry(String key) {
+        return atomixTreeMap.lowerEntry(key).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Map.Entry<String, Versioned<V>>> firstEntry() {
+        return atomixTreeMap.firstEntry().thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Map.Entry<String, Versioned<V>>> lastEntry() {
+        return atomixTreeMap.lastEntry().thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Map.Entry<String, Versioned<V>>> pollFirstEntry() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public CompletableFuture<Map.Entry<String, Versioned<V>>> pollLastEntry() {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public CompletableFuture<Boolean> containsKey(String key) {
+        return atomixTreeMap.containsKey(key);
+    }
+
+    @Override
+    public CompletableFuture<String> lowerKey(String key) {
+        return atomixTreeMap.lowerKey(key);
+    }
+
+    @Override
+    public CompletableFuture<String> floorKey(String key) {
+        return atomixTreeMap.floorKey(key);
+    }
+
+    @Override
+    public CompletableFuture<String> ceilingKey(String key) {
+        return atomixTreeMap.ceilingKey(key);
+    }
+
+    @Override
+    public CompletableFuture<Versioned<V>> get(String key) {
+        return atomixTreeMap.get(key).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<String> higherKey(String key) {
+        return atomixTreeMap.higherKey(key);
+    }
+
+    @Override
+    public CompletableFuture<NavigableSet<String>> navigableKeySet() {
+        return CompletableFuture.completedFuture(atomixTreeMap.navigableKeySet().sync());
+    }
+
+    @Override
+    public CompletableFuture<Versioned<V>> getOrDefault(String key, V defaultValue) {
+        return atomixTreeMap.getOrDefault(key, defaultValue).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<NavigableMap<String, V>> subMap(
+        String upperKey, String lowerKey, boolean inclusiveUpper, boolean inclusiveLower) {
+        return CompletableFuture.completedFuture(
+            new DelegatingAsyncDistributedNavigableMap<>(
+                atomixTreeMap.subMap(lowerKey, inclusiveLower, upperKey, inclusiveUpper)).sync());
+    }
+
+    @Override
+    public CompletableFuture<Versioned<V>> computeIf(
+        String key,
+        Predicate<? super V> condition, BiFunction<? super String, ? super V, ? extends V> remappingFunction) {
+        return adapt(atomixTreeMap.computeIf(key, condition, remappingFunction)).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Versioned<V>> put(String key, V value) {
+        return adapt(atomixTreeMap.put(key, value)).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Versioned<V>> putAndGet(String key, V value) {
+        return adapt(atomixTreeMap.putAndGet(key, value)).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Versioned<V>> remove(String key) {
+        return adapt(atomixTreeMap.remove(key)).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Set<String>> keySet() {
+        return CompletableFuture.completedFuture(atomixTreeMap.keySet().sync());
+    }
+
+    @Override
+    public CompletableFuture<Set<Map.Entry<String, Versioned<V>>>> entrySet() {
+        return CompletableFuture.completedFuture(atomixTreeMap.entrySet().stream()
+            .map(this::toVersioned)
+            .collect(Collectors.toSet()));
+    }
+
+    @Override
+    public CompletableFuture<Versioned<V>> putIfAbsent(String key, V value) {
+        return adapt(atomixTreeMap.putIfAbsent(key, value)).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> remove(String key, V value) {
+        return adapt(atomixTreeMap.remove(key, value));
+    }
+
+    @Override
+    public CompletableFuture<Boolean> remove(String key, long version) {
+        return adapt(atomixTreeMap.remove(key, version));
+    }
+
+    @Override
+    public CompletableFuture<Versioned<V>> replace(String key, V value) {
+        return adapt(atomixTreeMap.replace(key, value)).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> replace(String key, V oldValue, V newValue) {
+        return adapt(atomixTreeMap.replace(key, oldValue, newValue));
+    }
+
+    @Override
+    public CompletableFuture<Boolean> replace(String key, long oldVersion, V newValue) {
+        return adapt(atomixTreeMap.replace(key, oldVersion, newValue));
+    }
+
+    @Override
+    public CompletableFuture<Boolean> containsValue(V value) {
+        return atomixTreeMap.containsValue(value);
+    }
+
+    @Override
+    public CompletableFuture<AsyncIterator<Map.Entry<String, Versioned<V>>>> iterator() {
+        io.atomix.core.iterator.AsyncIterator<Map.Entry<String, io.atomix.utils.time.Versioned<V>>> atomixIterator
+            = atomixTreeMap.entrySet().iterator();
+        return CompletableFuture.completedFuture(new AsyncIterator<Map.Entry<String, Versioned<V>>>() {
+            @Override
+            public CompletableFuture<Boolean> hasNext() {
+                return atomixIterator.hasNext();
+            }
+
+            @Override
+            public CompletableFuture<Map.Entry<String, Versioned<V>>> next() {
+                return atomixIterator.next()
+                    .thenApply(entry -> Maps.immutableEntry(entry.getKey(), toVersioned(entry.getValue())));
+            }
+        });
+    }
+
+    @Override
+    public CompletableFuture<Void> clear() {
+        return atomixTreeMap.clear();
+    }
+
+    @Override
+    public CompletableFuture<Collection<Versioned<V>>> values() {
+        return CompletableFuture.completedFuture(
+            new TranscodingAsyncDistributedCollection<Versioned<V>, io.atomix.utils.time.Versioned<V>>(
+                atomixTreeMap.values(),
+                e -> new io.atomix.utils.time.Versioned<>(e.value(), e.version()),
+                e -> new Versioned<>(e.value(), e.version())).sync());
+    }
+
+    @Override
+    public synchronized CompletableFuture<Void> addListener(MapEventListener<String, V> listener, Executor executor) {
+        io.atomix.core.map.AtomicMapEventListener<String, V> atomixListener = event ->
+            listener.event(new MapEvent<String, V>(
+                MapEvent.Type.valueOf(event.type().name()),
+                name(),
+                event.key(),
+                toVersioned(event.newValue()),
+                toVersioned(event.oldValue())));
+        listenerMap.put(listener, atomixListener);
+        return atomixTreeMap.addListener(atomixListener, executor);
+    }
+
+    @Override
+    public CompletableFuture<Void> removeListener(MapEventListener<String, V> listener) {
+        io.atomix.core.map.AtomicMapEventListener<String, V> atomixListener = listenerMap.remove(listener);
+        if (atomixListener != null) {
+            return atomixTreeMap.removeListener(atomixListener);
+        }
+        return CompletableFuture.completedFuture(null);
+    }
+
+    @Override
+    public CompletableFuture<Version> begin(TransactionId transactionId) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public CompletableFuture<Boolean> prepare(TransactionLog<MapUpdate<String, V>> transactionLog) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<MapUpdate<String, V>> transactionLog) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public CompletableFuture<Void> commit(TransactionId transactionId) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public CompletableFuture<Void> rollback(TransactionId transactionId) {
+        throw new UnsupportedOperationException();
+    }
+
+    private <T> CompletableFuture<T> adapt(CompletableFuture<T> future) {
+        CompletableFuture<T> newFuture = new CompletableFuture<>();
+        future.whenComplete((result, error) -> {
+            if (error == null) {
+                newFuture.complete(result);
+            } else {
+                Throwable cause = Throwables.getRootCause(error);
+                if (cause instanceof io.atomix.primitive.PrimitiveException.ConcurrentModification) {
+                    newFuture.completeExceptionally(
+                        new ConsistentMapException.ConcurrentModification(error.getMessage()));
+                } else if (cause instanceof io.atomix.primitive.PrimitiveException.Timeout) {
+                    newFuture.completeExceptionally(new ConsistentMapException.Timeout(error.getMessage()));
+                } else if (cause instanceof io.atomix.primitive.PrimitiveException.Interrupted) {
+                    newFuture.completeExceptionally(new ConsistentMapException.Interrupted());
+                } else if (cause instanceof io.atomix.primitive.PrimitiveException.Unavailable) {
+                    newFuture.completeExceptionally(new ConsistentMapException.Unavailable());
+                } else if (cause instanceof io.atomix.primitive.PrimitiveException) {
+                    newFuture.completeExceptionally(new ConsistentMapException(cause.getMessage()));
+                }
+            }
+        });
+        return newFuture;
+    }
+
+    private Versioned<V> toVersioned(io.atomix.utils.time.Versioned<V> versioned) {
+        return versioned != null
+            ? new Versioned<>(versioned.value(), versioned.version(), versioned.creationTime())
+            : null;
+    }
+
+    private Map.Entry<String, Versioned<V>> toVersioned(Map.Entry<String, io.atomix.utils.time.Versioned<V>> entry) {
+        return entry != null ? Maps.immutableEntry(entry.getKey(), toVersioned(entry.getValue())) : null;
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentTreeMapBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentTreeMapBuilder.java
new file mode 100644
index 0000000..14178fd
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixConsistentTreeMapBuilder.java
@@ -0,0 +1,59 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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 io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.AsyncConsistentTreeMap;
+import org.onosproject.store.service.ConsistentTreeMap;
+import org.onosproject.store.service.ConsistentTreeMapBuilder;
+
+/**
+ * Default {@link org.onosproject.store.service.AsyncConsistentTreeMap} builder.
+ *
+ * @param <V> type for set value
+ */
+public class AtomixConsistentTreeMapBuilder<V> extends ConsistentTreeMapBuilder<V> {
+    private static final int MAX_RETRIES = 5;
+    private final Atomix atomix;
+    private final String group;
+
+    public AtomixConsistentTreeMapBuilder(Atomix atomix, String group) {
+        this.atomix = atomix;
+        this.group = group;
+    }
+
+    @Override
+    public ConsistentTreeMap<V> build() {
+        return buildTreeMap().asTreeMap();
+    }
+
+    @Override
+    public AsyncConsistentTreeMap<V> buildTreeMap() {
+        return new AtomixConsistentTreeMap<>(atomix.<String, V>atomicNavigableMapBuilder(name())
+            .withRegistrationRequired()
+            .withProtocol(MultiRaftProtocol.builder(group)
+                .withRecoveryStrategy(Recovery.RECOVER)
+                .withMaxRetries(MAX_RETRIES)
+                .build())
+            .withReadOnly(readOnly())
+            .withCacheEnabled(relaxedReadConsistency())
+            .withSerializer(new AtomixSerializerAdapter(serializer()))
+            .build()
+            .async());
+    }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedLock.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedLock.java
new file mode 100644
index 0000000..ef2f502
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedLock.java
@@ -0,0 +1,63 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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.time.Duration;
+import java.util.Optional;
+import java.util.concurrent.CompletableFuture;
+
+import org.onosproject.store.service.AsyncDistributedLock;
+import org.onosproject.store.service.Version;
+
+/**
+ * Atomix distributed lock.
+ */
+public class AtomixDistributedLock implements AsyncDistributedLock {
+    private final io.atomix.core.lock.AsyncAtomicLock atomixLock;
+
+    public AtomixDistributedLock(io.atomix.core.lock.AsyncAtomicLock atomixLock) {
+        this.atomixLock = atomixLock;
+    }
+
+    @Override
+    public String name() {
+        return atomixLock.name();
+    }
+
+    @Override
+    public CompletableFuture<Version> lock() {
+        return atomixLock.lock().thenApply(this::toVersion);
+    }
+
+    @Override
+    public CompletableFuture<Optional<Version>> tryLock() {
+        return atomixLock.tryLock().thenApply(optional -> optional.map(this::toVersion));
+    }
+
+    @Override
+    public CompletableFuture<Optional<Version>> tryLock(Duration timeout) {
+        return atomixLock.tryLock(timeout).thenApply(optional -> optional.map(this::toVersion));
+    }
+
+    @Override
+    public CompletableFuture<Void> unlock() {
+        return atomixLock.unlock();
+    }
+
+    private Version toVersion(io.atomix.utils.time.Version version) {
+        return new Version(version.value());
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedLockBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedLockBuilder.java
new file mode 100644
index 0000000..e16a610
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedLockBuilder.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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 io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.AsyncDistributedLock;
+import org.onosproject.store.service.DistributedLockBuilder;
+
+/**
+ * Default {@link org.onosproject.store.service.AsyncDistributedLock} builder.
+ */
+public class AtomixDistributedLockBuilder extends DistributedLockBuilder {
+    private static final int MAX_RETRIES = 5;
+    private final Atomix atomix;
+    private final String group;
+
+    public AtomixDistributedLockBuilder(Atomix atomix, String group) {
+        this.atomix = atomix;
+        this.group = group;
+    }
+
+    @Override
+    public AsyncDistributedLock build() {
+        return new AtomixDistributedLock(atomix.atomicLockBuilder(name())
+            .withProtocol(MultiRaftProtocol.builder(group)
+                .withRecoveryStrategy(Recovery.RECOVER)
+                .withMaxRetries(MAX_RETRIES)
+                .build())
+            .withReadOnly(readOnly())
+            .withSerializer(new AtomixSerializerAdapter(serializer()))
+            .build()
+            .async());
+    }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedSet.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedSet.java
new file mode 100644
index 0000000..e1aadb3
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedSet.java
@@ -0,0 +1,120 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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.Collection;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Maps;
+import org.onosproject.store.service.AsyncDistributedSet;
+import org.onosproject.store.service.SetEvent;
+import org.onosproject.store.service.SetEventListener;
+
+/**
+ * Atomix distributed set.
+ */
+public class AtomixDistributedSet<E> implements AsyncDistributedSet<E> {
+    private final io.atomix.core.set.AsyncDistributedSet<E> atomixSet;
+    private final Map<SetEventListener<E>, io.atomix.core.collection.CollectionEventListener<E>> listenerMap =
+        Maps.newIdentityHashMap();
+
+    public AtomixDistributedSet(io.atomix.core.set.AsyncDistributedSet<E> atomixSet) {
+        this.atomixSet = atomixSet;
+    }
+
+    @Override
+    public String name() {
+        return atomixSet.name();
+    }
+
+    @Override
+    public CompletableFuture<Integer> size() {
+        return atomixSet.size();
+    }
+
+    @Override
+    public CompletableFuture<Boolean> add(E element) {
+        return atomixSet.add(element);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> remove(E element) {
+        return atomixSet.remove(element);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> isEmpty() {
+        return atomixSet.isEmpty();
+    }
+
+    @Override
+    public CompletableFuture<Void> clear() {
+        return atomixSet.clear();
+    }
+
+    @Override
+    public CompletableFuture<Boolean> contains(E element) {
+        return atomixSet.contains(element);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> addAll(Collection<? extends E> c) {
+        return atomixSet.addAll(c);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> containsAll(Collection<? extends E> c) {
+        return atomixSet.containsAll(c);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> retainAll(Collection<? extends E> c) {
+        return atomixSet.retainAll(c);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> removeAll(Collection<? extends E> c) {
+        return atomixSet.removeAll(c);
+    }
+
+    @Override
+    public CompletableFuture<? extends Set<E>> getAsImmutableSet() {
+        return CompletableFuture.completedFuture(atomixSet.stream().collect(Collectors.toSet()));
+    }
+
+    @Override
+    public synchronized CompletableFuture<Void> addListener(SetEventListener<E> listener) {
+        io.atomix.core.collection.CollectionEventListener<E> atomixListener = event ->
+            listener.event(new SetEvent<E>(
+                name(),
+                SetEvent.Type.valueOf(event.type().name()),
+                event.element()));
+        listenerMap.put(listener, atomixListener);
+        return atomixSet.addListener(atomixListener);
+    }
+
+    @Override
+    public CompletableFuture<Void> removeListener(SetEventListener<E> listener) {
+        io.atomix.core.collection.CollectionEventListener<E> atomixListener = listenerMap.remove(listener);
+        if (atomixListener != null) {
+            return atomixSet.removeListener(atomixListener);
+        }
+        return CompletableFuture.completedFuture(null);
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedSetBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedSetBuilder.java
new file mode 100644
index 0000000..73c6f9d
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedSetBuilder.java
@@ -0,0 +1,53 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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 io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.AsyncDistributedSet;
+import org.onosproject.store.service.DistributedSetBuilder;
+
+/**
+ * Default {@link org.onosproject.store.service.AsyncDistributedSet} builder.
+ *
+ * @param <E> type for set value
+ */
+public class AtomixDistributedSetBuilder<E> extends DistributedSetBuilder<E> {
+    private static final int MAX_RETRIES = 5;
+    private final Atomix atomix;
+    private final String group;
+
+    public AtomixDistributedSetBuilder(Atomix atomix, String group) {
+        this.atomix = atomix;
+        this.group = group;
+    }
+
+    @Override
+    public AsyncDistributedSet<E> build() {
+        return new AtomixDistributedSet<E>(atomix.<E>setBuilder(name())
+            .withRegistrationRequired()
+            .withProtocol(MultiRaftProtocol.builder(group)
+                .withRecoveryStrategy(Recovery.RECOVER)
+                .withMaxRetries(MAX_RETRIES)
+                .build())
+            .withReadOnly(readOnly())
+            // TODO: Enable caching for DistributedSet in Atomix
+            .withSerializer(new AtomixSerializerAdapter(serializer()))
+            .build()
+            .async());
+    }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedTopic.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedTopic.java
similarity index 78%
rename from core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedTopic.java
rename to core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedTopic.java
index 7581c37..01b7da8 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedTopic.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedTopic.java
@@ -20,30 +20,29 @@
 import java.util.concurrent.Executor;
 import java.util.function.Consumer;
 
-import org.onosproject.store.service.AsyncAtomicValue;
-import org.onosproject.store.service.AtomicValueEventListener;
+import com.google.common.collect.Maps;
+import io.atomix.core.value.AsyncAtomicValue;
+import io.atomix.core.value.AtomicValueEventListener;
 import org.onosproject.store.service.DistributedPrimitive;
 import org.onosproject.store.service.Topic;
 
-import com.google.common.collect.Maps;
-
 /**
  * Default implementation of {@link Topic}.
  *
  * @param <T> topic message type.
  */
-public class DefaultDistributedTopic<T> implements Topic<T> {
+public class AtomixDistributedTopic<T> implements Topic<T> {
 
-    private final AsyncAtomicValue<T> atomicValue;
+    private final AsyncAtomicValue<T> atomixValue;
     private final Map<Consumer<T>, AtomicValueEventListener<T>> callbacks = Maps.newIdentityHashMap();
 
-    DefaultDistributedTopic(AsyncAtomicValue<T> atomicValue) {
-        this.atomicValue = atomicValue;
+    AtomixDistributedTopic(AsyncAtomicValue<T> atomixValue) {
+        this.atomixValue = atomixValue;
     }
 
     @Override
     public String name() {
-        return atomicValue.name();
+        return atomixValue.name();
     }
 
     @Override
@@ -52,13 +51,8 @@
     }
 
     @Override
-    public CompletableFuture<Void> destroy() {
-        return atomicValue.destroy();
-    }
-
-    @Override
     public CompletableFuture<Void> publish(T message) {
-        return atomicValue.set(message);
+        return atomixValue.set(message);
     }
 
     @Override
@@ -66,7 +60,7 @@
         AtomicValueEventListener<T> valueListener =
                 event -> executor.execute(() -> callback.accept(event.newValue()));
         if (callbacks.putIfAbsent(callback, valueListener) == null) {
-            return atomicValue.addListener(valueListener);
+            return atomixValue.addListener(valueListener);
         }
         return CompletableFuture.completedFuture(null);
     }
@@ -75,8 +69,13 @@
     public CompletableFuture<Void> unsubscribe(Consumer<T> callback) {
         AtomicValueEventListener<T> valueListener = callbacks.remove(callback);
         if (valueListener != null) {
-            return atomicValue.removeListener(valueListener);
+            return atomixValue.removeListener(valueListener);
         }
         return CompletableFuture.completedFuture(null);
     }
+
+    @Override
+    public CompletableFuture<Void> destroy() {
+        return atomixValue.close();
+    }
 }
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedTopicBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedTopicBuilder.java
new file mode 100644
index 0000000..34dda44
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDistributedTopicBuilder.java
@@ -0,0 +1,49 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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 io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.Topic;
+import org.onosproject.store.service.TopicBuilder;
+
+/**
+ * Default topic builder.
+ */
+public class AtomixDistributedTopicBuilder<T> extends TopicBuilder<T> {
+    private static final int MAX_RETRIES = 5;
+    private final Atomix atomix;
+    private final String group;
+
+    public AtomixDistributedTopicBuilder(Atomix atomix, String group) {
+        this.atomix = atomix;
+        this.group = group;
+    }
+
+    @Override
+    public Topic<T> build() {
+        return new AtomixDistributedTopic<>(atomix.<T>atomicValueBuilder(name())
+            .withProtocol(MultiRaftProtocol.builder(group)
+                .withRecoveryStrategy(Recovery.RECOVER)
+                .withMaxRetries(MAX_RETRIES)
+                .build())
+            .withReadOnly(readOnly())
+            .withSerializer(new AtomixSerializerAdapter(serializer()))
+            .build()
+            .async());
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDocumentTree.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDocumentTree.java
new file mode 100644
index 0000000..4bbc822
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDocumentTree.java
@@ -0,0 +1,157 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Maps;
+import org.onosproject.store.primitives.NodeUpdate;
+import org.onosproject.store.primitives.TransactionId;
+import org.onosproject.store.service.AsyncDocumentTree;
+import org.onosproject.store.service.DocumentPath;
+import org.onosproject.store.service.DocumentTreeEvent;
+import org.onosproject.store.service.DocumentTreeListener;
+import org.onosproject.store.service.TransactionLog;
+import org.onosproject.store.service.Version;
+import org.onosproject.store.service.Versioned;
+
+/**
+ * Atomix document tree.
+ */
+public class AtomixDocumentTree<V> implements AsyncDocumentTree<V> {
+    private final io.atomix.core.tree.AsyncAtomicDocumentTree<V> atomixTree;
+    private final Map<DocumentTreeListener<V>, io.atomix.core.tree.DocumentTreeEventListener<V>> listenerMap =
+        Maps.newIdentityHashMap();
+
+    public AtomixDocumentTree(io.atomix.core.tree.AsyncAtomicDocumentTree<V> atomixTree) {
+        this.atomixTree = atomixTree;
+    }
+
+    @Override
+    public String name() {
+        return atomixTree.name();
+    }
+
+    @Override
+    public DocumentPath root() {
+        return toOnosPath(atomixTree.root());
+    }
+
+    @Override
+    public CompletableFuture<Map<String, Versioned<V>>> getChildren(DocumentPath path) {
+        return atomixTree.getChildren(toAtomixPath(path))
+            .thenApply(map -> map.entrySet().stream()
+                .collect(Collectors.toMap(e -> e.getKey(),
+                    e -> toVersioned(e.getValue()))));
+    }
+
+    @Override
+    public CompletableFuture<Versioned<V>> get(DocumentPath path) {
+        return atomixTree.get(toAtomixPath(path)).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Versioned<V>> set(DocumentPath path, V value) {
+        return atomixTree.set(toAtomixPath(path), value).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> create(DocumentPath path, V value) {
+        return atomixTree.create(toAtomixPath(path), value);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> createRecursive(DocumentPath path, V value) {
+        return atomixTree.createRecursive(toAtomixPath(path), value);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> replace(DocumentPath path, V newValue, long version) {
+        return atomixTree.replace(toAtomixPath(path), newValue, version);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> replace(DocumentPath path, V newValue, V currentValue) {
+        return atomixTree.replace(toAtomixPath(path), newValue, currentValue);
+    }
+
+    @Override
+    public CompletableFuture<Versioned<V>> removeNode(DocumentPath path) {
+        return atomixTree.removeNode(toAtomixPath(path)).thenApply(this::toVersioned);
+    }
+
+    @Override
+    public synchronized CompletableFuture<Void> addListener(DocumentPath path, DocumentTreeListener<V> listener) {
+        io.atomix.core.tree.DocumentTreeEventListener<V> atomixListener = event ->
+            listener.event(new DocumentTreeEvent<V>(
+                DocumentPath.from(event.path().pathElements()),
+                DocumentTreeEvent.Type.valueOf(event.type().name()),
+                event.newValue().map(this::toVersioned),
+                event.oldValue().map(this::toVersioned)));
+        listenerMap.put(listener, atomixListener);
+        return atomixTree.addListener(toAtomixPath(path), atomixListener);
+    }
+
+    @Override
+    public CompletableFuture<Void> removeListener(DocumentTreeListener<V> listener) {
+        io.atomix.core.tree.DocumentTreeEventListener<V> atomixListener = listenerMap.remove(listener);
+        if (atomixListener != null) {
+            return atomixTree.removeListener(atomixListener);
+        }
+        return CompletableFuture.completedFuture(null);
+    }
+
+    @Override
+    public CompletableFuture<Version> begin(TransactionId transactionId) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public CompletableFuture<Boolean> prepare(TransactionLog<NodeUpdate<V>> transactionLog) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<NodeUpdate<V>> transactionLog) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public CompletableFuture<Void> commit(TransactionId transactionId) {
+        throw new UnsupportedOperationException();
+    }
+
+    @Override
+    public CompletableFuture<Void> rollback(TransactionId transactionId) {
+        throw new UnsupportedOperationException();
+    }
+
+    private DocumentPath toOnosPath(io.atomix.core.tree.DocumentPath path) {
+        return DocumentPath.from(path.pathElements());
+    }
+
+    private io.atomix.core.tree.DocumentPath toAtomixPath(DocumentPath path) {
+        return io.atomix.core.tree.DocumentPath.from(path.pathElements());
+    }
+
+    private Versioned<V> toVersioned(io.atomix.utils.time.Versioned<V> versioned) {
+        return versioned != null
+            ? new Versioned<>(versioned.value(), versioned.version(), versioned.creationTime())
+            : null;
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDocumentTreeBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDocumentTreeBuilder.java
new file mode 100644
index 0000000..91dea85
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixDocumentTreeBuilder.java
@@ -0,0 +1,57 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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 io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.AsyncDocumentTree;
+import org.onosproject.store.service.DocumentTreeBuilder;
+
+/**
+ * Default {@link org.onosproject.store.service.AsyncDocumentTree} builder.
+ *
+ * @param <V> type for tree value
+ */
+public class AtomixDocumentTreeBuilder<V> extends DocumentTreeBuilder<V> {
+    private static final int MAX_RETRIES = 5;
+    private final Atomix atomix;
+    private final String group;
+
+    public AtomixDocumentTreeBuilder(Atomix atomix, String group) {
+        this.atomix = atomix;
+        this.group = group;
+    }
+
+    @Override
+    public AsyncDocumentTree<V> build() {
+        return buildDocumentTree();
+    }
+
+    @Override
+    public AsyncDocumentTree<V> buildDocumentTree() {
+        return new AtomixDocumentTree<>(atomix.<V>atomicDocumentTreeBuilder(name())
+            .withProtocol(MultiRaftProtocol.builder(group)
+                .withRecoveryStrategy(Recovery.RECOVER)
+                .withMaxRetries(MAX_RETRIES)
+                .build())
+            .withReadOnly(readOnly())
+            // TODO: Enable caching for DocumentTree in Atomix
+            .withSerializer(new AtomixSerializerAdapter(serializer()))
+            .build()
+            .async());
+    }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixIterator.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixIterator.java
new file mode 100644
index 0000000..c3288d3
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixIterator.java
@@ -0,0 +1,41 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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.concurrent.CompletableFuture;
+
+import org.onosproject.store.service.AsyncIterator;
+
+/**
+ * Atomix asynchronous iterator.
+ */
+public class AtomixIterator<T> implements AsyncIterator<T> {
+    private final io.atomix.core.iterator.AsyncIterator<T> atomixIterator;
+
+    public AtomixIterator(io.atomix.core.iterator.AsyncIterator<T> atomixIterator) {
+        this.atomixIterator = atomixIterator;
+    }
+
+    @Override
+    public CompletableFuture<Boolean> hasNext() {
+        return atomixIterator.hasNext();
+    }
+
+    @Override
+    public CompletableFuture<T> next() {
+        return atomixIterator.next();
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixLeaderElector.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixLeaderElector.java
new file mode 100644
index 0000000..1098981
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixLeaderElector.java
@@ -0,0 +1,115 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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.Map;
+import java.util.concurrent.CompletableFuture;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import com.google.common.collect.Maps;
+import io.atomix.core.election.LeadershipEventListener;
+import org.onosproject.cluster.Leader;
+import org.onosproject.cluster.Leadership;
+import org.onosproject.cluster.NodeId;
+import org.onosproject.event.Change;
+import org.onosproject.store.service.AsyncLeaderElector;
+
+/**
+ * Atomix leader elector.
+ */
+public class AtomixLeaderElector implements AsyncLeaderElector {
+    private final io.atomix.core.election.AsyncLeaderElector<NodeId> atomixElector;
+    private final NodeId localNodeId;
+    private final Map<Consumer<Change<Leadership>>, LeadershipEventListener<NodeId>> listenerMap =
+        Maps.newIdentityHashMap();
+
+    public AtomixLeaderElector(io.atomix.core.election.AsyncLeaderElector<NodeId> atomixElector, NodeId localNodeId) {
+        this.atomixElector = atomixElector;
+        this.localNodeId = localNodeId;
+    }
+
+    @Override
+    public String name() {
+        return atomixElector.name();
+    }
+
+    @Override
+    public CompletableFuture<Leadership> run(String topic, NodeId nodeId) {
+        return atomixElector.run(topic, nodeId).thenApply(leadership -> toLeadership(topic, leadership));
+    }
+
+    @Override
+    public CompletableFuture<Void> withdraw(String topic) {
+        return atomixElector.withdraw(topic, localNodeId);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> anoint(String topic, NodeId nodeId) {
+        return atomixElector.anoint(topic, nodeId);
+    }
+
+    @Override
+    public CompletableFuture<Void> evict(NodeId nodeId) {
+        return atomixElector.evict(nodeId);
+    }
+
+    @Override
+    public CompletableFuture<Boolean> promote(String topic, NodeId nodeId) {
+        return atomixElector.promote(topic, nodeId);
+    }
+
+    @Override
+    public CompletableFuture<Leadership> getLeadership(String topic) {
+        return atomixElector.getLeadership(topic).thenApply(leadership -> toLeadership(topic, leadership));
+    }
+
+    @Override
+    public CompletableFuture<Map<String, Leadership>> getLeaderships() {
+        return atomixElector.getLeaderships()
+            .thenApply(leaderships -> leaderships.entrySet().stream()
+                .collect(Collectors.toMap(e -> e.getKey(), e -> toLeadership(e.getKey(), e.getValue()))));
+    }
+
+    @Override
+    public CompletableFuture<Void> addChangeListener(Consumer<Change<Leadership>> consumer) {
+        LeadershipEventListener<NodeId> atomixListener = event ->
+            consumer.accept(new Change<>(
+                toLeadership(event.topic(), event.oldLeadership()),
+                toLeadership(event.topic(), event.newLeadership())));
+        listenerMap.put(consumer, atomixListener);
+        return atomixElector.addListener(atomixListener);
+    }
+
+    @Override
+    public CompletableFuture<Void> removeChangeListener(Consumer<Change<Leadership>> consumer) {
+        LeadershipEventListener<NodeId> atomixListener = listenerMap.remove(consumer);
+        if (atomixListener != null) {
+            return atomixElector.removeListener(atomixListener);
+        }
+        return CompletableFuture.completedFuture(null);
+    }
+
+    private Leadership toLeadership(String topic, io.atomix.core.election.Leadership<NodeId> leadership) {
+        return leadership != null
+            ? new Leadership(topic, toLeader(leadership.leader()), leadership.candidates())
+            : null;
+    }
+
+    private Leader toLeader(io.atomix.core.election.Leader<NodeId> leader) {
+        return leader != null ? new Leader(leader.id(), leader.term(), leader.timestamp()) : null;
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixLeaderElectorBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixLeaderElectorBuilder.java
new file mode 100644
index 0000000..feb8127
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixLeaderElectorBuilder.java
@@ -0,0 +1,56 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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 io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.cluster.NodeId;
+import org.onosproject.store.serializers.KryoNamespaces;
+import org.onosproject.store.service.AsyncLeaderElector;
+import org.onosproject.store.service.LeaderElectorBuilder;
+import org.onosproject.store.service.Serializer;
+
+/**
+ * Default {@link org.onosproject.store.service.AsyncLeaderElector} builder.
+ */
+public class AtomixLeaderElectorBuilder extends LeaderElectorBuilder {
+    private static final int MAX_RETRIES = 5;
+    private final Atomix atomix;
+    private final String group;
+    private final NodeId localNodeId;
+
+    public AtomixLeaderElectorBuilder(Atomix atomix, String group, NodeId localNodeId) {
+        this.atomix = atomix;
+        this.group = group;
+        this.localNodeId = localNodeId;
+    }
+
+    @Override
+    public AsyncLeaderElector build() {
+        Serializer serializer = Serializer.using(KryoNamespaces.API);
+        return new AtomixLeaderElector(atomix.<NodeId>leaderElectorBuilder(name())
+            .withProtocol(MultiRaftProtocol.builder(group)
+                .withRecoveryStrategy(Recovery.RECOVER)
+                .withMaxRetries(MAX_RETRIES)
+                .build())
+            .withReadOnly(readOnly())
+            // TODO: Enable caching for LeaderElector in Atomix
+            .withSerializer(new AtomixSerializerAdapter(serializer))
+            .build()
+            .async(), localNodeId);
+    }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixSerializerAdapter.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixSerializerAdapter.java
similarity index 80%
rename from core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixSerializerAdapter.java
rename to core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixSerializerAdapter.java
index abb1580..46ca0b8 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixSerializerAdapter.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixSerializerAdapter.java
@@ -1,5 +1,5 @@
 /*
- * Copyright 2017-present Open Networking Foundation
+ * Copyright 2018-present Open Networking Foundation
  *
  * Licensed under the Apache License, Version 2.0 (the "License");
  * you may not use this file except in compliance with the License.
@@ -13,14 +13,14 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.onosproject.store.primitives.resources.impl;
+package org.onosproject.store.primitives.impl;
 
 import org.onosproject.store.service.Serializer;
 
 /**
- * ONOS to Atomix serializer adapter.
+ * Atomix serializer.
  */
-public class AtomixSerializerAdapter implements io.atomix.serializer.Serializer {
+public class AtomixSerializerAdapter implements io.atomix.utils.serializer.Serializer {
     private final Serializer serializer;
 
     public AtomixSerializerAdapter(Serializer serializer) {
@@ -36,4 +36,4 @@
     public <T> T decode(byte[] bytes) {
         return serializer.decode(bytes);
     }
-}
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixTransactionContext.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixTransactionContext.java
new file mode 100644
index 0000000..9260591
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixTransactionContext.java
@@ -0,0 +1,104 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import io.atomix.core.transaction.AsyncTransaction;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.primitives.TransactionId;
+import org.onosproject.store.service.CommitStatus;
+import org.onosproject.store.service.DistributedPrimitive;
+import org.onosproject.store.service.Serializer;
+import org.onosproject.store.service.StorageException;
+import org.onosproject.store.service.TransactionContext;
+import org.onosproject.store.service.TransactionalMap;
+
+/**
+ * Atomix transaction context.
+ */
+public class AtomixTransactionContext implements TransactionContext {
+    private static final int MAX_RETRIES = 5;
+
+    private final AsyncTransaction atomixTransaction;
+    private final String group;
+
+    public AtomixTransactionContext(AsyncTransaction atomixTransaction, String group) {
+        this.atomixTransaction = atomixTransaction;
+        this.group = group;
+    }
+
+    @Override
+    public String name() {
+        return atomixTransaction.name();
+    }
+
+    @Override
+    public TransactionId transactionId() {
+        return TransactionId.from(atomixTransaction.transactionId().id());
+    }
+
+    @Override
+    public boolean isOpen() {
+        return atomixTransaction.isOpen();
+    }
+
+    @Override
+    public void begin() {
+        try {
+            atomixTransaction.begin().get(DistributedPrimitive.DEFAULT_OPERATION_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS);
+        } catch (InterruptedException e) {
+            throw new StorageException.Interrupted();
+        } catch (TimeoutException e) {
+            throw new StorageException.Timeout();
+        } catch (ExecutionException e) {
+            throw new StorageException.Unavailable();
+        }
+    }
+
+    @Override
+    public CompletableFuture<CommitStatus> commit() {
+        return atomixTransaction.commit().thenApply(status -> CommitStatus.valueOf(status.name()));
+    }
+
+    @Override
+    public void abort() {
+        try {
+            atomixTransaction.abort().get(DistributedPrimitive.DEFAULT_OPERATION_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS);
+        } catch (InterruptedException e) {
+            throw new StorageException.Interrupted();
+        } catch (TimeoutException e) {
+            throw new StorageException.Timeout();
+        } catch (ExecutionException e) {
+            throw new StorageException.Unavailable();
+        }
+    }
+
+    @Override
+    public <K, V> TransactionalMap<K, V> getTransactionalMap(String mapName, Serializer serializer) {
+        return new AtomixTransactionalMap<>(atomixTransaction.<K, V>mapBuilder(mapName)
+            .withProtocol(MultiRaftProtocol.builder(group)
+                .withRecoveryStrategy(Recovery.RECOVER)
+                .withMaxRetries(MAX_RETRIES)
+                .build())
+            .withSerializer(new AtomixSerializerAdapter(serializer))
+            .build());
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixTransactionContextBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixTransactionContextBuilder.java
new file mode 100644
index 0000000..70ef628
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixTransactionContextBuilder.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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 io.atomix.core.Atomix;
+import io.atomix.core.transaction.Isolation;
+import org.onosproject.store.service.TransactionContext;
+import org.onosproject.store.service.TransactionContextBuilder;
+
+/**
+ * Atomix transaction context builder.
+ */
+public class AtomixTransactionContextBuilder extends TransactionContextBuilder {
+    private final Atomix atomix;
+    private final String group;
+
+    public AtomixTransactionContextBuilder(Atomix atomix, String group) {
+        this.atomix = atomix;
+        this.group = group;
+    }
+
+    @Override
+    public TransactionContext build() {
+        return new AtomixTransactionContext(atomix.transactionBuilder()
+            .withIsolation(Isolation.REPEATABLE_READS)
+            .build()
+            .async(), group);
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixTransactionalMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixTransactionalMap.java
new file mode 100644
index 0000000..8577a84
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixTransactionalMap.java
@@ -0,0 +1,64 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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.service.TransactionalMap;
+
+/**
+ * Atomix transactional map.
+ */
+public class AtomixTransactionalMap<K, V> implements TransactionalMap<K, V> {
+    private final io.atomix.core.transaction.TransactionalMap<K, V> atomixMap;
+
+    public AtomixTransactionalMap(io.atomix.core.transaction.TransactionalMap<K, V> atomixMap) {
+        this.atomixMap = atomixMap;
+    }
+
+    @Override
+    public V get(K key) {
+        return atomixMap.get(key);
+    }
+
+    @Override
+    public boolean containsKey(K key) {
+        return atomixMap.containsKey(key);
+    }
+
+    @Override
+    public V put(K key, V value) {
+        return atomixMap.put(key, value);
+    }
+
+    @Override
+    public V remove(K key) {
+        return atomixMap.remove(key);
+    }
+
+    @Override
+    public V putIfAbsent(K key, V value) {
+        return atomixMap.putIfAbsent(key, value);
+    }
+
+    @Override
+    public boolean remove(K key, V value) {
+        return atomixMap.remove(key, value);
+    }
+
+    @Override
+    public boolean replace(K key, V oldValue, V newValue) {
+        return atomixMap.replace(key, oldValue, newValue);
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixWorkQueue.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixWorkQueue.java
new file mode 100644
index 0000000..d3b9907
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixWorkQueue.java
@@ -0,0 +1,81 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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.Collection;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executor;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+import org.onosproject.store.service.Task;
+import org.onosproject.store.service.WorkQueue;
+import org.onosproject.store.service.WorkQueueStats;
+
+/**
+ * Atomix work queue.
+ */
+public class AtomixWorkQueue<E> implements WorkQueue<E> {
+    private final io.atomix.core.workqueue.AsyncWorkQueue<E> atomixWorkQueue;
+
+    public AtomixWorkQueue(io.atomix.core.workqueue.AsyncWorkQueue<E> atomixWorkQueue) {
+        this.atomixWorkQueue = atomixWorkQueue;
+    }
+
+    @Override
+    public String name() {
+        return atomixWorkQueue.name();
+    }
+
+    @Override
+    public CompletableFuture<Void> addMultiple(Collection<E> items) {
+        return atomixWorkQueue.addMultiple(items);
+    }
+
+    @Override
+    public CompletableFuture<Collection<Task<E>>> take(int maxItems) {
+        return atomixWorkQueue.take(maxItems)
+            .thenApply(tasks -> tasks.stream()
+                .map(task -> new Task<>(task.taskId(), task.payload()))
+                .collect(Collectors.toList()));
+    }
+
+    @Override
+    public CompletableFuture<Void> complete(Collection<String> taskIds) {
+        return atomixWorkQueue.complete(taskIds);
+    }
+
+    @Override
+    public CompletableFuture<Void> registerTaskProcessor(
+        Consumer<E> taskProcessor, int parallelism, Executor executor) {
+        return atomixWorkQueue.registerTaskProcessor(taskProcessor, parallelism, executor);
+    }
+
+    @Override
+    public CompletableFuture<Void> stopProcessing() {
+        return atomixWorkQueue.stopProcessing();
+    }
+
+    @Override
+    public CompletableFuture<WorkQueueStats> stats() {
+        return atomixWorkQueue.stats()
+            .thenApply(stats -> WorkQueueStats.builder()
+                .withTotalCompleted(stats.totalCompleted())
+                .withTotalInProgress(stats.totalInProgress())
+                .withTotalPending(stats.totalPending())
+                .build());
+    }
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixWorkQueueBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixWorkQueueBuilder.java
new file mode 100644
index 0000000..9c65106
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/AtomixWorkQueueBuilder.java
@@ -0,0 +1,51 @@
+/*
+ * Copyright 2018-present Open Networking Foundation
+ *
+ * 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 io.atomix.core.Atomix;
+import io.atomix.primitive.Recovery;
+import io.atomix.protocols.raft.MultiRaftProtocol;
+import org.onosproject.store.service.WorkQueue;
+import org.onosproject.store.service.WorkQueueBuilder;
+
+/**
+ * Default {@link org.onosproject.store.service.WorkQueue} builder.
+ *
+ * @param <E> type for set value
+ */
+public class AtomixWorkQueueBuilder<E> extends WorkQueueBuilder<E> {
+    private static final int MAX_RETRIES = 5;
+    private final Atomix atomix;
+    private final String group;
+
+    public AtomixWorkQueueBuilder(Atomix atomix, String group) {
+        this.atomix = atomix;
+        this.group = group;
+    }
+
+    @Override
+    public WorkQueue<E> build() {
+        return new AtomixWorkQueue<>(atomix.<E>workQueueBuilder(name())
+            .withProtocol(MultiRaftProtocol.builder(group)
+                .withRecoveryStrategy(Recovery.RECOVER)
+                .withMaxRetries(MAX_RETRIES)
+                .build())
+            .withReadOnly(readOnly())
+            .withSerializer(new AtomixSerializerAdapter(serializer()))
+            .build()
+            .async());
+    }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncConsistentMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncConsistentMap.java
deleted file mode 100644
index a25fb79..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncConsistentMap.java
+++ /dev/null
@@ -1,245 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.Map;
-import java.util.Objects;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executor;
-import java.util.function.BiFunction;
-import java.util.function.Consumer;
-import java.util.function.Predicate;
-
-import com.google.common.util.concurrent.MoreExecutors;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.Versioned;
-import org.slf4j.Logger;
-
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-
-import static org.onosproject.store.service.DistributedPrimitive.Status.INACTIVE;
-import static org.onosproject.store.service.DistributedPrimitive.Status.SUSPENDED;
-
-/**
- * {@code AsyncConsistentMap} that caches entries on read.
- * <p>
- * The cache entries are automatically invalidated when updates are detected either locally or
- * remotely.
- * <p> This implementation only attempts to serve cached entries for {@link AsyncConsistentMap#get get}
- * {@link AsyncConsistentMap#getOrDefault(Object, Object) getOrDefault}, and
- * {@link AsyncConsistentMap#containsKey(Object) containsKey} calls. All other calls skip the cache
- * and directly go the backing map.
- *
- * @param <K> key type
- * @param <V> value type
- */
-public class CachingAsyncConsistentMap<K, V> extends DelegatingAsyncConsistentMap<K, V> {
-    private static final int DEFAULT_CACHE_SIZE = 10000;
-    private final Logger log = getLogger(getClass());
-
-    private final Map<MapEventListener<K, V>, Executor> mapEventListeners = new ConcurrentHashMap<>();
-    private final LoadingCache<K, CompletableFuture<Versioned<V>>> cache;
-    private final AsyncConsistentMap<K, V> backingMap;
-    private final MapEventListener<K, V> cacheUpdater;
-    private final Consumer<Status> statusListener;
-
-    /**
-     * Default constructor.
-     *
-     * @param backingMap a distributed, strongly consistent map for backing
-     */
-    public CachingAsyncConsistentMap(AsyncConsistentMap<K, V> backingMap) {
-        this(backingMap, DEFAULT_CACHE_SIZE);
-    }
-
-    /**
-     * Constructor to configure cache size.
-     *
-     * @param backingMap a distributed, strongly consistent map for backing
-     * @param cacheSize the maximum size of the cache
-     */
-    public CachingAsyncConsistentMap(AsyncConsistentMap<K, V> backingMap, int cacheSize) {
-        super(backingMap);
-        this.backingMap = backingMap;
-        cache = CacheBuilder.newBuilder()
-                            .maximumSize(cacheSize)
-                            .build(CacheLoader.from(CachingAsyncConsistentMap.super::get));
-        cacheUpdater = event -> {
-            Versioned<V> newValue = event.newValue();
-            if (newValue == null) {
-                cache.invalidate(event.key());
-            } else {
-                cache.put(event.key(), CompletableFuture.completedFuture(newValue));
-            }
-            mapEventListeners.forEach((listener, executor) -> executor.execute(() -> listener.event(event)));
-        };
-        statusListener = status -> {
-            log.debug("{} status changed to {}", this.name(), status);
-            // If the status of the underlying map is SUSPENDED or INACTIVE
-            // we can no longer guarantee that the cache will be in sync.
-            if (status == SUSPENDED || status == INACTIVE) {
-                cache.invalidateAll();
-            }
-        };
-        super.addListener(cacheUpdater, MoreExecutors.directExecutor());
-        super.addStatusChangeListener(statusListener);
-    }
-
-    @Override
-    public CompletableFuture<Void> destroy() {
-        super.removeStatusChangeListener(statusListener);
-        return super.destroy().thenCompose(v -> removeListener(cacheUpdater));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> get(K key) {
-        return cache.getUnchecked(key)
-                .whenComplete((r, e) -> {
-                    if (e != null) {
-                        cache.invalidate(key);
-                    }
-                });
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> getOrDefault(K key, V defaultValue) {
-        return cache.getUnchecked(key).thenCompose(r -> {
-            if (r == null) {
-                CompletableFuture<Versioned<V>> versioned = backingMap.getOrDefault(key, defaultValue);
-                cache.put(key, versioned);
-                return versioned;
-            } else {
-                return CompletableFuture.completedFuture(r);
-            }
-        }).whenComplete((r, e) -> {
-            if (e != null) {
-                cache.invalidate(key);
-            }
-        });
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> computeIf(K key,
-            Predicate<? super V> condition,
-            BiFunction<? super K, ? super V, ? extends V> remappingFunction) {
-        return super.computeIf(key, condition, remappingFunction)
-                .whenComplete((r, e) -> cache.invalidate(key));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> put(K key, V value) {
-        return super.put(key, value)
-                .whenComplete((r, e) -> cache.invalidate(key));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> putAndGet(K key, V value) {
-        return super.putAndGet(key, value)
-                .whenComplete((r, e) -> cache.invalidate(key));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> putIfAbsent(K key, V value) {
-        return super.putIfAbsent(key, value)
-                .whenComplete((r, e) -> cache.invalidate(key));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> remove(K key) {
-        return super.remove(key)
-                .whenComplete((r, e) -> cache.invalidate(key));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsKey(K key) {
-        return cache.getUnchecked(key).thenApply(Objects::nonNull)
-                .whenComplete((r, e) -> {
-                    if (e != null) {
-                        cache.invalidate(key);
-                    }
-                });
-    }
-
-    @Override
-    public CompletableFuture<Void> clear() {
-        return super.clear()
-                .whenComplete((r, e) -> cache.invalidateAll());
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(K key, V value) {
-        return super.remove(key, value)
-                .whenComplete((r, e) -> {
-                    if (r) {
-                        cache.invalidate(key);
-                    }
-                });
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(K key, long version) {
-        return super.remove(key, version)
-                .whenComplete((r, e) -> {
-                    if (r) {
-                        cache.invalidate(key);
-                    }
-                });
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> replace(K key, V value) {
-        return super.replace(key, value)
-                .whenComplete((r, e) -> cache.invalidate(key));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(K key, V oldValue, V newValue) {
-        return super.replace(key, oldValue, newValue)
-                .whenComplete((r, e) -> {
-                    if (r) {
-                        cache.invalidate(key);
-                    }
-                });
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(K key, long oldVersion, V newValue) {
-        return super.replace(key, oldVersion, newValue)
-                .whenComplete((r, e) -> {
-                    if (r) {
-                        cache.invalidate(key);
-                    }
-                });
-    }
-
-    @Override
-    public CompletableFuture<Void> addListener(MapEventListener<K, V> listener, Executor executor) {
-        mapEventListeners.put(listener, executor);
-        return CompletableFuture.completedFuture(null);
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(MapEventListener<K, V> listener) {
-        mapEventListeners.remove(listener);
-        return CompletableFuture.completedFuture(null);
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncConsistentMultimap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncConsistentMultimap.java
deleted file mode 100644
index 3d49f7f..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncConsistentMultimap.java
+++ /dev/null
@@ -1,198 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * 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.Collection;
-import java.util.Map;
-import java.util.Objects;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executor;
-import java.util.function.Consumer;
-import java.util.stream.Collectors;
-
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.util.concurrent.MoreExecutors;
-import org.onosproject.store.service.AsyncConsistentMultimap;
-import org.onosproject.store.service.MultimapEventListener;
-import org.onosproject.store.service.Versioned;
-import org.slf4j.Logger;
-
-import static org.onosproject.store.service.DistributedPrimitive.Status.INACTIVE;
-import static org.onosproject.store.service.DistributedPrimitive.Status.SUSPENDED;
-import static org.slf4j.LoggerFactory.getLogger;
-
-/**
- * Caching {@link AsyncConsistentMultimap} implementation.
- */
-public class CachingAsyncConsistentMultimap<K, V> extends DelegatingAsyncConsistentMultimap<K, V> {
-    private static final int DEFAULT_CACHE_SIZE = 10000;
-    private final Logger log = getLogger(getClass());
-
-    private final Map<MultimapEventListener<K, V>, Executor> mapEventListeners = new ConcurrentHashMap<>();
-    private final LoadingCache<K, CompletableFuture<Versioned<Collection<? extends V>>>> cache;
-    private final MultimapEventListener<K, V> cacheUpdater;
-    private final Consumer<Status> statusListener;
-
-    /**
-     * Default constructor.
-     *
-     * @param backingMap a distributed, strongly consistent map for backing
-     */
-    public CachingAsyncConsistentMultimap(AsyncConsistentMultimap<K, V> backingMap) {
-        this(backingMap, DEFAULT_CACHE_SIZE);
-    }
-
-    /**
-     * Constructor to configure cache size.
-     *
-     * @param backingMap a distributed, strongly consistent map for backing
-     * @param cacheSize  the maximum size of the cache
-     */
-    public CachingAsyncConsistentMultimap(AsyncConsistentMultimap<K, V> backingMap, int cacheSize) {
-        super(backingMap);
-        cache = CacheBuilder.newBuilder()
-            .maximumSize(cacheSize)
-            .build(CacheLoader.from(CachingAsyncConsistentMultimap.super::get));
-        cacheUpdater = event -> {
-            V oldValue = event.oldValue();
-            V newValue = event.newValue();
-            CompletableFuture<Versioned<Collection<? extends V>>> future = cache.getUnchecked(event.key());
-            switch (event.type()) {
-                case INSERT:
-                    if (future.isDone()) {
-                        Versioned<Collection<? extends V>> oldVersioned = future.join();
-                        Versioned<Collection<? extends V>> newVersioned = new Versioned<>(
-                            ImmutableSet.<V>builder().addAll(oldVersioned.value()).add(newValue).build(),
-                            oldVersioned.version(),
-                            oldVersioned.creationTime());
-                        cache.put(event.key(), CompletableFuture.completedFuture(newVersioned));
-                    } else {
-                        cache.put(event.key(), future.thenApply(versioned -> new Versioned<>(
-                            ImmutableSet.<V>builder().addAll(versioned.value()).add(newValue).build(),
-                            versioned.version(),
-                            versioned.creationTime())));
-                    }
-                    break;
-                case REMOVE:
-                    if (future.isDone()) {
-                        Versioned<Collection<? extends V>> oldVersioned = future.join();
-                        cache.put(event.key(), CompletableFuture.completedFuture(new Versioned<>(oldVersioned.value()
-                            .stream()
-                            .filter(value -> !Objects.equals(value, oldValue))
-                            .collect(Collectors.toSet()), oldVersioned.version(), oldVersioned.creationTime())));
-                    } else {
-                        cache.put(event.key(), future.thenApply(versioned -> new Versioned<>(versioned.value()
-                            .stream()
-                            .filter(value -> !Objects.equals(value, oldValue))
-                            .collect(Collectors.toSet()), versioned.version(), versioned.creationTime())));
-                    }
-                    break;
-                default:
-                    break;
-            }
-            mapEventListeners.forEach((listener, executor) -> executor.execute(() -> listener.event(event)));
-        };
-        statusListener = status -> {
-            log.debug("{} status changed to {}", this.name(), status);
-            // If the status of the underlying map is SUSPENDED or INACTIVE
-            // we can no longer guarantee that the cache will be in sync.
-            if (status == SUSPENDED || status == INACTIVE) {
-                cache.invalidateAll();
-            }
-        };
-        super.addListener(cacheUpdater, MoreExecutors.directExecutor());
-        super.addStatusChangeListener(statusListener);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsKey(K key) {
-        return get(key).thenApply(value -> value != null && !value.value().isEmpty());
-    }
-
-    @Override
-    public CompletableFuture<Boolean> put(K key, V value) {
-        return super.put(key, value)
-            .whenComplete((r, e) -> cache.invalidate(key));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(K key, V value) {
-        return super.remove(key, value)
-            .whenComplete((r, e) -> cache.invalidate(key));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> removeAll(K key, Collection<? extends V> values) {
-        return super.removeAll(key, values)
-            .whenComplete((r, e) -> cache.invalidate(key));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends V>>> removeAll(K key) {
-        return super.removeAll(key)
-            .whenComplete((r, e) -> cache.invalidate(key));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> putAll(K key, Collection<? extends V> values) {
-        return super.putAll(key, values)
-            .whenComplete((r, e) -> cache.invalidate(key));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends V>>> replaceValues(K key, Collection<V> values) {
-        return super.replaceValues(key, values)
-            .whenComplete((r, e) -> cache.invalidate(key));
-    }
-
-    @Override
-    public CompletableFuture<Void> clear() {
-        return super.clear()
-            .whenComplete((r, e) -> cache.invalidateAll());
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends V>>> get(K key) {
-        return cache.getUnchecked(key)
-            .whenComplete((r, e) -> {
-                if (e != null) {
-                    cache.invalidate(key);
-                }
-            });
-    }
-
-    @Override
-    public CompletableFuture<Void> addListener(MultimapEventListener<K, V> listener, Executor executor) {
-        mapEventListeners.put(listener, executor);
-        return CompletableFuture.completedFuture(null);
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(MultimapEventListener<K, V> listener) {
-        mapEventListeners.remove(listener);
-        return CompletableFuture.completedFuture(null);
-    }
-
-    @Override
-    public CompletableFuture<Void> destroy() {
-        super.removeStatusChangeListener(statusListener);
-        return super.destroy().thenCompose(v -> removeListener(cacheUpdater));
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncDocumentTree.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncDocumentTree.java
deleted file mode 100644
index eece91e..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncDocumentTree.java
+++ /dev/null
@@ -1,166 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Consumer;
-
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import com.google.common.collect.Maps;
-import org.onosproject.store.service.AsyncDocumentTree;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTreeEvent;
-import org.onosproject.store.service.DocumentTreeListener;
-import org.onosproject.store.service.Versioned;
-import org.slf4j.Logger;
-
-import static org.onosproject.store.service.DistributedPrimitive.Status.INACTIVE;
-import static org.onosproject.store.service.DistributedPrimitive.Status.SUSPENDED;
-import static org.slf4j.LoggerFactory.getLogger;
-
-/**
- * Caching asynchronous document tree.
- */
-public class CachingAsyncDocumentTree<V> extends DelegatingAsyncDocumentTree<V> implements AsyncDocumentTree<V> {
-    private static final int DEFAULT_CACHE_SIZE = 10000;
-    private final Logger log = getLogger(getClass());
-
-    private final Map<DocumentTreeListener<V>, DocumentTreeListener<V>> eventListeners = Maps.newConcurrentMap();
-    private final LoadingCache<DocumentPath, CompletableFuture<Versioned<V>>> cache;
-    private final DocumentTreeListener<V> cacheUpdater;
-    private final Consumer<Status> statusListener;
-
-    /**
-     * Default constructor.
-     *
-     * @param backingTree a distributed, strongly consistent map for backing
-     */
-    public CachingAsyncDocumentTree(AsyncDocumentTree<V> backingTree) {
-        this(backingTree, DEFAULT_CACHE_SIZE);
-    }
-
-    /**
-     * Constructor to configure cache size.
-     *
-     * @param backingTree a distributed, strongly consistent map for backing
-     * @param cacheSize the maximum size of the cache
-     */
-    public CachingAsyncDocumentTree(AsyncDocumentTree<V> backingTree, int cacheSize) {
-        super(backingTree);
-        cache = CacheBuilder.newBuilder()
-                .maximumSize(cacheSize)
-                .build(CacheLoader.from(CachingAsyncDocumentTree.super::get));
-        cacheUpdater = event -> {
-            if (!event.newValue().isPresent()) {
-                cache.invalidate(event.path());
-            } else {
-                cache.put(event.path(), CompletableFuture.completedFuture(event.newValue().get()));
-            }
-            eventListeners.values().forEach(listener -> listener.event(event));
-        };
-        statusListener = status -> {
-            log.debug("{} status changed to {}", this.name(), status);
-            // If the status of the underlying map is SUSPENDED or INACTIVE
-            // we can no longer guarantee that the cache will be in sync.
-            if (status == SUSPENDED || status == INACTIVE) {
-                cache.invalidateAll();
-            }
-        };
-        super.addListener(root(), cacheUpdater);
-        super.addStatusChangeListener(statusListener);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> get(DocumentPath path) {
-        return cache.getUnchecked(path);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> set(DocumentPath path, V value) {
-        return super.set(path, value)
-                .whenComplete((r, e) -> cache.invalidate(path));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> create(DocumentPath path, V value) {
-        return super.create(path, value)
-                .whenComplete((r, e) -> cache.invalidate(path));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> createRecursive(DocumentPath path, V value) {
-        return super.createRecursive(path, value)
-                .whenComplete((r, e) -> cache.invalidate(path));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(DocumentPath path, V newValue, long version) {
-        return super.replace(path, newValue, version)
-                .whenComplete((r, e) -> {
-                    if (r) {
-                        cache.invalidate(path);
-                    }
-                });
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(DocumentPath path, V newValue, V currentValue) {
-        return super.replace(path, newValue, currentValue)
-                .whenComplete((r, e) -> {
-                    if (r) {
-                        cache.invalidate(path);
-                    }
-                });
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> removeNode(DocumentPath path) {
-        return super.removeNode(path)
-                .whenComplete((r, e) -> cache.invalidate(path));
-    }
-
-    @Override
-    public CompletableFuture<Void> addListener(DocumentPath path, DocumentTreeListener<V> listener) {
-        eventListeners.put(listener, new InternalListener(path, listener));
-        return CompletableFuture.completedFuture(null);
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(DocumentTreeListener<V> listener) {
-        eventListeners.remove(listener);
-        return CompletableFuture.completedFuture(null);
-    }
-
-    private class InternalListener implements DocumentTreeListener<V> {
-        private final DocumentPath path;
-        private final DocumentTreeListener<V> listener;
-
-        public InternalListener(DocumentPath path, DocumentTreeListener<V> listener) {
-            this.path = path;
-            this.listener = listener;
-        }
-
-        @Override
-        public void event(DocumentTreeEvent<V> event) {
-            if (event.path().isDescendentOf(path)) {
-                listener.event(event);
-            }
-        }
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncLeaderElector.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncLeaderElector.java
deleted file mode 100644
index 91dcca5..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CachingAsyncLeaderElector.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * 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.concurrent.CompletableFuture;
-import java.util.function.Consumer;
-
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.CacheLoader;
-import com.google.common.cache.LoadingCache;
-import org.onosproject.cluster.Leadership;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.event.Change;
-import org.onosproject.store.service.AsyncLeaderElector;
-
-/**
- * Caching async leader elector.
- */
-public class CachingAsyncLeaderElector extends DelegatingAsyncLeaderElector {
-    private final LoadingCache<String, CompletableFuture<Leadership>> cache;
-    private final Consumer<Change<Leadership>> cacheUpdater;
-    private final Consumer<Status> statusListener;
-
-    public CachingAsyncLeaderElector(AsyncLeaderElector delegateLeaderElector) {
-        super(delegateLeaderElector);
-        cache = CacheBuilder.newBuilder()
-            .maximumSize(1000)
-            .build(CacheLoader.from(super::getLeadership));
-
-        cacheUpdater = change -> {
-            Leadership leadership = change.newValue();
-            cache.put(leadership.topic(), CompletableFuture.completedFuture(leadership));
-        };
-        statusListener = status -> {
-            if (status == Status.SUSPENDED || status == Status.INACTIVE) {
-                cache.invalidateAll();
-            }
-        };
-        addChangeListener(cacheUpdater);
-        addStatusChangeListener(statusListener);
-    }
-
-    @Override
-    public CompletableFuture<Leadership> getLeadership(String topic) {
-        return cache.getUnchecked(topic)
-            .whenComplete((r, e) -> {
-                if (e != null) {
-                    cache.invalidate(topic);
-                }
-            });
-    }
-
-    @Override
-    public CompletableFuture<Leadership> run(String topic, NodeId nodeId) {
-        return super.run(topic, nodeId).whenComplete((r, e) -> cache.invalidate(topic));
-    }
-
-    @Override
-    public CompletableFuture<Void> withdraw(String topic) {
-        return super.withdraw(topic).whenComplete((r, e) -> cache.invalidate(topic));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> anoint(String topic, NodeId nodeId) {
-        return super.anoint(topic, nodeId).whenComplete((r, e) -> cache.invalidate(topic));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> promote(String topic, NodeId nodeId) {
-        return super.promote(topic, nodeId).whenComplete((r, e) -> cache.invalidate(topic));
-    }
-
-    @Override
-    public CompletableFuture<Void> evict(NodeId nodeId) {
-        return super.evict(nodeId).whenComplete((r, e) -> cache.invalidateAll());
-    }
-
-    @Override
-    public CompletableFuture<Void> destroy() {
-        removeStatusChangeListener(statusListener);
-        return removeChangeListener(cacheUpdater);
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CompatibleValue.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CompatibleValue.java
deleted file mode 100644
index 73c4d12..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CompatibleValue.java
+++ /dev/null
@@ -1,59 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * 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.core.Version;
-
-import static com.google.common.base.MoreObjects.toStringHelper;
-
-/**
- * Compatibility wrapper for primitive values.
- */
-public class CompatibleValue<T> {
-    private final T value;
-    private final Version version;
-
-    public CompatibleValue(T value, Version version) {
-        this.value = value;
-        this.version = version;
-    }
-
-    /**
-     * Returns the wrapped value.
-     *
-     * @return the wrapped value
-     */
-    public T value() {
-        return value;
-    }
-
-    /**
-     * Returns the compatibilty version.
-     *
-     * @return the compatibility version
-     */
-    public Version version() {
-        return version;
-    }
-
-    @Override
-    public String toString() {
-        return toStringHelper(this)
-            .add("value", value)
-            .add("version", version)
-            .toString();
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CoordinationManager.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CoordinationManager.java
index 1f6310f..aa9a78ef 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CoordinationManager.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/CoordinationManager.java
@@ -15,9 +15,12 @@
  */
 package org.onosproject.store.primitives.impl;
 
+import java.util.List;
 import java.util.function.Supplier;
 import java.util.stream.Collectors;
 
+import io.atomix.core.Atomix;
+import io.atomix.primitive.partition.PartitionGroup;
 import org.apache.felix.scr.annotations.Activate;
 import org.apache.felix.scr.annotations.Component;
 import org.apache.felix.scr.annotations.Deactivate;
@@ -26,14 +29,10 @@
 import org.apache.felix.scr.annotations.Service;
 import org.onosproject.cluster.ClusterService;
 import org.onosproject.cluster.ControllerNode;
-import org.onosproject.cluster.DefaultPartition;
 import org.onosproject.cluster.NodeId;
-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.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncAtomicValue;
+import org.onosproject.store.impl.AtomixManager;
 import org.onosproject.store.service.AsyncConsistentMultimap;
 import org.onosproject.store.service.AsyncConsistentTreeMap;
 import org.onosproject.store.service.AsyncDocumentTree;
@@ -54,17 +53,15 @@
 import org.onosproject.store.service.TopicBuilder;
 import org.onosproject.store.service.TransactionContextBuilder;
 import org.onosproject.store.service.WorkQueue;
+import org.onosproject.store.service.WorkQueueBuilder;
 import org.slf4j.Logger;
 
-import java.util.List;
-
 import static org.onosproject.security.AppGuard.checkPermission;
 import static org.onosproject.security.AppPermission.Type.STORAGE_WRITE;
 import static org.slf4j.LoggerFactory.getLogger;
 
 /**
- * Implementation of {@code CoordinationService} that uses a {@link StoragePartition} that spans all the nodes
- * in the cluster regardless of version.
+ * Implementation of {@code CoordinationService} that uses the Atomix management partition group.
  */
 @Service
 @Component(immediate = true)
@@ -81,23 +78,16 @@
     @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
     protected PersistenceService persistenceService;
 
-    private StoragePartition partition;
-    private DistributedPrimitiveCreator primitiveCreator;
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected AtomixManager atomixManager;
+
+    private Atomix atomix;
+    private PartitionGroup group;
 
     @Activate
     public void activate() {
-        partition = new StoragePartition(
-                new DefaultPartition(
-                        PartitionId.from(0),
-                        null,
-                        clusterService.getNodes()
-                                .stream()
-                                .map(ControllerNode::id)
-                                .collect(Collectors.toSet())),
-                clusterCommunicator,
-                clusterService);
-        partition.open().join();
-        primitiveCreator = partition.client();
+        atomix = atomixManager.getAtomix();
+        group = atomix.getPartitionService().getSystemPartitionGroup();
         log.info("Started");
     }
 
@@ -112,134 +102,146 @@
         final NodeId localNodeId = clusterService.getLocalNode().id();
 
         Supplier<List<NodeId>> peersSupplier = () -> clusterService.getNodes().stream()
-                .map(ControllerNode::id)
-                .filter(nodeId -> !nodeId.equals(localNodeId))
-                .filter(id -> clusterService.getState(id).isActive())
-                .collect(Collectors.toList());
+            .map(ControllerNode::id)
+            .filter(nodeId -> !nodeId.equals(localNodeId))
+            .filter(id -> clusterService.getState(id).isActive())
+            .collect(Collectors.toList());
 
         Supplier<List<NodeId>> bootstrapPeersSupplier = () -> clusterService.getNodes()
-                .stream()
-                .map(ControllerNode::id)
-                .filter(id -> !localNodeId.equals(id))
-                .filter(id -> clusterService.getState(id).isActive())
-                .collect(Collectors.toList());
+            .stream()
+            .map(ControllerNode::id)
+            .filter(id -> !localNodeId.equals(id))
+            .filter(id -> clusterService.getState(id).isActive())
+            .collect(Collectors.toList());
 
         return new EventuallyConsistentMapBuilderImpl<>(
-                localNodeId,
-                clusterCommunicator,
-                persistenceService,
-                peersSupplier,
-                bootstrapPeersSupplier
+            localNodeId,
+            clusterCommunicator,
+            persistenceService,
+            peersSupplier,
+            bootstrapPeersSupplier
         );
     }
 
     @Override
     public <K, V> ConsistentMapBuilder<K, V> consistentMapBuilder() {
         checkPermission(STORAGE_WRITE);
-        return new DefaultConsistentMapBuilder<>(primitiveCreator);
+        return new AtomixConsistentMapBuilder<>(atomix, group.name());
     }
 
     @Override
     public <V> DocumentTreeBuilder<V> documentTreeBuilder() {
         checkPermission(STORAGE_WRITE);
-        return new DefaultDocumentTreeBuilder<>(primitiveCreator);
+        return new AtomixDocumentTreeBuilder<>(atomix, group.name());
     }
 
     @Override
     public <V> ConsistentTreeMapBuilder<V> consistentTreeMapBuilder() {
-        return new DefaultConsistentTreeMapBuilder<>(primitiveCreator);
+        return new AtomixConsistentTreeMapBuilder<>(atomix, group.name());
     }
 
     @Override
     public <K, V> ConsistentMultimapBuilder<K, V> consistentMultimapBuilder() {
         checkPermission(STORAGE_WRITE);
-        return new DefaultConsistentMultimapBuilder<>(primitiveCreator);
+        return new AtomixConsistentMultimapBuilder<>(atomix, group.name());
     }
 
     @Override
     public <K> AtomicCounterMapBuilder<K> atomicCounterMapBuilder() {
         checkPermission(STORAGE_WRITE);
-        return new DefaultAtomicCounterMapBuilder<>(primitiveCreator);
+        return new AtomixAtomicCounterMapBuilder<>(atomix, group.name());
     }
 
     @Override
     public <E> DistributedSetBuilder<E> setBuilder() {
         checkPermission(STORAGE_WRITE);
-        return new DefaultDistributedSetBuilder<>(this::<E, Boolean>consistentMapBuilder);
+        return new AtomixDistributedSetBuilder<>(atomix, group.name());
     }
 
     @Override
     public AtomicCounterBuilder atomicCounterBuilder() {
         checkPermission(STORAGE_WRITE);
-        return new DefaultAtomicCounterBuilder(primitiveCreator);
+        return new AtomixAtomicCounterBuilder(atomix, group.name());
     }
 
     @Override
     public AtomicIdGeneratorBuilder atomicIdGeneratorBuilder() {
         checkPermission(STORAGE_WRITE);
-        return new DefaultAtomicIdGeneratorBuilder(primitiveCreator);
+        return new AtomixAtomicIdGeneratorBuilder(atomix, group.name());
     }
 
     @Override
     public <V> AtomicValueBuilder<V> atomicValueBuilder() {
         checkPermission(STORAGE_WRITE);
-        Supplier<ConsistentMapBuilder<String, byte[]>> mapBuilderSupplier =
-                () -> this.<String, byte[]>consistentMapBuilder()
-                          .withName("onos-atomic-values")
-                          .withSerializer(Serializer.using(KryoNamespaces.BASIC));
-        return new DefaultAtomicValueBuilder<>(mapBuilderSupplier);
+        return new AtomixAtomicValueBuilder<>(atomix, group.name());
     }
 
     @Override
     public TransactionContextBuilder transactionContextBuilder() {
-        throw new UnsupportedOperationException();
+        checkPermission(STORAGE_WRITE);
+        return new AtomixTransactionContextBuilder(atomix, group.name());
     }
 
     @Override
     public LeaderElectorBuilder leaderElectorBuilder() {
         checkPermission(STORAGE_WRITE);
-        return new DefaultLeaderElectorBuilder(primitiveCreator);
+        return new AtomixLeaderElectorBuilder(atomix, group.name(), clusterService.getLocalNode().id());
     }
 
     @Override
     public <T> TopicBuilder<T> topicBuilder() {
         checkPermission(STORAGE_WRITE);
-        return new DefaultDistributedTopicBuilder<>(atomicValueBuilder());
+        return new AtomixDistributedTopicBuilder<>(atomix, group.name());
+    }
+
+    @Override
+    public <E> WorkQueueBuilder<E> workQueueBuilder() {
+        checkPermission(STORAGE_WRITE);
+        return new AtomixWorkQueueBuilder<>(atomix, group.name());
     }
 
     @Override
     public <E> WorkQueue<E> getWorkQueue(String name, Serializer serializer) {
         checkPermission(STORAGE_WRITE);
-        return primitiveCreator.newWorkQueue(name, serializer);
+        return this.<E>workQueueBuilder()
+            .withName(name)
+            .withSerializer(serializer)
+            .build();
     }
 
     @Override
     public <V> AsyncDocumentTree<V> getDocumentTree(String name, Serializer serializer) {
         checkPermission(STORAGE_WRITE);
-        return primitiveCreator.newAsyncDocumentTree(name, serializer);
+        return this.<V>documentTreeBuilder()
+            .withName(name)
+            .withSerializer(serializer)
+            .build();
     }
 
     @Override
-    public <K, V> AsyncConsistentMultimap<K, V> getAsyncSetMultimap(
-            String name, Serializer serializer) {
+    public <K, V> AsyncConsistentMultimap<K, V> getAsyncSetMultimap(String name, Serializer serializer) {
         checkPermission(STORAGE_WRITE);
-        return primitiveCreator.newAsyncConsistentSetMultimap(name,
-                                                                serializer);
+        return this.<K, V>consistentMultimapBuilder()
+            .withName(name)
+            .withSerializer(serializer)
+            .buildMultimap();
     }
 
     @Override
-    public <V> AsyncConsistentTreeMap<V> getAsyncTreeMap(
-            String name, Serializer serializer) {
+    public <V> AsyncConsistentTreeMap<V> getAsyncTreeMap(String name, Serializer serializer) {
         checkPermission(STORAGE_WRITE);
-        return primitiveCreator.newAsyncConsistentTreeMap(name, serializer);
+        return this.<V>consistentTreeMapBuilder()
+            .withName(name)
+            .withSerializer(serializer)
+            .buildTreeMap();
     }
 
     @Override
     public <T> Topic<T> getTopic(String name, Serializer serializer) {
-        AsyncAtomicValue<T> atomicValue = this.<T>atomicValueBuilder()
-                                              .withName("topic-" + name)
-                                              .withSerializer(serializer)
-                                              .build();
-        return new DefaultDistributedTopic<>(atomicValue);
+        checkPermission(STORAGE_WRITE);
+        return this.<T>topicBuilder()
+            .withName(name)
+            .withSerializer(serializer)
+            .build();
     }
 }
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAsyncAtomicValue.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAsyncAtomicValue.java
deleted file mode 100644
index 33af601..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAsyncAtomicValue.java
+++ /dev/null
@@ -1,184 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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 com.google.common.collect.Maps;
-import org.onosproject.store.service.AsyncAtomicValue;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.AtomicValueEvent;
-import org.onosproject.store.service.AtomicValueEventListener;
-import org.onosproject.store.service.MapEvent;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.Versioned;
-import org.onosproject.utils.MeteringAgent;
-
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * Default implementation of a {@code AsyncAtomicValue}.
- *
- * @param <V> value type
- */
-public class DefaultAsyncAtomicValue<V> implements AsyncAtomicValue<V> {
-
-    private final String name;
-    private final Serializer serializer;
-    private final AsyncConsistentMap<String, byte[]> backingMap;
-    private final Map<AtomicValueEventListener<V>, MapEventListener<String, byte[]>> listeners =
-            Maps.newIdentityHashMap();
-    private final MeteringAgent monitor;
-
-    private static final String COMPONENT_NAME = "atomicValue";
-    private static final String GET = "get";
-    private static final String GET_AND_SET = "getAndSet";
-    private static final String SET = "set";
-    private static final String COMPARE_AND_SET = "compareAndSet";
-    private static final String ADD_LISTENER = "addListener";
-    private static final String REMOVE_LISTENER = "removeListener";
-    private static final String NOTIFY_LISTENER = "notifyListener";
-    private static final String DESTROY = "destroy";
-
-    public DefaultAsyncAtomicValue(String name, Serializer serializer, AsyncConsistentMap<String, byte[]> backingMap) {
-        this.name = checkNotNull(name, "name must not be null");
-        this.serializer = checkNotNull(serializer, "serializer must not be null");
-        this.backingMap = checkNotNull(backingMap, "backingMap must not be null");
-        this.monitor = new MeteringAgent(COMPONENT_NAME, name, true);
-    }
-
-    //Bypass osgi error with 'activated' MeteringAgent parameter set to false
-    DefaultAsyncAtomicValue(String name, Serializer serializer, AsyncConsistentMap<String, byte[]> backingMap,
-                             MeteringAgent meteringAgent) {
-        this.name = checkNotNull(name, "name must not be null");
-        this.serializer = checkNotNull(serializer, "serializer must not be null");
-        this.backingMap = checkNotNull(backingMap, "backingMap must not be null");
-        this.monitor = meteringAgent;
-
-    }
-
-    @Override
-    public String name() {
-        return name;
-    }
-
-    @Override
-    public CompletableFuture<Void> destroy() {
-        final MeteringAgent.Context newTimer = monitor.startTimer(DESTROY);
-        return backingMap.remove(name)
-                         .whenComplete((r, e) -> newTimer.stop(e))
-                         .thenApply(v -> null);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> compareAndSet(V expect, V update) {
-        final MeteringAgent.Context newTimer = monitor.startTimer(COMPARE_AND_SET);
-        return backingMap.replace(name, serializer.encode(expect), serializer.encode(update))
-                         .whenComplete((r, e) -> newTimer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<V> get() {
-        final MeteringAgent.Context newTimer = monitor.startTimer(GET);
-        return backingMap.get(name)
-                         .thenApply(Versioned::valueOrNull)
-                         .thenApply(v -> v == null ? null : serializer.<V>decode(v))
-                         .whenComplete((r, e) -> newTimer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<V> getAndSet(V value) {
-        final MeteringAgent.Context newTimer = monitor.startTimer(GET_AND_SET);
-        if (value == null) {
-            return backingMap.remove(name)
-                             .thenApply(Versioned::valueOrNull)
-                             .thenApply(v -> v == null ? null : serializer.<V>decode(v))
-                             .whenComplete((r, e) -> newTimer.stop(e));
-        }
-        return backingMap.put(name, serializer.encode(value))
-                         .thenApply(Versioned::valueOrNull)
-                         .thenApply(v -> v == null ? null : serializer.<V>decode(v))
-                         .whenComplete((r, e) -> newTimer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Void> set(V value) {
-        final MeteringAgent.Context newTimer = monitor.startTimer(SET);
-        if (value == null) {
-            return backingMap.remove(name)
-                             .whenComplete((r, e) -> newTimer.stop(e))
-                             .thenApply(v -> null);
-
-        }
-        return backingMap.put(name, serializer.encode(value))
-                         .whenComplete((r, e) -> newTimer.stop(e))
-                         .thenApply(v -> null);
-    }
-
-    @Override
-    public CompletableFuture<Void> addListener(AtomicValueEventListener<V> listener) {
-        checkNotNull(listener, "listener must not be null");
-        final MeteringAgent.Context newTimer = monitor.startTimer(ADD_LISTENER);
-        MapEventListener<String, byte[]> mapListener =
-                listeners.computeIfAbsent(listener, key -> new InternalMapValueEventListener(listener));
-        return backingMap.addListener(mapListener).whenComplete((r, e) -> newTimer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(AtomicValueEventListener<V> listener) {
-        checkNotNull(listener, "listener must not be null");
-        final MeteringAgent.Context newTimer = monitor.startTimer(REMOVE_LISTENER);
-        MapEventListener<String, byte[]> mapListener = listeners.remove(listener);
-        if (mapListener != null) {
-            return backingMap.removeListener(mapListener)
-                             .whenComplete((r, e) -> newTimer.stop(e));
-        } else {
-            newTimer.stop(null);
-            return CompletableFuture.completedFuture(null);
-        }
-    }
-
-    private class InternalMapValueEventListener implements MapEventListener<String, byte[]> {
-
-        private final AtomicValueEventListener<V> listener;
-
-        InternalMapValueEventListener(AtomicValueEventListener<V> listener) {
-            this.listener = listener;
-        }
-
-        @Override
-        public void event(MapEvent<String, byte[]> event) {
-            if (event.key().equals(name)) {
-                final MeteringAgent.Context newTimer = monitor.startTimer(NOTIFY_LISTENER);
-                byte[] rawNewValue = Versioned.valueOrNull(event.newValue());
-                byte[] rawOldValue = Versioned.valueOrNull(event.oldValue());
-
-                try {
-                    listener.event(new AtomicValueEvent<>(name,
-                            rawNewValue == null ? null : serializer.decode(rawNewValue),
-                            rawOldValue == null ? null : serializer.decode(rawOldValue)));
-                    newTimer.stop(null);
-                } catch (Exception e) {
-                    newTimer.stop(e);
-                    throw new IllegalStateException(e.getCause());
-                }
-            }
-        }
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAsyncDistributedSet.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAsyncDistributedSet.java
deleted file mode 100644
index d79c618..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAsyncDistributedSet.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.Collection;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.stream.Collectors;
-
-import org.onlab.util.Tools;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.AsyncDistributedSet;
-import org.onosproject.store.service.MapEvent;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.SetEvent;
-import org.onosproject.store.service.SetEventListener;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import org.onosproject.utils.MeteringAgent;
-
-/**
- * Implementation of {@link AsyncDistributedSet}.
- *
- * @param <E> set entry type
- */
-public class DefaultAsyncDistributedSet<E> implements AsyncDistributedSet<E> {
-
-    private static final String CONTAINS = "contains";
-    private static final String PRIMITIVE_NAME = "distributedSet";
-    private static final String SIZE = "size";
-    private static final String IS_EMPTY = "isEmpty";
-    private static final String ADD = "add";
-    private static final String REMOVE = "remove";
-    private static final String CONTAINS_ALL = "containsAll";
-    private static final String ADD_ALL = "addAll";
-    private static final String RETAIN_ALL = "retainAll";
-    private static final String REMOVE_ALL = "removeAll";
-    private static final String CLEAR = "clear";
-    private static final String GET_AS_IMMUTABLE_SET = "getAsImmutableSet";
-
-    private final String name;
-    private final AsyncConsistentMap<E, Boolean> backingMap;
-    private final Map<SetEventListener<E>, MapEventListener<E, Boolean>> listenerMapping = Maps.newIdentityHashMap();
-    private final MeteringAgent monitor;
-
-    public DefaultAsyncDistributedSet(AsyncConsistentMap<E, Boolean> backingMap, String name, boolean meteringEnabled) {
-        this.backingMap = backingMap;
-        this.name = name;
-        monitor = new MeteringAgent(PRIMITIVE_NAME, name, meteringEnabled);
-    }
-
-    @Override
-    public String name() {
-        return name;
-    }
-
-    @Override
-    public CompletableFuture<Integer> size() {
-        final MeteringAgent.Context timer = monitor.startTimer(SIZE);
-        return backingMap.size().whenComplete((r, e) -> timer.stop(null));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> isEmpty() {
-        final MeteringAgent.Context timer = monitor.startTimer(IS_EMPTY);
-        return backingMap.isEmpty().whenComplete((r, e) -> timer.stop(null));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> contains(E element) {
-        final MeteringAgent.Context timer = monitor.startTimer(CONTAINS);
-        return backingMap.containsKey(element).whenComplete((r, e) -> timer.stop(null));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> add(E entry) {
-        final MeteringAgent.Context timer = monitor.startTimer(ADD);
-        return backingMap.putIfAbsent(entry, true).thenApply(Objects::isNull).whenComplete((r, e) -> timer.stop(null));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(E entry) {
-        final MeteringAgent.Context timer = monitor.startTimer(REMOVE);
-        return backingMap.remove(entry, true).whenComplete((r, e) -> timer.stop(null));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsAll(Collection<? extends E> c) {
-        final MeteringAgent.Context timer = monitor.startTimer(CONTAINS_ALL);
-        return Tools.allOf(c.stream().map(this::contains).collect(Collectors.toList())).thenApply(v ->
-            v.stream().reduce(Boolean::logicalAnd).orElse(true)).whenComplete((r, e) -> timer.stop(null));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> addAll(Collection<? extends E> c) {
-        final MeteringAgent.Context timer = monitor.startTimer(ADD_ALL);
-        return Tools.allOf(c.stream().map(this::add).collect(Collectors.toList())).thenApply(v ->
-            v.stream().reduce(Boolean::logicalOr).orElse(false)).whenComplete((r, e) -> timer.stop(null));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> retainAll(Collection<? extends E> c) {
-        final MeteringAgent.Context timer = monitor.startTimer(RETAIN_ALL);
-        return backingMap.keySet().thenApply(set -> Sets.difference(set, Sets.newHashSet(c)))
-                                  .thenCompose(this::removeAll)
-                                  .whenComplete((r, e) -> timer.stop(null));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> removeAll(Collection<? extends E> c) {
-        final MeteringAgent.Context timer = monitor.startTimer(REMOVE_ALL);
-        return Tools.allOf(c.stream().map(this::remove).collect(Collectors.toList())).thenApply(v ->
-            v.stream().reduce(Boolean::logicalOr).orElse(false)).whenComplete((r, e) -> timer.stop(null));
-    }
-
-    @Override
-    public CompletableFuture<Void> clear() {
-        final MeteringAgent.Context timer = monitor.startTimer(CLEAR);
-        return backingMap.clear().whenComplete((r, e) -> timer.stop(null));
-    }
-
-    @Override
-    public CompletableFuture<? extends Set<E>> getAsImmutableSet() {
-        final MeteringAgent.Context timer = monitor.startTimer(GET_AS_IMMUTABLE_SET);
-        return backingMap.keySet().thenApply(s -> ImmutableSet.copyOf(s)).whenComplete((r, e) -> timer.stop(null));
-    }
-
-    @Override
-    public CompletableFuture<Void> addListener(SetEventListener<E> listener) {
-        MapEventListener<E, Boolean> mapEventListener = mapEvent -> {
-            if (mapEvent.type() == MapEvent.Type.INSERT) {
-                listener.event(new SetEvent<>(name, SetEvent.Type.ADD, mapEvent.key()));
-            } else if (mapEvent.type() == MapEvent.Type.REMOVE) {
-                listener.event(new SetEvent<>(name, SetEvent.Type.REMOVE, mapEvent.key()));
-            }
-        };
-        if (listenerMapping.putIfAbsent(listener, mapEventListener) == null) {
-            return backingMap.addListener(mapEventListener);
-        }
-        return CompletableFuture.completedFuture(null);
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(SetEventListener<E> listener) {
-        MapEventListener<E, Boolean> mapEventListener = listenerMapping.remove(listener);
-        if (mapEventListener != null) {
-            return backingMap.removeListener(mapEventListener);
-        }
-        return CompletableFuture.completedFuture(null);
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicCounterBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicCounterBuilder.java
deleted file mode 100644
index 11a0724..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicCounterBuilder.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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 DefaultAtomicCounterBuilder extends AtomicCounterBuilder {
-
-    private final DistributedPrimitiveCreator primitiveCreator;
-
-    public DefaultAtomicCounterBuilder(DistributedPrimitiveCreator primitiveCreator) {
-        this.primitiveCreator = primitiveCreator;
-    }
-
-    @Override
-    public AsyncAtomicCounter build() {
-        return primitiveCreator.newAsyncCounter(name());
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicCounterMapBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicCounterMapBuilder.java
deleted file mode 100644
index d5a143a..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicCounterMapBuilder.java
+++ /dev/null
@@ -1,43 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.AsyncAtomicCounterMap;
-import org.onosproject.store.service.AtomicCounterMap;
-import org.onosproject.store.service.AtomicCounterMapBuilder;
-
-/**
- * Default {@code AtomicCounterMapBuilder}.
- */
-public class DefaultAtomicCounterMapBuilder<K> extends AtomicCounterMapBuilder<K> {
-
-    private final DistributedPrimitiveCreator primitiveCreator;
-
-    public DefaultAtomicCounterMapBuilder(DistributedPrimitiveCreator primitiveCreator) {
-        this.primitiveCreator = primitiveCreator;
-    }
-
-    @Override
-    public AsyncAtomicCounterMap<K> buildAsyncMap() {
-        return primitiveCreator.newAsyncAtomicCounterMap(name(), serializer());
-    }
-
-    @Override
-    public AtomicCounterMap<K> build() {
-        return buildAsyncMap().asAtomicCounterMap();
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicIdGeneratorBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicIdGeneratorBuilder.java
deleted file mode 100644
index c9e8486..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicIdGeneratorBuilder.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.AsyncAtomicIdGenerator;
-import org.onosproject.store.service.AtomicIdGeneratorBuilder;
-
-/**
- * Default implementation of AtomicIdGeneratorBuilder.
- */
-public class DefaultAtomicIdGeneratorBuilder extends AtomicIdGeneratorBuilder {
-
-    private final DistributedPrimitiveCreator primitiveCreator;
-
-    public DefaultAtomicIdGeneratorBuilder(DistributedPrimitiveCreator primitiveCreator) {
-        this.primitiveCreator = primitiveCreator;
-    }
-
-    @Override
-    public AsyncAtomicIdGenerator build() {
-        return primitiveCreator.newAsyncIdGenerator(name());
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicValueBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicValueBuilder.java
deleted file mode 100644
index 9058189..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultAtomicValueBuilder.java
+++ /dev/null
@@ -1,63 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.function.Supplier;
-
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncAtomicValue;
-import org.onosproject.store.service.AtomicValueBuilder;
-import org.onosproject.store.service.ConsistentMapBuilder;
-import org.onosproject.store.service.Serializer;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * Default implementation of AtomicValueBuilder.
- *
- * @param <V> value type
- */
-public class DefaultAtomicValueBuilder<V> extends AtomicValueBuilder<V> {
-
-    private ConsistentMapBuilder<String, byte[]> mapBuilder;
-
-    public DefaultAtomicValueBuilder(Supplier<ConsistentMapBuilder<String, byte[]>> mapBuilderSupplier) {
-        mapBuilder = mapBuilderSupplier.get();
-    }
-
-    @Override
-    public AsyncAtomicValue<V> build() {
-        if (compatibilityFunction != null) {
-            Serializer serializer = Serializer.using(KryoNamespaces.API, CompatibleValue.class);
-
-            AsyncAtomicValue<CompatibleValue<byte[]>> rawValue = new DefaultAsyncAtomicValue<>(
-                checkNotNull(name()), serializer, mapBuilder.buildAsyncMap());
-
-            AsyncAtomicValue<CompatibleValue<V>> compatibleValue =
-                DistributedPrimitives.newTranscodingAtomicValue(
-                    rawValue,
-                    value -> value == null ? null :
-                        new CompatibleValue<byte[]>(serializer().encode(value.value()), value.version()),
-                    value -> value == null ? null :
-                        new CompatibleValue<V>(serializer().decode(value.value()), value.version()));
-            return DistributedPrimitives.newCompatibleAtomicValue(compatibleValue, compatibilityFunction, version());
-        }
-        return new DefaultAsyncAtomicValue<>(
-            checkNotNull(name()),
-            checkNotNull(serializer()),
-            mapBuilder.buildAsyncMap());
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultConsistentMapBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultConsistentMapBuilder.java
deleted file mode 100644
index d6b9441..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultConsistentMapBuilder.java
+++ /dev/null
@@ -1,79 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.ConsistentMap;
-import org.onosproject.store.service.ConsistentMapBuilder;
-import org.onosproject.store.service.Serializer;
-
-/**
- * Default {@link AsyncConsistentMap} builder.
- *
- * @param <K> type for map key
- * @param <V> type for map value
- */
-public class DefaultConsistentMapBuilder<K, V> extends ConsistentMapBuilder<K, V> {
-
-    private final DistributedPrimitiveCreator primitiveCreator;
-
-    public DefaultConsistentMapBuilder(DistributedPrimitiveCreator primitiveCreator) {
-        this.primitiveCreator = primitiveCreator;
-    }
-
-    @Override
-    public ConsistentMap<K, V> build() {
-        return buildAsyncMap().asConsistentMap();
-    }
-
-    @Override
-    public AsyncConsistentMap<K, V> buildAsyncMap() {
-        AsyncConsistentMap<K, V> map;
-
-        // If a compatibility function is defined, we don't assume CompatibleValue and Version is registered in
-        // the user-provided serializer since it's an implementation detail. Instead, we use the user-provided
-        // serializer to convert the CompatibleValue value to a raw byte[] and use a separate serializer to encode
-        // the CompatibleValue to binary.
-        if (compatibilityFunction != null) {
-            Serializer serializer = serializer();
-
-            // Convert the byte[] value to CompatibleValue<byte[]>
-            AsyncConsistentMap<K, CompatibleValue<byte[]>> rawMap = primitiveCreator.newAsyncConsistentMap(
-                withSerializer(Serializer.using(KryoNamespaces.API, CompatibleValue.class)));
-
-            // Convert the CompatibleValue<byte[]> value to CompatibleValue<V> using the user-provided serializer.
-            AsyncConsistentMap<K, CompatibleValue<V>> compatibleMap =
-                DistributedPrimitives.newTranscodingMap(
-                    rawMap,
-                    key -> key,
-                    key -> key,
-                    value -> value == null ? null :
-                        new CompatibleValue<byte[]>(serializer.encode(value.value()), value.version()),
-                    value -> value == null ? null :
-                        new CompatibleValue<V>(serializer.decode(value.value()), value.version()));
-            map = DistributedPrimitives.newCompatibleMap(compatibleMap, compatibilityFunction, version());
-        } else {
-            map = primitiveCreator.newAsyncConsistentMap(name(), serializer());
-        }
-
-        map = nullValues() ? map : DistributedPrimitives.newNotNullMap(map);
-        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/DefaultConsistentMultimapBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultConsistentMultimapBuilder.java
deleted file mode 100644
index 18ea1f1..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultConsistentMultimapBuilder.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Copyright 2016 Open Networking Foundation
- *
- * 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.AsyncConsistentMultimap;
-import org.onosproject.store.service.ConsistentMultimap;
-import org.onosproject.store.service.ConsistentMultimapBuilder;
-
-/**
- * Default {@link AsyncConsistentMultimap} builder.
- */
-public class DefaultConsistentMultimapBuilder<K, V>
-    extends ConsistentMultimapBuilder<K, V> {
-
-    private final DistributedPrimitiveCreator primitiveCreator;
-
-    public DefaultConsistentMultimapBuilder(
-        DistributedPrimitiveCreator primitiveCreator) {
-        this.primitiveCreator = primitiveCreator;
-    }
-
-    @Override
-    public AsyncConsistentMultimap<K, V> buildMultimap() {
-        return primitiveCreator.newAsyncConsistentSetMultimap(name(), serializer());
-    }
-
-    @Override
-    public ConsistentMultimap<K, V> build() {
-        AsyncConsistentMultimap<K, V> multimap = buildMultimap();
-        if (relaxedReadConsistency()) {
-            multimap = new CachingAsyncConsistentMultimap<>(multimap);
-        }
-        return multimap.asMultimap();
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultConsistentTreeMapBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultConsistentTreeMapBuilder.java
deleted file mode 100644
index b80d93e..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultConsistentTreeMapBuilder.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.AsyncConsistentTreeMap;
-import org.onosproject.store.service.ConsistentTreeMap;
-import org.onosproject.store.service.ConsistentTreeMapBuilder;
-
-/**
- * Default {@link org.onosproject.store.service.AsyncConsistentTreeMap} builder.
- *
- * @param <V> type for map value
- */
-public class DefaultConsistentTreeMapBuilder<V> extends ConsistentTreeMapBuilder<V> {
-
-    private final DistributedPrimitiveCreator primitiveCreator;
-
-    public DefaultConsistentTreeMapBuilder(DistributedPrimitiveCreator primitiveCreator) {
-        this.primitiveCreator = primitiveCreator;
-    }
-
-    @Override
-    public AsyncConsistentTreeMap<V> buildTreeMap() {
-        return primitiveCreator.newAsyncConsistentTreeMap(name(), serializer());
-    }
-
-    @Override
-    public ConsistentTreeMap<V> build() {
-        return buildTreeMap().asTreeMap();
-    }
-
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedDocumentTree.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedDocumentTree.java
deleted file mode 100644
index 8d4f9c3..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedDocumentTree.java
+++ /dev/null
@@ -1,179 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.Map;
-import java.util.concurrent.CompletableFuture;
-
-import org.onosproject.store.primitives.NodeUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.AsyncDocumentTree;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTreeEvent;
-import org.onosproject.store.service.DocumentTreeListener;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-import com.google.common.collect.Maps;
-
-/**
- * Default implementation of {@link AsyncDocumentTree}.
- * <p>
- * This implementation delegates execution to a backing tree implemented on top of Atomix framework.
- *
- * @param <V> tree node value type.
- */
-public class DefaultDistributedDocumentTree<V> implements AsyncDocumentTree<V> {
-
-    private final String name;
-    private final AsyncDocumentTree<byte[]> backingTree;
-    private final Serializer serializer;
-    private final Map<DocumentTreeListener<V>, InternalBackingDocumentTreeListener> listeners =
-            Maps.newIdentityHashMap();
-
-    DefaultDistributedDocumentTree(String name, AsyncDocumentTree<byte[]> backingTree, Serializer serializer) {
-        this.name = name;
-        this.backingTree = backingTree;
-        this.serializer = serializer;
-    }
-
-    @Override
-    public String name() {
-        return name;
-    }
-
-    @Override
-    public Type primitiveType() {
-        return backingTree.primitiveType();
-    }
-
-    @Override
-    public DocumentPath root() {
-        return backingTree.root();
-    }
-
-    @Override
-    public CompletableFuture<Map<String, Versioned<V>>> getChildren(DocumentPath path) {
-        return backingTree.getChildren(path)
-                          .thenApply(map -> Maps.transformValues(map, v -> v.map(serializer::decode)));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> get(DocumentPath path) {
-        return backingTree.get(path)
-                          .thenApply(v -> v == null ? null : v.map(serializer::decode));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> set(DocumentPath path, V value) {
-        return backingTree.set(path, serializer.encode(value))
-                          .thenApply(v -> v == null ? null : v.map(serializer::decode));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> create(DocumentPath path, V value) {
-        return backingTree.create(path, serializer.encode(value));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> createRecursive(DocumentPath path, V value) {
-        return backingTree.createRecursive(path, serializer.encode(value));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(DocumentPath path, V newValue, long version) {
-        return backingTree.replace(path, serializer.encode(newValue), version);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(DocumentPath path, V newValue, V currentValue) {
-        return backingTree.replace(path, serializer.encode(newValue), serializer.encode(currentValue));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> removeNode(DocumentPath path) {
-        return backingTree.removeNode(path)
-                          .thenApply(v -> v == null ? null : v.map(serializer::decode));
-    }
-
-
-    @Override
-    public CompletableFuture<Version> begin(TransactionId transactionId) {
-        return backingTree.begin(transactionId);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepare(TransactionLog<NodeUpdate<V>> transactionLog) {
-        return backingTree.prepare(transactionLog.map(record -> record.map(serializer::encode)));
-    }
-
-    @Override
-    public CompletableFuture<Void> commit(TransactionId transactionId) {
-        return backingTree.commit(transactionId);
-    }
-
-    @Override
-    public CompletableFuture<Void> rollback(TransactionId transactionId) {
-        return backingTree.rollback(transactionId);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<NodeUpdate<V>> transactionLog) {
-        return backingTree.prepareAndCommit(transactionLog.map(record -> record.map(serializer::encode)));
-    }
-
-
-    @Override
-    public CompletableFuture<Void> addListener(DocumentPath path, DocumentTreeListener<V> listener) {
-        synchronized (listeners) {
-            InternalBackingDocumentTreeListener backingListener =
-                    listeners.computeIfAbsent(listener, k -> new InternalBackingDocumentTreeListener(listener));
-            return backingTree.addListener(path, backingListener);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(DocumentTreeListener<V> listener) {
-        synchronized (listeners) {
-            InternalBackingDocumentTreeListener backingListener = listeners.remove(listener);
-            if (backingListener != null) {
-                return backingTree.removeListener(backingListener);
-            } else {
-                return CompletableFuture.completedFuture(null);
-            }
-        }
-    }
-
-    private class InternalBackingDocumentTreeListener implements DocumentTreeListener<byte[]> {
-
-        private final DocumentTreeListener<V> listener;
-
-        InternalBackingDocumentTreeListener(DocumentTreeListener<V> listener) {
-            this.listener = listener;
-        }
-
-        @Override
-        public void event(DocumentTreeEvent<byte[]> event) {
-            listener.event(new DocumentTreeEvent<V>(event.path(),
-                    event.type(),
-                    event.newValue().map(v -> v.map(serializer::decode)),
-                    event.oldValue().map(v -> v.map(serializer::decode))));
-        }
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedLockBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedLockBuilder.java
deleted file mode 100644
index fffebcc..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedLockBuilder.java
+++ /dev/null
@@ -1,37 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * 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.AsyncDistributedLock;
-import org.onosproject.store.service.DistributedLockBuilder;
-
-/**
- * Default implementation of DistributedLockBuilder.
- */
-public class DefaultDistributedLockBuilder extends DistributedLockBuilder {
-
-    private final DistributedPrimitiveCreator primitiveCreator;
-
-    public DefaultDistributedLockBuilder(DistributedPrimitiveCreator primitiveCreator) {
-        this.primitiveCreator = primitiveCreator;
-    }
-
-    @Override
-    public AsyncDistributedLock build() {
-        return primitiveCreator.newAsyncDistributedLock(name());
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedSetBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedSetBuilder.java
deleted file mode 100644
index a601239..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedSetBuilder.java
+++ /dev/null
@@ -1,89 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.function.Supplier;
-
-import org.onosproject.core.ApplicationId;
-import org.onosproject.store.service.AsyncDistributedSet;
-import org.onosproject.store.service.ConsistentMapBuilder;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.DistributedSetBuilder;
-
-/**
- * Default distributed set builder.
- *
- * @param <E> type for set elements
- */
-public class DefaultDistributedSetBuilder<E> extends DistributedSetBuilder<E> {
-
-    private String name;
-    private ConsistentMapBuilder<E, Boolean>  mapBuilder;
-    private boolean metering = true;
-
-    public DefaultDistributedSetBuilder(Supplier<ConsistentMapBuilder<E, Boolean>> mapBuilderSupplier) {
-        this.mapBuilder = mapBuilderSupplier.get();
-        mapBuilder.withMeteringDisabled();
-    }
-
-    @Override
-    public DistributedSetBuilder<E> withName(String name) {
-        mapBuilder.withName(name);
-        this.name = name;
-        return this;
-    }
-
-    @Override
-    public DistributedSetBuilder<E> withApplicationId(ApplicationId id) {
-        mapBuilder.withApplicationId(id);
-        return this;
-    }
-
-    @Override
-    public DistributedSetBuilder<E> withPurgeOnUninstall() {
-        mapBuilder.withPurgeOnUninstall();
-        return this;
-    }
-
-    @Override
-    public DistributedSetBuilder<E> withSerializer(Serializer serializer) {
-        mapBuilder.withSerializer(serializer);
-        return this;
-    }
-
-    @Override
-    public DistributedSetBuilder<E> withUpdatesDisabled() {
-        mapBuilder.withUpdatesDisabled();
-        return this;
-    }
-
-    @Override
-    public DistributedSetBuilder<E> withRelaxedReadConsistency() {
-        mapBuilder.withRelaxedReadConsistency();
-        return this;
-    }
-
-    @Override
-    public DistributedSetBuilder<E> withMeteringDisabled() {
-        metering = false;
-        return this;
-    }
-
-    @Override
-    public AsyncDistributedSet<E> build() {
-        return new DefaultAsyncDistributedSet<E>(mapBuilder.buildAsyncMap(), name, metering);
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedTopicBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedTopicBuilder.java
deleted file mode 100644
index 21ca7a1..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedTopicBuilder.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * 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.function.BiFunction;
-
-import org.onosproject.core.Version;
-import org.onosproject.store.service.AtomicValueBuilder;
-import org.onosproject.store.service.RevisionType;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.Topic;
-import org.onosproject.store.service.TopicBuilder;
-
-/**
- * Default topic builder.
- */
-public class DefaultDistributedTopicBuilder<T> extends TopicBuilder<T> {
-    private final AtomicValueBuilder<T> valueBuilder;
-
-    public DefaultDistributedTopicBuilder(AtomicValueBuilder<T> valueBuilder) {
-        this.valueBuilder = valueBuilder;
-    }
-
-    @Override
-    public TopicBuilder<T> withName(String name) {
-        valueBuilder.withName(name);
-        return this;
-    }
-
-    @Override
-    public TopicBuilder<T> withSerializer(Serializer serializer) {
-        valueBuilder.withSerializer(serializer);
-        return this;
-    }
-
-    @Override
-    public TopicBuilder<T> withVersion(Version version) {
-        valueBuilder.withVersion(version);
-        return this;
-    }
-
-    @Override
-    public TopicBuilder<T> withRevisionType(RevisionType revisionType) {
-        valueBuilder.withRevisionType(revisionType);
-        return this;
-    }
-
-    @Override
-    public TopicBuilder<T> withCompatibilityFunction(BiFunction<T, Version, T> compatibilityFunction) {
-        valueBuilder.withCompatibilityFunction(compatibilityFunction);
-        return this;
-    }
-
-    @Override
-    public Topic<T> build() {
-        return new DefaultDistributedTopic<>(valueBuilder.build());
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedWorkQueue.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedWorkQueue.java
deleted file mode 100644
index 11895db..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDistributedWorkQueue.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.ArrayList;
-import java.util.Collection;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.Consumer;
-import java.util.stream.Collectors;
-
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.Task;
-import org.onosproject.store.service.WorkQueue;
-import org.onosproject.store.service.WorkQueueStats;
-
-import com.google.common.collect.Collections2;
-
-/**
- * Default implementation of {@link WorkQueue}.
- *
- * @param <E> task payload type.
- */
-public class DefaultDistributedWorkQueue<E> implements WorkQueue<E> {
-
-    private final WorkQueue<byte[]> backingQueue;
-    private final Serializer serializer;
-
-    public DefaultDistributedWorkQueue(WorkQueue<byte[]> backingQueue, Serializer serializer) {
-        this.backingQueue = backingQueue;
-        this.serializer = serializer;
-    }
-
-    @Override
-    public String name() {
-        return backingQueue.name();
-    }
-
-    @Override
-    public CompletableFuture<Void> addMultiple(Collection<E> items) {
-        return backingQueue.addMultiple(items.stream()
-                                             .map(serializer::encode)
-                                             .collect(Collectors.toCollection(ArrayList::new)));
-    }
-
-    private Collection<Task<E>> decodeCollection(Collection<Task<byte[]>> tasks) {
-        return Collections2.transform(tasks, task -> task.map(serializer::decode));
-    }
-
-    @Override
-    public CompletableFuture<Collection<Task<E>>> take(int maxTasks) {
-        return backingQueue.take(maxTasks)
-                           .thenApply(this::decodeCollection);
-    }
-
-    @Override
-    public CompletableFuture<Void> complete(Collection<String> ids) {
-        return backingQueue.complete(ids);
-    }
-
-    @Override
-    public CompletableFuture<WorkQueueStats> stats() {
-        return backingQueue.stats();
-    }
-
-    @Override
-    public CompletableFuture<Void> registerTaskProcessor(Consumer<E> callback,
-                                                         int parallelism,
-                                                         Executor executor) {
-        Consumer<byte[]> backingQueueCallback = payload -> callback.accept(serializer.decode(payload));
-        return backingQueue.registerTaskProcessor(backingQueueCallback, parallelism, executor);
-    }
-
-    @Override
-    public CompletableFuture<Void> stopProcessing() {
-        return backingQueue.stopProcessing();
-    }
-
-    @Override
-    public CompletableFuture<Void> destroy() {
-        return backingQueue.destroy();
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDocumentTreeBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDocumentTreeBuilder.java
deleted file mode 100644
index f0da9ab..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultDocumentTreeBuilder.java
+++ /dev/null
@@ -1,54 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.AsyncDocumentTree;
-import org.onosproject.store.service.DocumentTreeBuilder;
-
-/**
- * Default {@link AsyncDocumentTree} builder.
- *
- * @param <V> type for document tree value
- */
-public class DefaultDocumentTreeBuilder<V> extends DocumentTreeBuilder<V> {
-
-    private final DistributedPrimitiveCreator primitiveCreator;
-
-    public DefaultDocumentTreeBuilder(DistributedPrimitiveCreator primitiveCreator) {
-        this.primitiveCreator = primitiveCreator;
-    }
-
-    @Override
-    public AsyncDocumentTree<V> buildDocumentTree() {
-        return primitiveCreator.newAsyncDocumentTree(name(), serializer(), ordering());
-    }
-
-    //TODO
-    /*
-     public ConsistentDocumentTree<V> build() {
-        return buildDocumentTree().asDocumentTree();
-    }
-    }*/
-    //writing a dummy implementation till we have ConsistentDocumentTree.
-    @Deprecated
-    @Override
-    public AsyncDocumentTree<V> build() {
-        AsyncDocumentTree<V> tree = primitiveCreator.newAsyncDocumentTree(name(), serializer(), ordering());
-        tree = relaxedReadConsistency() ? DistributedPrimitives.newCachingDocumentTree(tree) : tree;
-        return tree;
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultLeaderElectorBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultLeaderElectorBuilder.java
deleted file mode 100644
index 21ddf03..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultLeaderElectorBuilder.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.AsyncLeaderElector;
-import org.onosproject.store.service.LeaderElectorBuilder;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * Default implementation of {@code LeaderElectorBuilder}.
- */
-public class DefaultLeaderElectorBuilder extends LeaderElectorBuilder {
-
-    private final DistributedPrimitiveCreator primitiveCreator;
-
-    public DefaultLeaderElectorBuilder(DistributedPrimitiveCreator primitiveCreator) {
-        this.primitiveCreator = primitiveCreator;
-    }
-
-    @Override
-    public AsyncLeaderElector build() {
-        AsyncLeaderElector leaderElector = primitiveCreator.newAsyncLeaderElector(name(), electionTimeoutMillis(),
-                                                                                  TimeUnit.MILLISECONDS);
-        if (relaxedReadConsistency()) {
-            leaderElector = new CachingAsyncLeaderElector(leaderElector);
-        }
-        return leaderElector;
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionContext.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionContext.java
deleted file mode 100644
index 19ea562..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionContext.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.concurrent.CompletableFuture;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.CommitStatus;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionContext;
-import org.onosproject.store.service.TransactionalMap;
-import org.onosproject.utils.MeteringAgent;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static com.google.common.base.MoreObjects.toStringHelper;
-import static com.google.common.base.Preconditions.checkState;
-
-/**
- * Default implementation of transaction context.
- */
-public class DefaultTransactionContext implements TransactionContext {
-    private final Logger log = LoggerFactory.getLogger(getClass());
-    private final AtomicBoolean isOpen = new AtomicBoolean(false);
-    private final TransactionId transactionId;
-    private final TransactionCoordinator transactionCoordinator;
-    private final MeteringAgent monitor;
-
-    public DefaultTransactionContext(TransactionId transactionId, TransactionCoordinator transactionCoordinator) {
-        this.transactionId = transactionId;
-        this.transactionCoordinator = transactionCoordinator;
-        this.monitor = new MeteringAgent("transactionContext", "*", true);
-    }
-
-    @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)) {
-            log.trace("Opened transaction {}", transactionId);
-        } else {
-            throw new IllegalStateException("TransactionContext is already open");
-        }
-    }
-
-    @Override
-    public CompletableFuture<CommitStatus> commit() {
-        checkState(isOpen.get(), "Transaction not open");
-        final MeteringAgent.Context timer = monitor.startTimer("commit");
-        log.debug("Committing transaction {}", transactionId);
-        return transactionCoordinator.commit().whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public void abort() {
-        if (isOpen.compareAndSet(true, false)) {
-            log.debug("Aborted transaction {}", transactionId);
-        }
-    }
-
-    @Override
-    public <K, V> TransactionalMap<K, V> getTransactionalMap(String mapName, Serializer serializer) {
-        return transactionCoordinator.getTransactionalMap(mapName, serializer);
-    }
-
-    @Override
-    public String toString() {
-        return toStringHelper(this)
-                .add("transactionId", transactionId)
-                .toString();
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionContextBuilder.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionContextBuilder.java
deleted file mode 100644
index ecbb1d2..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionContextBuilder.java
+++ /dev/null
@@ -1,40 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.TransactionId;
-import org.onosproject.store.service.TransactionContext;
-import org.onosproject.store.service.TransactionContextBuilder;
-
-/**
- * Default Transaction Context Builder.
- */
-public class DefaultTransactionContextBuilder extends TransactionContextBuilder {
-
-    private final TransactionId transactionId;
-    private final TransactionManager transactionManager;
-
-    public DefaultTransactionContextBuilder(TransactionId transactionId, TransactionManager transactionManager) {
-        this.transactionId = transactionId;
-        this.transactionManager = transactionManager;
-    }
-
-    @Override
-    public TransactionContext build() {
-        return new DefaultTransactionContext(transactionId,
-                new TransactionCoordinator(transactionId, transactionManager));
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionalMapParticipant.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionalMapParticipant.java
deleted file mode 100644
index 14346ce..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionalMapParticipant.java
+++ /dev/null
@@ -1,84 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.Map;
-import java.util.stream.Stream;
-
-import com.google.common.collect.Maps;
-import org.apache.commons.lang3.tuple.Pair;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.service.ConsistentMap;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-/**
- * Repeatable read based map participant.
- */
-public class DefaultTransactionalMapParticipant<K, V> extends TransactionalMapParticipant<K, V> {
-    private final Map<K, Versioned<V>> readCache = Maps.newConcurrentMap();
-
-    public DefaultTransactionalMapParticipant(
-            ConsistentMap<K, V> backingMap, Transaction<MapUpdate<K, V>> transaction) {
-        super(backingMap, transaction);
-    }
-
-    @Override
-    protected V read(K key) {
-        Versioned<V> value = backingMap.getOrDefault(key, null);
-        readCache.put(key, value);
-        return value.value();
-    }
-
-    @Override
-    public boolean hasPendingUpdates() {
-        return !writeCache.isEmpty() || !deleteSet.isEmpty();
-    }
-
-    @Override
-    protected Stream<MapUpdate<K, V>> records(Version lockVersion) {
-        return Stream.concat(deleteStream(), writeStream(lockVersion));
-    }
-
-    /**
-     * Returns a transaction record stream for deleted keys.
-     */
-    private Stream<MapUpdate<K, V>> deleteStream() {
-        return deleteSet.stream()
-                .map(key -> Pair.of(key, readCache.get(key)))
-                .map(e -> MapUpdate.<K, V>newBuilder()
-                        .withType(MapUpdate.Type.REMOVE_IF_VERSION_MATCH)
-                        .withKey(e.getKey())
-                        .withVersion(e.getValue().version())
-                        .build());
-    }
-
-    /**
-     * Returns a transaction record stream for updated keys.
-     */
-    private Stream<MapUpdate<K, V>> writeStream(Version lockVersion) {
-        return writeCache.entrySet().stream()
-                .map(entry -> {
-                    Versioned<V> original = readCache.get(entry.getKey());
-                    return MapUpdate.<K, V>newBuilder()
-                            .withType(MapUpdate.Type.PUT_IF_VERSION_MATCH)
-                            .withKey(entry.getKey())
-                            .withValue(entry.getValue())
-                            .withVersion(Math.max(original.version(), lockVersion.value()))
-                            .build();
-                });
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionalSet.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionalSet.java
deleted file mode 100644
index a2bac21..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DefaultTransactionalSet.java
+++ /dev/null
@@ -1,58 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.service.Serializer;
-import org.onosproject.store.service.TransactionContext;
-import org.onosproject.store.service.TransactionalMap;
-import org.onosproject.store.service.TransactionalSet;
-
-/**
- * Default TransactionalSet implementation that provides a repeatable reads
- * transaction isolation level.
- *
- * @param <E> element type.
- */
-public class DefaultTransactionalSet<E> implements TransactionalSet<E> {
-
-    private TransactionalMap<E, Boolean> map;
-
-    // dummy value to associate with an Object in the backing map
-    private static final Boolean PRESENT = Boolean.TRUE;
-
-    public DefaultTransactionalSet(
-            String name,
-            TransactionContext txContext,
-            Serializer serializer) {
-        map = txContext.getTransactionalMap(name, serializer);
-    }
-
-    @Override
-    public boolean add(E e) {
-        return map.put(e, PRESENT) == null;
-    }
-
-    @Override
-    public boolean remove(E e) {
-        return map.remove(e) != null;
-    }
-
-    @Override
-    public boolean contains(E e) {
-        return map.get(e) != null;
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncConsistentMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncConsistentMap.java
deleted file mode 100644
index 193a250..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncConsistentMap.java
+++ /dev/null
@@ -1,228 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.Collection;
-import java.util.Map.Entry;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.BiFunction;
-import java.util.function.Consumer;
-import java.util.function.Predicate;
-
-import com.google.common.base.MoreObjects;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-/**
- * {@code AsyncConsistentMap} that merely delegates control to
- * another AsyncConsistentMap.
- *
- * @param <K> key type
- * @param <V> value type
- */
-public class DelegatingAsyncConsistentMap<K, V>
-        extends DelegatingDistributedPrimitive implements AsyncConsistentMap<K, V> {
-
-    private final AsyncConsistentMap<K, V> delegateMap;
-
-    DelegatingAsyncConsistentMap(AsyncConsistentMap<K, V> delegateMap) {
-        super(delegateMap);
-        this.delegateMap = delegateMap;
-    }
-
-    @Override
-    public CompletableFuture<Integer> size() {
-        return delegateMap.size();
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsKey(K key) {
-        return delegateMap.containsKey(key);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsValue(V value) {
-        return delegateMap.containsValue(value);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> get(K key) {
-        return delegateMap.get(key);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> getOrDefault(K key, V defaultValue) {
-        return delegateMap.getOrDefault(key, defaultValue);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> computeIf(K key,
-            Predicate<? super V> condition,
-            BiFunction<? super K, ? super V, ? extends V> remappingFunction) {
-        return delegateMap.computeIf(key, condition, remappingFunction);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> put(K key, V value) {
-        return delegateMap.put(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> putAndGet(K key, V value) {
-        return delegateMap.putAndGet(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> remove(K key) {
-        return delegateMap.remove(key);
-    }
-
-    @Override
-    public CompletableFuture<Void> clear() {
-        return delegateMap.clear();
-    }
-
-    @Override
-    public CompletableFuture<Set<K>> keySet() {
-        return delegateMap.keySet();
-    }
-
-    @Override
-    public CompletableFuture<Collection<Versioned<V>>> values() {
-        return delegateMap.values();
-    }
-
-    @Override
-    public CompletableFuture<Set<Entry<K, Versioned<V>>>> entrySet() {
-        return delegateMap.entrySet();
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> putIfAbsent(K key, V value) {
-        return delegateMap.putIfAbsent(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(K key, V value) {
-        return delegateMap.remove(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(K key, long version) {
-        return delegateMap.remove(key, version);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> replace(K key, V value) {
-        return delegateMap.replace(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(K key, V oldValue, V newValue) {
-        return delegateMap.replace(key, oldValue, newValue);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(K key, long oldVersion, V newValue) {
-        return delegateMap.replace(key, oldVersion, newValue);
-    }
-
-    @Override
-    public CompletableFuture<AsyncIterator<Entry<K, Versioned<V>>>> iterator() {
-        return delegateMap.iterator();
-    }
-
-    @Override
-    public CompletableFuture<Void> addListener(MapEventListener<K, V> listener, Executor executor) {
-        return delegateMap.addListener(listener, executor);
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(MapEventListener<K, V> listener) {
-        return delegateMap.removeListener(listener);
-    }
-
-    @Override
-    public CompletableFuture<Version> begin(TransactionId transactionId) {
-        return delegateMap.begin(transactionId);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepare(TransactionLog<MapUpdate<K, V>> transactionLog) {
-        return delegateMap.prepare(transactionLog);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<MapUpdate<K, V>> transactionLog) {
-        return delegateMap.prepareAndCommit(transactionLog);
-    }
-
-    @Override
-    public CompletableFuture<Void> commit(TransactionId transactionId) {
-        return delegateMap.commit(transactionId);
-    }
-
-    @Override
-    public CompletableFuture<Void> rollback(TransactionId transactionId) {
-        return delegateMap.rollback(transactionId);
-    }
-
-    @Override
-    public void addStatusChangeListener(Consumer<Status> listener) {
-        delegateMap.addStatusChangeListener(listener);
-    }
-
-    @Override
-    public void removeStatusChangeListener(Consumer<Status> listener) {
-        delegateMap.removeStatusChangeListener(listener);
-    }
-
-    @Override
-    public Collection<Consumer<Status>> statusChangeListeners() {
-        return delegateMap.statusChangeListeners();
-    }
-
-    @Override
-    public String toString() {
-        return MoreObjects.toStringHelper(getClass())
-                          .add("delegateMap", delegateMap)
-                          .toString();
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(delegateMap);
-    }
-
-    @Override
-    public boolean equals(Object other) {
-        if (other instanceof DelegatingAsyncConsistentMap) {
-            DelegatingAsyncConsistentMap<K, V> that = (DelegatingAsyncConsistentMap) other;
-            return this.delegateMap.equals(that.delegateMap);
-        }
-        return false;
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncConsistentMultimap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncConsistentMultimap.java
deleted file mode 100644
index 779bb7a..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncConsistentMultimap.java
+++ /dev/null
@@ -1,168 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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 com.google.common.collect.Multiset;
-import org.onosproject.store.service.AsyncConsistentMultimap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MultimapEventListener;
-import org.onosproject.store.service.Versioned;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-
-import java.util.concurrent.Executor;
-
-/**
- * {@code AsyncConsistentMultimap} that merely delegates control to
- * another AsyncConsistentMultimap.
- *
- * @param <K> key type
- * @param <V> value type
- */
-public class DelegatingAsyncConsistentMultimap<K, V>
-        extends DelegatingDistributedPrimitive implements AsyncConsistentMultimap<K, V> {
-
-    private final AsyncConsistentMultimap<K, V> delegateMap;
-
-    public DelegatingAsyncConsistentMultimap(
-            AsyncConsistentMultimap<K, V> delegateMap) {
-        super(delegateMap);
-        this.delegateMap = delegateMap;
-    }
-
-    @Override
-    public CompletableFuture<Integer> size() {
-        return delegateMap.size();
-    }
-
-    @Override
-    public CompletableFuture<Boolean> isEmpty() {
-        return delegateMap.isEmpty();
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsKey(K key) {
-        return delegateMap.containsKey(key);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsValue(V value) {
-        return delegateMap.containsValue(value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsEntry(K key, V value) {
-        return delegateMap.containsEntry(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> put(K key, V value) {
-        return delegateMap.put(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends V>>> putAndGet(K key, V value) {
-        return delegateMap.putAndGet(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(K key, V value) {
-        return delegateMap.remove(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends V>>> removeAndGet(K key, V value) {
-        return delegateMap.removeAndGet(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> removeAll(
-            K key, Collection<? extends V> values) {
-        return delegateMap.removeAll(key, values);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends V>>>
-        removeAll(K key) {
-        return delegateMap.removeAll(key);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> putAll(
-            K key, Collection<? extends V> values) {
-        return delegateMap.putAll(key, values);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends V>>>
-        replaceValues(K key, Collection<V> values) {
-        return delegateMap.replaceValues(key, values);
-    }
-
-    @Override
-    public CompletableFuture<Void> clear() {
-        return delegateMap.clear();
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends V>>> get(K key) {
-        return delegateMap.get(key);
-    }
-
-    @Override
-    public CompletableFuture<Set<K>> keySet() {
-        return delegateMap.keySet();
-    }
-
-    @Override
-    public CompletableFuture<Multiset<K>> keys() {
-        return delegateMap.keys();
-    }
-
-    @Override
-    public CompletableFuture<Multiset<V>> values() {
-        return delegateMap.values();
-    }
-
-    @Override
-    public CompletableFuture<AsyncIterator<Map.Entry<K, V>>> iterator() {
-        return delegateMap.iterator();
-    }
-
-    @Override
-    public CompletableFuture<Collection<Map.Entry<K, V>>> entries() {
-        return delegateMap.entries();
-    }
-
-    @Override
-    public CompletableFuture<Void> addListener(MultimapEventListener<K, V> listener, Executor executor) {
-        return delegateMap.addListener(listener, executor);
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(MultimapEventListener<K, V> listener) {
-        return delegateMap.removeListener(listener);
-    }
-
-    @Override
-    public CompletableFuture<Map<K, Collection<V>>> asMap() {
-        return delegateMap.asMap();
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncConsistentTreeMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncConsistentTreeMap.java
deleted file mode 100644
index b3a1f95..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncConsistentTreeMap.java
+++ /dev/null
@@ -1,298 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.Collection;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.NavigableSet;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.BiFunction;
-import java.util.function.Predicate;
-
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.AsyncConsistentTreeMap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * A {@link AsyncConsistentTreeMap} that delegates control to another instance
- * of {@link AsyncConsistentTreeMap}.
- */
-public class DelegatingAsyncConsistentTreeMap<V>
-        extends DelegatingDistributedPrimitive implements AsyncConsistentTreeMap<V> {
-
-    private final AsyncConsistentTreeMap<V> delegateMap;
-
-    DelegatingAsyncConsistentTreeMap(AsyncConsistentTreeMap<V> delegateMap) {
-        super(delegateMap);
-        this.delegateMap = checkNotNull(delegateMap,
-                                        "delegate map cannot be null");
-    }
-
-    @Override
-    public CompletableFuture<String> firstKey() {
-        return delegateMap.firstKey();
-    }
-
-    @Override
-    public CompletableFuture<String> lastKey() {
-        return delegateMap.lastKey();
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<V>>> ceilingEntry(String key) {
-        return delegateMap.ceilingEntry(key);
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<V>>> floorEntry(String key) {
-        return delegateMap.floorEntry(key);
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<V>>> higherEntry(String key) {
-        return delegateMap.higherEntry(key);
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<V>>> lowerEntry(String key) {
-        return delegateMap.lowerEntry(key);
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<V>>> firstEntry() {
-        return delegateMap.firstEntry();
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<V>>> lastEntry() {
-        return delegateMap.lastEntry();
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<V>>> pollFirstEntry() {
-        return delegateMap.pollFirstEntry();
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<V>>> pollLastEntry() {
-        return delegateMap.pollLastEntry();
-    }
-
-    @Override
-    public CompletableFuture<String> lowerKey(String key) {
-        return delegateMap.lowerKey(key);
-    }
-
-    @Override
-    public CompletableFuture<String> floorKey(String key) {
-        return delegateMap.floorKey(key);
-    }
-
-    @Override
-    public CompletableFuture<String> ceilingKey(String key) {
-        return delegateMap.ceilingKey(key);
-    }
-
-    @Override
-    public CompletableFuture<String> higherKey(String key) {
-        return delegateMap.higherKey(key);
-    }
-
-    @Override
-    public CompletableFuture<NavigableSet<String>> navigableKeySet() {
-        return delegateMap.navigableKeySet();
-    }
-
-    @Override
-    public CompletableFuture<NavigableMap<String, V>> subMap(
-            String upperKey,
-            String lowerKey,
-            boolean inclusiveUpper,
-            boolean inclusiveLower) {
-        return delegateMap.subMap(upperKey, lowerKey,
-                                  inclusiveUpper, inclusiveLower);
-    }
-
-    @Override
-    public CompletableFuture<Integer> size() {
-        return delegateMap.size();
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsKey(String key) {
-        return delegateMap.containsKey(key);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsValue(V value) {
-        return delegateMap.containsValue(value);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> get(String key) {
-        return delegateMap.get(key);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> getOrDefault(String key, V defaultValue) {
-        return delegateMap.getOrDefault(key, defaultValue);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> computeIf(
-            String key,
-            Predicate<? super V> condition,
-            BiFunction<? super String, ? super V,
-                    ? extends V> remappingFunction) {
-        return delegateMap.computeIf(key, condition, remappingFunction);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> put(String key, V value) {
-        return delegateMap.put(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> putAndGet(String key, V value) {
-        return delegateMap.putAndGet(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> remove(String key) {
-        return delegateMap.remove(key);
-    }
-
-    @Override
-    public CompletableFuture<Void> clear() {
-        return delegateMap.clear();
-    }
-
-    @Override
-    public CompletableFuture<Set<String>> keySet() {
-        return delegateMap.keySet();
-    }
-
-    @Override
-    public CompletableFuture<Collection<Versioned<V>>> values() {
-        return delegateMap.values();
-    }
-
-    @Override
-    public CompletableFuture<Set<Map.Entry<String, Versioned<V>>>> entrySet() {
-        return delegateMap.entrySet();
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> putIfAbsent(String key, V value) {
-        return delegateMap.putIfAbsent(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(String key, V value) {
-        return delegateMap.remove(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(String key, long version) {
-        return delegateMap.remove(key, version);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> replace(String key, V value) {
-        return delegateMap.replace(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(String key, V oldValue,
-                                              V newValue) {
-        return delegateMap.replace(key, oldValue, newValue);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(String key, long oldVersion,
-                                              V newValue) {
-        return delegateMap.replace(key, oldVersion, newValue);
-    }
-
-    @Override
-    public CompletableFuture<AsyncIterator<Map.Entry<String, Versioned<V>>>> iterator() {
-        return delegateMap.iterator();
-    }
-
-    @Override
-    public CompletableFuture<Void> addListener(
-            MapEventListener<String, V> listener, Executor executor) {
-        return delegateMap.addListener(listener, executor);
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(
-            MapEventListener<String, V> listener) {
-        return delegateMap.removeListener(listener);
-    }
-
-    @Override
-    public CompletableFuture<Version> begin(TransactionId transactionId) {
-        return delegateMap.begin(transactionId);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepare(TransactionLog<MapUpdate<String, V>> transactionLog) {
-        return delegateMap.prepare(transactionLog);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<MapUpdate<String, V>> transactionLog) {
-        return delegateMap.prepareAndCommit(transactionLog);
-    }
-
-    @Override
-    public CompletableFuture<Void> commit(TransactionId transactionId) {
-        return delegateMap.commit(transactionId);
-    }
-
-    @Override
-    public CompletableFuture<Void> rollback(TransactionId transactionId) {
-        return delegateMap.rollback(transactionId);
-    }
-
-    @Override
-    public boolean equals(Object other) {
-        if (other instanceof DelegatingAsyncConsistentTreeMap) {
-            DelegatingAsyncConsistentTreeMap<V> that =
-                    (DelegatingAsyncConsistentTreeMap) other;
-            return this.delegateMap.equals(that.delegateMap);
-        }
-        return false;
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(delegateMap);
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncDocumentTree.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncDocumentTree.java
deleted file mode 100644
index 6182c48..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncDocumentTree.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.Map;
-import java.util.Objects;
-import java.util.concurrent.CompletableFuture;
-
-import com.google.common.base.MoreObjects;
-import org.onosproject.store.primitives.NodeUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.AsyncDocumentTree;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTreeListener;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-/**
- * Document tree that delegates to an underlying instance.
- */
-public class DelegatingAsyncDocumentTree<V> extends DelegatingDistributedPrimitive implements AsyncDocumentTree<V> {
-    private final AsyncDocumentTree<V> delegateTree;
-
-    public DelegatingAsyncDocumentTree(AsyncDocumentTree<V> delegateTree) {
-        super(delegateTree);
-        this.delegateTree = delegateTree;
-    }
-
-    @Override
-    public DocumentPath root() {
-        return delegateTree.root();
-    }
-
-    @Override
-    public CompletableFuture<Map<String, Versioned<V>>> getChildren(DocumentPath path) {
-        return delegateTree.getChildren(path);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> get(DocumentPath path) {
-        return delegateTree.get(path);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> set(DocumentPath path, V value) {
-        return delegateTree.set(path, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> create(DocumentPath path, V value) {
-        return delegateTree.create(path, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> createRecursive(DocumentPath path, V value) {
-        return delegateTree.createRecursive(path, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(DocumentPath path, V newValue, long version) {
-        return delegateTree.replace(path, newValue, version);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(DocumentPath path, V newValue, V currentValue) {
-        return delegateTree.replace(path, newValue, currentValue);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> removeNode(DocumentPath path) {
-        return delegateTree.removeNode(path);
-    }
-
-    @Override
-    public CompletableFuture<Void> addListener(DocumentPath path, DocumentTreeListener<V> listener) {
-        return delegateTree.addListener(path, listener);
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(DocumentTreeListener<V> listener) {
-        return delegateTree.removeListener(listener);
-    }
-
-    @Override
-    public CompletableFuture<Version> begin(TransactionId transactionId) {
-        return delegateTree.begin(transactionId);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepare(TransactionLog<NodeUpdate<V>> transactionLog) {
-        return delegateTree.prepare(transactionLog);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<NodeUpdate<V>> transactionLog) {
-        return delegateTree.prepareAndCommit(transactionLog);
-    }
-
-    @Override
-    public CompletableFuture<Void> commit(TransactionId transactionId) {
-        return delegateTree.commit(transactionId);
-    }
-
-    @Override
-    public CompletableFuture<Void> rollback(TransactionId transactionId) {
-        return delegateTree.rollback(transactionId);
-    }
-
-    @Override
-    public String toString() {
-        return MoreObjects.toStringHelper(getClass())
-                .add("delegateTree", delegateTree)
-                .toString();
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(delegateTree);
-    }
-
-    @Override
-    public boolean equals(Object other) {
-        if (other instanceof DelegatingAsyncDocumentTree) {
-            DelegatingAsyncDocumentTree<V> that = (DelegatingAsyncDocumentTree) other;
-            return this.delegateTree.equals(that.delegateTree);
-        }
-        return false;
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncLeaderElector.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncLeaderElector.java
deleted file mode 100644
index 0235616..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingAsyncLeaderElector.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * 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.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Consumer;
-
-import org.onosproject.cluster.Leadership;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.event.Change;
-import org.onosproject.store.service.AsyncLeaderElector;
-
-/**
- * Delegating leader elector.
- */
-public class DelegatingAsyncLeaderElector extends DelegatingDistributedPrimitive implements AsyncLeaderElector {
-
-    private final AsyncLeaderElector delegateLeaderElector;
-
-    public DelegatingAsyncLeaderElector(AsyncLeaderElector delegateLeaderElector) {
-        super(delegateLeaderElector);
-        this.delegateLeaderElector = delegateLeaderElector;
-    }
-
-    @Override
-    public CompletableFuture<Leadership> run(String topic, NodeId nodeId) {
-        return delegateLeaderElector.run(topic, nodeId);
-    }
-
-    @Override
-    public CompletableFuture<Void> withdraw(String topic) {
-        return delegateLeaderElector.withdraw(topic);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> anoint(String topic, NodeId nodeId) {
-        return delegateLeaderElector.anoint(topic, nodeId);
-    }
-
-    @Override
-    public CompletableFuture<Void> evict(NodeId nodeId) {
-        return delegateLeaderElector.evict(nodeId);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> promote(String topic, NodeId nodeId) {
-        return delegateLeaderElector.promote(topic, nodeId);
-    }
-
-    @Override
-    public CompletableFuture<Leadership> getLeadership(String topic) {
-        return delegateLeaderElector.getLeadership(topic);
-    }
-
-    @Override
-    public CompletableFuture<Map<String, Leadership>> getLeaderships() {
-        return delegateLeaderElector.getLeaderships();
-    }
-
-    @Override
-    public CompletableFuture<Void> addChangeListener(Consumer<Change<Leadership>> consumer) {
-        return delegateLeaderElector.addChangeListener(consumer);
-    }
-
-    @Override
-    public CompletableFuture<Void> removeChangeListener(Consumer<Change<Leadership>> consumer) {
-        return delegateLeaderElector.removeChangeListener(consumer);
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingDistributedPrimitive.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingDistributedPrimitive.java
deleted file mode 100644
index ff6876f..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DelegatingDistributedPrimitive.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.Collection;
-import java.util.Objects;
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Consumer;
-
-import com.google.common.base.MoreObjects;
-import org.onosproject.core.ApplicationId;
-import org.onosproject.store.service.DistributedPrimitive;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * Base class for primitive delegates.
- */
-public abstract class DelegatingDistributedPrimitive implements DistributedPrimitive {
-    private final DistributedPrimitive primitive;
-
-    public DelegatingDistributedPrimitive(DistributedPrimitive primitive) {
-        this.primitive = checkNotNull(primitive);
-    }
-
-    @Override
-    public String name() {
-        return primitive.name();
-    }
-
-    @Override
-    public Type primitiveType() {
-        return primitive.primitiveType();
-    }
-
-    @Override
-    public ApplicationId applicationId() {
-        return primitive.applicationId();
-    }
-
-    @Override
-    public CompletableFuture<Void> destroy() {
-        return primitive.destroy();
-    }
-
-    @Override
-    public void addStatusChangeListener(Consumer<Status> listener) {
-        primitive.addStatusChangeListener(listener);
-    }
-
-    @Override
-    public void removeStatusChangeListener(Consumer<Status> listener) {
-        primitive.removeStatusChangeListener(listener);
-    }
-
-    @Override
-    public Collection<Consumer<Status>> statusChangeListeners() {
-        return primitive.statusChangeListeners();
-    }
-
-    @Override
-    public String toString() {
-        return MoreObjects.toStringHelper(getClass())
-                .add("delegate", primitive)
-                .toString();
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(primitive);
-    }
-
-    @Override
-    public boolean equals(Object other) {
-        return other instanceof DelegatingDistributedPrimitive
-                && primitive.equals(((DelegatingDistributedPrimitive) other).primitive);
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DistributedPrimitives.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DistributedPrimitives.java
deleted file mode 100644
index 9b209ba..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/DistributedPrimitives.java
+++ /dev/null
@@ -1,267 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.function.BiFunction;
-import java.util.function.Function;
-
-import org.onosproject.core.Version;
-import org.onosproject.store.service.AsyncAtomicCounterMap;
-import org.onosproject.store.service.AsyncAtomicValue;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.AsyncConsistentMultimap;
-import org.onosproject.store.service.AsyncConsistentTreeMap;
-import org.onosproject.store.service.AsyncDistributedSet;
-import org.onosproject.store.service.AsyncDocumentTree;
-
-/**
- * Misc utilities for working with {@code DistributedPrimitive}s.
- */
-public final class DistributedPrimitives {
-
-    private DistributedPrimitives() {}
-
-    /**
-     * Creates an instance of {@code AsyncDistributedSet} that is backed by a {@code AsyncConsistentMap}.
-     *
-     * @param map backing map
-     * @return set
-     * @param <E> set element type
-     */
-    public static <E> AsyncDistributedSet<E> newSetFromMap(AsyncConsistentMap<E, Boolean> map) {
-        return new DefaultAsyncDistributedSet<>(map, map.name(), true);
-    }
-
-    /**
-     * Creates an instance of {@code AsyncConsistentMap} that records metrics for all its operations.
-     *
-     * @param map map whose operations are to be metered
-     * @return metered map
-     * @param <K> map key type
-     * @param <V> map value type
-     */
-    public static <K, V> AsyncConsistentMap<K, V> newMeteredMap(AsyncConsistentMap<K, V> map) {
-        return new MeteredAsyncConsistentMap<>(map);
-    }
-
-    /**
-     * Creates an instance of {@code AsyncConsistentMap} that caches entries on get.
-     *
-     * @param map backing map
-     * @return caching map
-     * @param <K> map key type
-     * @param <V> map value type
-     */
-    public static <K, V> AsyncConsistentMap<K, V> newCachingMap(AsyncConsistentMap<K, V> map) {
-        return new CachingAsyncConsistentMap<>(map);
-    }
-
-    /**
-     * Creates an instance of {@code AsyncConsistentMap} that disallows updates.
-     *
-     * @param map backing map
-     * @return unmodifiable map
-     * @param <K> map key type
-     * @param <V> map value type
-     */
-    public static <K, V> AsyncConsistentMap<K, V> newUnmodifiableMap(AsyncConsistentMap<K, V> map) {
-        return new UnmodifiableAsyncConsistentMap<>(map);
-    }
-
-    /**
-     * Creates an instance of {@code AsyncConsistentMap} that disallows null values.
-     *
-     * @param map backing map
-     * @return not null map
-     * @param <K> map key type
-     * @param <V> map value type
-     */
-    public static <K, V> AsyncConsistentMap<K, V> newNotNullMap(AsyncConsistentMap<K, V> map) {
-        return new NotNullAsyncConsistentMap<>(map);
-    }
-
-    /**
-     * Creates an instance of {@code AsyncConsistentMap} that converts values from other versions.
-     *
-     * @param map backing map
-     * @param compatibilityFunction the compatibility function
-     * @param version local node version
-     * @param <K> map key type
-     * @param <V> map value type
-     * @return compatible map
-     */
-    public static <K, V> AsyncConsistentMap<K, V> newCompatibleMap(
-        AsyncConsistentMap<K, CompatibleValue<V>> map,
-        BiFunction<V, Version, V> compatibilityFunction,
-        Version version) {
-        Function<V, CompatibleValue<V>> encoder = value -> new CompatibleValue<>(value, version);
-        Function<CompatibleValue<V>, V> decoder = value -> {
-            if (!value.version().equals(version)) {
-                return compatibilityFunction.apply(value.value(), value.version());
-            }
-            return value.value();
-        };
-        return new TranscodingAsyncConsistentMap<>(map, k -> k, k -> k, encoder, decoder);
-    }
-
-    /**
-     * Creates an instance of {@code AsyncAtomicValue} that transforms value types.
-     *
-     * @param value backing value
-     * @param valueEncoder transformer for value type of returned value to value type of input value
-     * @param valueDecoder transformer for value type of input value to value type of returned value
-     * @param <V1> returned value type
-     * @param <V2> input value type
-     * @return new counter map
-     */
-    public static <V1, V2> AsyncAtomicValue<V1> newTranscodingAtomicValue(AsyncAtomicValue<V2> value,
-        Function<V1, V2> valueEncoder,
-        Function<V2, V1> valueDecoder) {
-        return new TranscodingAsyncAtomicValue<>(value, valueEncoder, valueDecoder);
-    }
-
-    /**
-     * Creates an instance of {@code AsyncAtomicValue} that converts values from other versions.
-     *
-     * @param atomicValue backing value
-     * @param compatibilityFunction the compatibility function
-     * @param version local node version
-     * @param <V> value type
-     * @return compatible map
-     */
-    public static <V> AsyncAtomicValue<V> newCompatibleAtomicValue(
-        AsyncAtomicValue<CompatibleValue<V>> atomicValue,
-        BiFunction<V, Version, V> compatibilityFunction,
-        Version version) {
-        Function<V, CompatibleValue<V>> encoder = value -> new CompatibleValue<>(value, version);
-        Function<CompatibleValue<V>, V> decoder = value -> {
-            if (!value.version().equals(version)) {
-                return compatibilityFunction.apply(value.value(), value.version());
-            }
-            return value.value();
-        };
-        return new TranscodingAsyncAtomicValue<>(atomicValue, encoder, decoder);
-    }
-
-    /**
-     * Creates an instance of {@code AsyncAtomicCounterMap} that transforms key types.
-     *
-     * @param map backing map
-     * @param keyEncoder transformer for key type of returned map to key type of input map
-     * @param keyDecoder transformer for key type of input map to key type of returned map
-     * @param <K1> returned map key type
-     * @param <K2> input map key type
-     * @return new counter map
-     */
-    public static <K1, K2> AsyncAtomicCounterMap<K1> newTranscodingAtomicCounterMap(AsyncAtomicCounterMap<K2> map,
-        Function<K1, K2> keyEncoder,
-        Function<K2, K1> keyDecoder) {
-        return new TranscodingAsyncAtomicCounterMap<>(map, keyEncoder, keyDecoder);
-    }
-
-    /**
-     * Creates an instance of {@code AsyncConsistentMap} that transforms operations inputs and applies them
-     * to corresponding operation in a different typed map and returns the output after reverse transforming it.
-     *
-     * @param map backing map
-     * @param keyEncoder transformer for key type of returned map to key type of input map
-     * @param keyDecoder transformer for key type of input map to key type of returned map
-     * @param valueEncoder transformer for value type of returned map to value type of input map
-     * @param valueDecoder transformer for value type of input map to value type of returned map
-     * @param <K1> returned map key type
-     * @param <K2> input map key type
-     * @param <V1> returned map value type
-     * @param <V2> input map key type
-     * @return new map
-     */
-    public static <K1, V1, K2, V2> AsyncConsistentMap<K1, V1> newTranscodingMap(AsyncConsistentMap<K2, V2> map,
-            Function<K1, K2> keyEncoder,
-            Function<K2, K1> keyDecoder,
-            Function<V1, V2> valueEncoder,
-            Function<V2, V1> valueDecoder) {
-        return new TranscodingAsyncConsistentMap<K1, V1, K2, V2>(map,
-                keyEncoder,
-                keyDecoder,
-                valueEncoder,
-                valueDecoder);
-    }
-
-    /**
-     * Creates an instance of {@code DistributedTreeMap} that transforms operations inputs and applies them
-     * to corresponding operation in a different typed map and returns the output after reverse transforming it.
-     *
-     * @param map backing map
-     * @param valueEncoder transformer for value type of returned map to value type of input map
-     * @param valueDecoder transformer for value type of input map to value type of returned map
-     * @param <V1> returned map value type
-     * @param <V2> input map key type
-     * @return new map
-     */
-    public static <V1, V2> AsyncConsistentTreeMap<V1> newTranscodingTreeMap(
-            AsyncConsistentTreeMap<V2> map,
-            Function<V1, V2> valueEncoder,
-            Function<V2, V1> valueDecoder) {
-        return new TranscodingAsyncConsistentTreeMap<>(map,
-                                                       valueEncoder,
-                                                       valueDecoder);
-    }
-
-    /**
-     * Creates an instance of {@code AsyncConsistentMultimap} that transforms
-     * operations inputs and applies them to corresponding operation in a
-     * differently typed map and returns the output after reverse transforming
-     * it.
-     *
-     * @param multimap backing multimap
-     * @param keyEncoder transformer for key type of returned map to key type
-     *                   of input map
-     * @param keyDecoder transformer for key type of input map to key type of
-     *                   returned map
-     * @param valueEncoder transformer for value type of returned map to value
-     *                     type of input map
-     * @param valueDecoder transformer for value type of input map to value
-     *                     type of returned map
-     * @param <K1> returned map key type
-     * @param <K2> input map key type
-     * @param <V1> returned map value type
-     * @param <V2> input map key type
-     * @return new map
-     */
-    public static <K1, V1, K2, V2> AsyncConsistentMultimap<K1, V1>
-    newTranscodingMultimap(AsyncConsistentMultimap<K2, V2> multimap,
-                           Function<K1, K2> keyEncoder,
-                           Function<K2, K1> keyDecoder,
-                           Function<V1, V2> valueEncoder,
-                           Function<V2, V1> valueDecoder) {
-        return new TranscodingAsyncConsistentMultimap<>(multimap,
-                                                        keyEncoder,
-                                                        keyDecoder,
-                                                        valueDecoder,
-                                                        valueEncoder);
-    }
-
-    /**
-     * Creates an instance of {@code AsyncDocumentTree} that caches values on get.
-     *
-     * @param tree backing tree
-     * @return caching tree
-     * @param <V> tree value type
-     */
-    public static <V> AsyncDocumentTree<V> newCachingDocumentTree(AsyncDocumentTree<V> tree) {
-        return new CachingAsyncDocumentTree<V>(tree);
-    }
-
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/EventuallyConsistentMapBuilderImpl.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/EventuallyConsistentMapBuilderImpl.java
index df35bd6..da00555 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/EventuallyConsistentMapBuilderImpl.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/EventuallyConsistentMapBuilderImpl.java
@@ -15,14 +15,6 @@
  */
 package org.onosproject.store.primitives.impl;
 
-import org.onlab.util.KryoNamespace;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.persistence.PersistenceService;
-import org.onosproject.store.Timestamp;
-import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
-import org.onosproject.store.service.EventuallyConsistentMap;
-import org.onosproject.store.service.EventuallyConsistentMapBuilder;
-
 import java.util.Collection;
 import java.util.List;
 import java.util.concurrent.ExecutorService;
@@ -31,6 +23,14 @@
 import java.util.function.BiFunction;
 import java.util.function.Supplier;
 
+import org.onlab.util.KryoNamespace;
+import org.onosproject.cluster.NodeId;
+import org.onosproject.persistence.PersistenceService;
+import org.onosproject.store.Timestamp;
+import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
+import org.onosproject.store.service.EventuallyConsistentMap;
+import org.onosproject.store.service.EventuallyConsistentMapBuilder;
+
 import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
 
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/EventuallyConsistentMapImpl.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/EventuallyConsistentMapImpl.java
index 74e567b..762689c 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/EventuallyConsistentMapImpl.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/EventuallyConsistentMapImpl.java
@@ -15,6 +15,31 @@
  */
 package org.onosproject.store.primitives.impl;
 
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.Timer;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.function.BiFunction;
+import java.util.function.Function;
+import java.util.function.Supplier;
+import java.util.stream.Collectors;
+
 import com.google.common.collect.Collections2;
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
@@ -42,31 +67,6 @@
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.Set;
-import java.util.Timer;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
 import static com.google.common.base.Preconditions.checkNotNull;
 import static com.google.common.base.Preconditions.checkState;
 import static java.util.concurrent.Executors.newSingleThreadScheduledExecutor;
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/FederatedDistributedPrimitiveCreator.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/FederatedDistributedPrimitiveCreator.java
deleted file mode 100644
index 3fe721f..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/FederatedDistributedPrimitiveCreator.java
+++ /dev/null
@@ -1,231 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.function.Function;
-
-import com.google.common.base.Charsets;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import com.google.common.hash.Funnel;
-import com.google.common.hash.Funnels;
-import com.google.common.hash.Hashing;
-import org.onlab.util.HexString;
-import org.onosproject.cluster.PartitionId;
-import org.onosproject.store.primitives.DistributedPrimitiveCreator;
-import org.onosproject.store.service.AsyncAtomicCounter;
-import org.onosproject.store.service.AsyncAtomicCounterMap;
-import org.onosproject.store.service.AsyncAtomicIdGenerator;
-import org.onosproject.store.service.AsyncAtomicValue;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.AsyncConsistentMultimap;
-import org.onosproject.store.service.AsyncConsistentTreeMap;
-import org.onosproject.store.service.AsyncDistributedLock;
-import org.onosproject.store.service.AsyncDistributedSet;
-import org.onosproject.store.service.AsyncDocumentTree;
-import org.onosproject.store.service.AsyncLeaderElector;
-import org.onosproject.store.service.AtomicCounterMapOptions;
-import org.onosproject.store.service.AtomicCounterOptions;
-import org.onosproject.store.service.AtomicIdGeneratorOptions;
-import org.onosproject.store.service.AtomicValueOptions;
-import org.onosproject.store.service.ConsistentMapOptions;
-import org.onosproject.store.service.ConsistentMultimapOptions;
-import org.onosproject.store.service.ConsistentTreeMapOptions;
-import org.onosproject.store.service.DistributedLockOptions;
-import org.onosproject.store.service.DistributedSetOptions;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTreeOptions;
-import org.onosproject.store.service.LeaderElectorOptions;
-import org.onosproject.store.service.WorkQueue;
-import org.onosproject.store.service.WorkQueueOptions;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * {@code DistributedPrimitiveCreator} that federates responsibility for creating
- * distributed primitives to a collection of other {@link DistributedPrimitiveCreator creators}.
- */
-public class FederatedDistributedPrimitiveCreator implements DistributedPrimitiveCreator {
-
-    private static final Funnel<Iterable<? extends CharSequence>> STR_LIST_FUNNEL =
-                Funnels.sequentialFunnel(Funnels.unencodedCharsFunnel());
-
-    private final TreeMap<PartitionId, DistributedPrimitiveCreator> members;
-    private final List<PartitionId> sortedMemberPartitionIds;
-    private final int buckets;
-
-    public FederatedDistributedPrimitiveCreator(Map<PartitionId, DistributedPrimitiveCreator> members, int buckets) {
-        this.members = Maps.newTreeMap();
-        this.members.putAll(checkNotNull(members));
-        this.sortedMemberPartitionIds = Lists.newArrayList(members.keySet());
-        this.buckets = buckets;
-    }
-
-    @Override
-    public <K, V> AsyncConsistentMap<K, V> newAsyncConsistentMap(ConsistentMapOptions options) {
-        Map<PartitionId, AsyncConsistentMap<byte[], byte[]>> maps =
-                Maps.transformValues(members,
-                                     partition -> DistributedPrimitives.newTranscodingMap(
-                                             partition.<String, byte[]>newAsyncConsistentMap(options.name(), null),
-                                             HexString::toHexString,
-                                             HexString::fromHexString,
-                                             Function.identity(),
-                                             Function.identity()));
-        Hasher<byte[]> hasher = key -> {
-            int bucket = Math.abs(Hashing.murmur3_32().hashBytes(key).asInt()) % buckets;
-            return sortedMemberPartitionIds.get(Hashing.consistentHash(bucket, sortedMemberPartitionIds.size()));
-        };
-        AsyncConsistentMap<byte[], byte[]> partitionedMap =
-            new PartitionedAsyncConsistentMap<>(options.name(), maps, hasher);
-        return DistributedPrimitives.newTranscodingMap(partitionedMap,
-                key -> options.serializer().encode(key),
-                bytes -> options.serializer().decode(bytes),
-                value -> value == null ? null : options.serializer().encode(value),
-                bytes -> options.serializer().decode(bytes));
-    }
-
-    @Override
-    public <V> AsyncConsistentTreeMap<V> newAsyncConsistentTreeMap(ConsistentTreeMapOptions options) {
-        return getCreator(options.name()).newAsyncConsistentTreeMap(options.name(), options.serializer());
-    }
-
-    @Override
-    public <K, V> AsyncConsistentMultimap<K, V> newAsyncConsistentSetMultimap(ConsistentMultimapOptions options) {
-        Map<PartitionId, AsyncConsistentMultimap<byte[], byte[]>> maps =
-            Maps.transformValues(members,
-                partition -> DistributedPrimitives.newTranscodingMultimap(
-                    partition.<String, byte[]>newAsyncConsistentSetMultimap(options.name(), null),
-                    HexString::toHexString,
-                    HexString::fromHexString,
-                    Function.identity(),
-                    Function.identity()));
-        Hasher<byte[]> hasher = key -> {
-            int bucket = Math.abs(Hashing.murmur3_32().hashBytes(key).asInt()) % buckets;
-            return sortedMemberPartitionIds.get(Hashing.consistentHash(bucket, sortedMemberPartitionIds.size()));
-        };
-        AsyncConsistentMultimap<byte[], byte[]> partitionedMap =
-            new PartitionedAsyncConsistentMultimap<>(options.name(), maps, hasher);
-        return DistributedPrimitives.newTranscodingMultimap(partitionedMap,
-            key -> options.serializer().encode(key),
-            bytes -> options.serializer().decode(bytes),
-            value -> value == null ? null : options.serializer().encode(value),
-            bytes -> options.serializer().decode(bytes));
-    }
-
-    @Override
-    public <E> AsyncDistributedSet<E> newAsyncDistributedSet(DistributedSetOptions options) {
-        return DistributedPrimitives.newSetFromMap(newAsyncConsistentMap(options.name(), options.serializer()));
-    }
-
-    @Override
-    public <K> AsyncAtomicCounterMap<K> newAsyncAtomicCounterMap(AtomicCounterMapOptions options) {
-        return getCreator(options.name()).newAsyncAtomicCounterMap(options);
-    }
-
-    @Override
-    public AsyncAtomicCounter newAsyncCounter(AtomicCounterOptions options) {
-        return getCreator(options.name()).newAsyncCounter(options);
-    }
-
-    @Override
-    public AsyncAtomicIdGenerator newAsyncIdGenerator(AtomicIdGeneratorOptions options) {
-        return getCreator(options.name()).newAsyncIdGenerator(options);
-    }
-
-    @Override
-    public <V> AsyncAtomicValue<V> newAsyncAtomicValue(AtomicValueOptions options) {
-        return getCreator(options.name()).newAsyncAtomicValue(options);
-    }
-
-    @Override
-    public AsyncDistributedLock newAsyncDistributedLock(DistributedLockOptions options) {
-        return getCreator(options.name()).newAsyncDistributedLock(options);
-    }
-
-    @Override
-    public AsyncLeaderElector newAsyncLeaderElector(LeaderElectorOptions options) {
-        Map<PartitionId, AsyncLeaderElector> leaderElectors =
-                Maps.transformValues(members,
-                                     partition -> partition.newAsyncLeaderElector(options));
-        Hasher<String> hasher = topic -> {
-            int hashCode = Hashing.sha256().hashString(topic, Charsets.UTF_8).asInt();
-            return sortedMemberPartitionIds.get(Math.abs(hashCode) % members.size());
-        };
-        return new PartitionedAsyncLeaderElector(options.name(), leaderElectors, hasher);
-    }
-
-    @Override
-    public <E> WorkQueue<E> newWorkQueue(WorkQueueOptions options) {
-        return getCreator(options.name()).newWorkQueue(options);
-    }
-
-    @Override
-    public <V> AsyncDocumentTree<V> newAsyncDocumentTree(DocumentTreeOptions options) {
-        Map<PartitionId, AsyncDocumentTree<V>> trees =
-                Maps.transformValues(members, part -> part.<V>newAsyncDocumentTree(options));
-        Hasher<DocumentPath> hasher = key -> {
-            int bucket = (key == null) ? 0 :
-                    Math.abs(Hashing.murmur3_32()
-                                  .hashObject(key.pathElements(), STR_LIST_FUNNEL)
-                                  .asInt()) % buckets;
-            return sortedMemberPartitionIds.get(Hashing.consistentHash(bucket, sortedMemberPartitionIds.size()));
-        };
-        return new PartitionedAsyncDocumentTree<>(options.name(), trees, hasher);
-    }
-
-    @Override
-    public Set<String> getAsyncConsistentMapNames() {
-        return members.values()
-                      .stream()
-                      .map(DistributedPrimitiveCreator::getAsyncConsistentMapNames)
-                      .reduce(Sets::union)
-                      .orElse(ImmutableSet.of());
-    }
-
-    @Override
-    public Set<String> getAsyncAtomicCounterNames() {
-        return members.values()
-                      .stream()
-                      .map(DistributedPrimitiveCreator::getAsyncAtomicCounterNames)
-                      .reduce(Sets::union)
-                      .orElse(ImmutableSet.of());
-    }
-
-    @Override
-    public Set<String> getWorkQueueNames() {
-        return members.values()
-                      .stream()
-                      .map(DistributedPrimitiveCreator::getWorkQueueNames)
-                      .reduce(Sets::union)
-                      .orElse(ImmutableSet.of());
-    }
-
-    /**
-     * Returns the {@code DistributedPrimitiveCreator} to use for hosting a primitive.
-     * @param name primitive name
-     * @return primitive creator
-     */
-    private DistributedPrimitiveCreator getCreator(String name) {
-        int hashCode = Hashing.sha256().hashString(name, Charsets.UTF_8).asInt();
-        return members.get(sortedMemberPartitionIds.get(Math.abs(hashCode) % members.size()));
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/Hasher.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/Hasher.java
deleted file mode 100644
index 5714e33..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/Hasher.java
+++ /dev/null
@@ -1,32 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.cluster.PartitionId;
-
-/**
- * Interface for mapping from an object to {@link PartitionId}.
- *
- * @param <K> object type.
- */
-public interface Hasher<K> {
-    /**
-     * Returns the {@link PartitionId} to which the specified object maps.
-     * @param object object
-     * @return partition identifier
-     */
-    PartitionId hash(K object);
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/Managed.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/Managed.java
deleted file mode 100644
index 1d5977f..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/Managed.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.concurrent.CompletableFuture;
-
-/**
- * Interface for types that can be asynchronously opened and closed.
- */
-public interface Managed<T> {
-
-    /**
-     * Opens the managed object.
-     *
-     * @return A completable future to be completed once the object has been opened.
-     */
-    CompletableFuture<Void> open();
-
-    /**
-     * Closes the managed object.
-     *
-     * @return A completable future to be completed once the object has been closed.
-     */
-    CompletableFuture<Void> close();
-
-    /**
-     * Return {@code true} if the managed object is open.
-     * @return {@code true} if open
-     */
-    boolean isOpen();
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/MapValue.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/MapValue.java
index 1c6845d..3175963 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/MapValue.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/MapValue.java
@@ -15,10 +15,9 @@
  */
 package org.onosproject.store.primitives.impl;
 
-import org.onosproject.store.Timestamp;
-
 import com.google.common.base.MoreObjects;
 import com.google.common.base.Objects;
+import org.onosproject.store.Timestamp;
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/MeteredAsyncConsistentMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/MeteredAsyncConsistentMap.java
deleted file mode 100644
index de53121..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/MeteredAsyncConsistentMap.java
+++ /dev/null
@@ -1,314 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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 com.google.common.collect.Maps;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.MapEvent;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-import org.onosproject.utils.MeteringAgent;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-import java.util.function.Predicate;
-
-/**
- * {@link AsyncConsistentMap} that meters all its operations.
- *
- * @param <K> key type
- * @param <V> value type
- */
-public class MeteredAsyncConsistentMap<K, V>  extends DelegatingAsyncConsistentMap<K, V> {
-
-    private static final String PRIMITIVE_NAME = "consistentMap";
-    private static final String SIZE = "size";
-    private static final String IS_EMPTY = "isEmpty";
-    private static final String CONTAINS_KEY = "containsKey";
-    private static final String CONTAINS_VALUE = "containsValue";
-    private static final String GET = "get";
-    private static final String GET_OR_DEFAULT = "getOrDefault";
-    private static final String COMPUTE_IF = "computeIf";
-    private static final String PUT = "put";
-    private static final String PUT_AND_GET = "putAndGet";
-    private static final String PUT_IF_ABSENT = "putIfAbsent";
-    private static final String REMOVE = "remove";
-    private static final String CLEAR = "clear";
-    private static final String KEY_SET = "keySet";
-    private static final String VALUES = "values";
-    private static final String ENTRY_SET = "entrySet";
-    private static final String REPLACE = "replace";
-    private static final String COMPUTE_IF_ABSENT = "computeIfAbsent";
-    private static final String BEGIN = "begin";
-    private static final String PREPARE = "prepare";
-    private static final String COMMIT = "commit";
-    private static final String ROLLBACK = "rollback";
-    private static final String PREPARE_AND_COMMIT = "prepareAndCommit";
-    private static final String ADD_LISTENER = "addListener";
-    private static final String REMOVE_LISTENER = "removeListener";
-    private static final String NOTIFY_LISTENER = "notifyListener";
-
-    private final Map<MapEventListener<K, V>, InternalMeteredMapEventListener> listeners =
-            Maps.newIdentityHashMap();
-    private final MeteringAgent monitor;
-
-    public MeteredAsyncConsistentMap(AsyncConsistentMap<K, V> backingMap) {
-        super(backingMap);
-        this.monitor = new MeteringAgent(PRIMITIVE_NAME, backingMap.name(), true);
-    }
-
-    @Override
-    public CompletableFuture<Integer> size() {
-        final MeteringAgent.Context timer = monitor.startTimer(SIZE);
-        return super.size()
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> isEmpty() {
-        final MeteringAgent.Context timer = monitor.startTimer(IS_EMPTY);
-        return super.isEmpty()
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsKey(K key) {
-        final MeteringAgent.Context timer = monitor.startTimer(CONTAINS_KEY);
-        return super.containsKey(key)
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsValue(V value) {
-        final MeteringAgent.Context timer = monitor.startTimer(CONTAINS_VALUE);
-        return super.containsValue(value)
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> get(K key) {
-        final MeteringAgent.Context timer = monitor.startTimer(GET);
-        return super.get(key)
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> getOrDefault(K key, V defaultValue) {
-        final MeteringAgent.Context timer = monitor.startTimer(GET_OR_DEFAULT);
-        return super.getOrDefault(key, defaultValue)
-                .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> computeIfAbsent(K key,
-                                                           Function<? super K, ? extends V> mappingFunction) {
-        final MeteringAgent.Context timer = monitor.startTimer(COMPUTE_IF_ABSENT);
-        return super.computeIfAbsent(key, mappingFunction)
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> computeIf(K key,
-                                                     Predicate<? super V> condition,
-                                                     BiFunction<? super K, ? super V, ? extends V> remappingFunction) {
-        final MeteringAgent.Context timer = monitor.startTimer(COMPUTE_IF);
-        return super.computeIf(key, condition, remappingFunction)
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> put(K key, V value) {
-        final MeteringAgent.Context timer = monitor.startTimer(PUT);
-        return super.put(key, value)
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> putAndGet(K key, V value) {
-        final MeteringAgent.Context timer = monitor.startTimer(PUT_AND_GET);
-        return super.putAndGet(key, value)
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> remove(K key) {
-        final MeteringAgent.Context timer = monitor.startTimer(REMOVE);
-        return super.remove(key)
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Void> clear() {
-        final MeteringAgent.Context timer = monitor.startTimer(CLEAR);
-        return super.clear()
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Set<K>> keySet() {
-        final MeteringAgent.Context timer = monitor.startTimer(KEY_SET);
-        return super.keySet()
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Collection<Versioned<V>>> values() {
-        final MeteringAgent.Context timer = monitor.startTimer(VALUES);
-        return super.values()
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Set<Entry<K, Versioned<V>>>> entrySet() {
-        final MeteringAgent.Context timer = monitor.startTimer(ENTRY_SET);
-        return super.entrySet()
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> putIfAbsent(K key, V value) {
-        final MeteringAgent.Context timer = monitor.startTimer(PUT_IF_ABSENT);
-        return super.putIfAbsent(key, value)
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(K key, V value) {
-        final MeteringAgent.Context timer = monitor.startTimer(REMOVE);
-        return super.remove(key, value)
-                    .whenComplete((r, e) -> timer.stop(e));
-
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(K key, long version) {
-        final MeteringAgent.Context timer = monitor.startTimer(REMOVE);
-        return super.remove(key, version)
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> replace(K key, V value) {
-        final MeteringAgent.Context timer = monitor.startTimer(REPLACE);
-        return super.replace(key, value)
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(K key, V oldValue, V newValue) {
-        final MeteringAgent.Context timer = monitor.startTimer(REPLACE);
-        return super.replace(key, oldValue, newValue)
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(K key, long oldVersion, V newValue) {
-        final MeteringAgent.Context timer = monitor.startTimer(REPLACE);
-        return super.replace(key, oldVersion, newValue)
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Void> addListener(MapEventListener<K, V> listener, Executor executor) {
-        final MeteringAgent.Context timer = monitor.startTimer(ADD_LISTENER);
-        synchronized (listeners) {
-            InternalMeteredMapEventListener meteredListener =
-                    listeners.computeIfAbsent(listener, k -> new InternalMeteredMapEventListener(listener));
-            return super.addListener(meteredListener, executor)
-                        .whenComplete((r, e) -> timer.stop(e));
-        }
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(MapEventListener<K, V> listener) {
-        final MeteringAgent.Context timer = monitor.startTimer(REMOVE_LISTENER);
-        InternalMeteredMapEventListener meteredListener = listeners.remove(listener);
-        if (meteredListener != null) {
-            return super.removeListener(meteredListener)
-                        .whenComplete((r, e) -> timer.stop(e));
-        } else {
-            timer.stop(null);
-            return CompletableFuture.completedFuture(null);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Version> begin(TransactionId transactionId) {
-        final MeteringAgent.Context timer = monitor.startTimer(BEGIN);
-        return super.begin(transactionId)
-                .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepare(TransactionLog<MapUpdate<K, V>> transactionLog) {
-        final MeteringAgent.Context timer = monitor.startTimer(PREPARE);
-        return super.prepare(transactionLog)
-                .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Void> commit(TransactionId transactionId) {
-        final MeteringAgent.Context timer = monitor.startTimer(COMMIT);
-        return super.commit(transactionId)
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Void> rollback(TransactionId transactionId) {
-        final MeteringAgent.Context timer = monitor.startTimer(ROLLBACK);
-        return super.rollback(transactionId)
-                    .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<MapUpdate<K, V>> transactionLog) {
-        final MeteringAgent.Context timer = monitor.startTimer(PREPARE_AND_COMMIT);
-        return super.prepareAndCommit(transactionLog)
-                .whenComplete((r, e) -> timer.stop(e));
-    }
-
-    private class InternalMeteredMapEventListener implements MapEventListener<K, V> {
-
-        private final MapEventListener<K, V> listener;
-
-        InternalMeteredMapEventListener(MapEventListener<K, V> listener) {
-            this.listener = listener;
-        }
-
-        @Override
-        public void event(MapEvent<K, V> event) {
-            final MeteringAgent.Context timer = monitor.startTimer(NOTIFY_LISTENER);
-            try {
-                listener.event(event);
-                timer.stop(null);
-            } catch (Exception e) {
-                timer.stop(e);
-                throw new IllegalStateException(e);
-            }
-        }
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/NotNullAsyncConsistentMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/NotNullAsyncConsistentMap.java
deleted file mode 100644
index 9282cbf..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/NotNullAsyncConsistentMap.java
+++ /dev/null
@@ -1,130 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.Collection;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.stream.Collectors;
-
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.Versioned;
-
-/**
- * {@link org.onosproject.store.service.AsyncConsistentMap} that doesn't allow null values.
- */
-public class NotNullAsyncConsistentMap<K, V> extends DelegatingAsyncConsistentMap<K, V> {
-    private final AsyncConsistentMap<K, V> delegateMap;
-
-    public NotNullAsyncConsistentMap(AsyncConsistentMap<K, V> delegateMap) {
-        super(delegateMap);
-        this.delegateMap = delegateMap;
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsValue(V value) {
-        if (value == null) {
-            return CompletableFuture.completedFuture(false);
-        }
-        return super.containsValue(value);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> get(K key) {
-        return super.get(key).thenApply(v -> v != null && v.value() == null ? null : v);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> getOrDefault(K key, V defaultValue) {
-        return super.getOrDefault(key, defaultValue).thenApply(v -> v != null && v.value() == null ? null : v);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> put(K key, V value) {
-        if (value == null) {
-            return super.remove(key);
-        }
-        return super.put(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> putAndGet(K key, V value) {
-        if (value == null) {
-            return super.remove(key).thenApply(v -> null);
-        }
-        return super.putAndGet(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Collection<Versioned<V>>> values() {
-        return super.values().thenApply(value -> value.stream()
-            .filter(v -> v.value() != null)
-            .collect(Collectors.toList()));
-    }
-
-    @Override
-    public CompletableFuture<Set<Map.Entry<K, Versioned<V>>>> entrySet() {
-        return super.entrySet().thenApply(entries -> entries.stream()
-            .filter(e -> e.getValue().value() != null)
-            .collect(Collectors.toSet()));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> putIfAbsent(K key, V value) {
-        if (value == null) {
-            return super.remove(key);
-        }
-        return super.putIfAbsent(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(K key, V value) {
-        if (value == null) {
-            return CompletableFuture.completedFuture(false);
-        }
-        return super.remove(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(K key, long version) {
-        return super.remove(key, version);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> replace(K key, V value) {
-        if (value == null) {
-            return super.remove(key);
-        }
-        return super.replace(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(K key, V oldValue, V newValue) {
-        if (oldValue == null) {
-            return super.putIfAbsent(key, newValue).thenApply(Objects::isNull);
-        } else if (newValue == null) {
-            return super.remove(key, oldValue);
-        }
-        return super.replace(key, oldValue, newValue);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(K key, long oldVersion, V newValue) {
-        return super.replace(key, oldVersion, newValue);
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionManager.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionManager.java
index 28cc69e..f9a8ec0 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionManager.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionManager.java
@@ -17,32 +17,22 @@
 package org.onosproject.store.primitives.impl;
 
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.atomic.AtomicReference;
 import java.util.stream.Collectors;
 
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Maps;
+import io.atomix.cluster.MemberId;
+import io.atomix.primitive.partition.PartitionGroup;
+import io.atomix.protocols.raft.MultiRaftProtocol;
 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.ClusterMetadata;
-import org.onosproject.cluster.ClusterMetadataDiff;
-import org.onosproject.cluster.ClusterMetadataEvent;
-import org.onosproject.cluster.ClusterMetadataEventListener;
-import org.onosproject.cluster.ClusterMetadataService;
-import org.onosproject.cluster.ClusterService;
 import org.onosproject.cluster.NodeId;
-import org.onosproject.cluster.PartitionDiff;
 import org.onosproject.cluster.PartitionId;
 import org.onosproject.event.AbstractListenerManager;
-import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
+import org.onosproject.store.impl.AtomixManager;
 import org.onosproject.store.primitives.DistributedPrimitiveCreator;
 import org.onosproject.store.primitives.PartitionAdminService;
 import org.onosproject.store.primitives.PartitionEvent;
@@ -67,74 +57,52 @@
     private final Logger log = getLogger(getClass());
 
     @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
-    protected ClusterCommunicationService clusterCommunicator;
+    protected AtomixManager atomixManager;
 
-    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
-    protected ClusterMetadataService metadataService;
-
-    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
-    protected ClusterService clusterService;
-
-    private final Map<PartitionId, StoragePartition> partitions = Maps.newConcurrentMap();
-    private final AtomicReference<ClusterMetadata> currentClusterMetadata = new AtomicReference<>();
-
-    private final ClusterMetadataEventListener metadataListener = new InternalClusterMetadataListener();
+    private PartitionGroup partitionGroup;
 
     @Activate
     public void activate() {
+        partitionGroup = atomixManager.getAtomix().getPartitionService().getPartitionGroup(MultiRaftProtocol.TYPE);
         eventDispatcher.addSink(PartitionEvent.class, listenerRegistry);
-        currentClusterMetadata.set(metadataService.getClusterMetadata());
-
-        metadataService.addListener(metadataListener);
-
-        currentClusterMetadata.get()
-            .getPartitions()
-            .forEach(partition -> partitions.put(partition.getId(), new StoragePartition(
-                partition,
-                clusterCommunicator,
-                clusterService)));
-        CompletableFuture.allOf(partitions.values().stream()
-            .map(StoragePartition::open)
-            .toArray(CompletableFuture[]::new))
-            .join();
         log.info("Started");
     }
 
     @Deactivate
     public void deactivate() {
-        metadataService.removeListener(metadataListener);
         eventDispatcher.removeSink(PartitionEvent.class);
-
-        CompletableFuture.allOf(partitions.values().stream()
-            .map(StoragePartition::close)
-            .toArray(CompletableFuture[]::new))
-            .join();
         log.info("Stopped");
     }
 
     @Override
     public int getNumberOfPartitions() {
         checkPermission(PARTITION_READ);
-        return partitions.size();
+        return partitionGroup.getPartitions().size();
     }
 
     @Override
     public Set<PartitionId> getAllPartitionIds() {
         checkPermission(PARTITION_READ);
-        return partitions.keySet();
+        return partitionGroup.getPartitionIds().stream()
+            .map(partitionId -> PartitionId.from(partitionId.id()))
+            .collect(Collectors.toSet());
     }
 
     @Override
     public DistributedPrimitiveCreator getDistributedPrimitiveCreator(PartitionId partitionId) {
         checkPermission(PARTITION_READ);
-        return partitions.get(partitionId).client();
+        throw new UnsupportedOperationException();
     }
 
     @Override
     public Set<NodeId> getConfiguredMembers(PartitionId partitionId) {
         checkPermission(PARTITION_READ);
-        StoragePartition partition = partitions.get(partitionId);
-        return ImmutableSet.copyOf(partition.getMembers());
+        io.atomix.primitive.partition.PartitionId atomixPartitionId =
+            io.atomix.primitive.partition.PartitionId.from(partitionGroup.name(), partitionId.id());
+        return partitionGroup.getPartition(atomixPartitionId).members()
+            .stream()
+            .map(member -> NodeId.nodeId(member.id()))
+            .collect(Collectors.toSet());
     }
 
     @Override
@@ -147,36 +115,30 @@
 
     @Override
     public List<PartitionInfo> partitionInfo() {
-        return partitions.values()
-                         .stream()
-                         .flatMap(x -> Tools.stream(x.info()))
-                         .collect(Collectors.toList());
-    }
-
-    private void processMetadataUpdate(ClusterMetadata clusterMetadata) {
-        ClusterMetadataDiff diffExaminer =
-                new ClusterMetadataDiff(currentClusterMetadata.get(), clusterMetadata);
-        diffExaminer.partitionDiffs()
-                    .values()
-                    .stream()
-                    .filter(PartitionDiff::hasChanged)
-                    .forEach(diff -> partitions.get(diff.partitionId()).onUpdate(diff.newValue()));
-        currentClusterMetadata.set(clusterMetadata);
-    }
-
-    private class InternalClusterMetadataListener implements ClusterMetadataEventListener {
-        @Override
-        public void event(ClusterMetadataEvent event) {
-            processMetadataUpdate(event.subject());
-        }
+        checkPermission(PARTITION_READ);
+        return partitionGroup.getPartitions()
+            .stream()
+            .map(partition -> {
+                MemberId primary = partition.primary();
+                return new PartitionInfo(
+                    PartitionId.from(partition.id().id()),
+                    partition.term(),
+                    partition.members().stream().map(member -> member.id()).collect(Collectors.toList()),
+                    primary != null ? primary.id() : null);
+            })
+            .collect(Collectors.toList());
     }
 
     @Override
     public List<PartitionClientInfo> partitionClientInfo() {
-        return partitions.values()
-                         .stream()
-                         .map(StoragePartition::client)
-                         .map(StoragePartitionClient::clientInfo)
-                         .collect(Collectors.toList());
+        checkPermission(PARTITION_READ);
+        return partitionGroup.getPartitions()
+            .stream()
+            .map(partition -> new PartitionClientInfo(
+                PartitionId.from(partition.id().id()),
+                partition.members().stream()
+                    .map(member -> NodeId.nodeId(member.id()))
+                    .collect(Collectors.toList())))
+            .collect(Collectors.toList());
     }
 }
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncConsistentMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncConsistentMap.java
deleted file mode 100644
index 38ba0f3..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncConsistentMap.java
+++ /dev/null
@@ -1,305 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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 com.google.common.base.Preconditions.checkNotNull;
-
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.NoSuchElementException;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.BiFunction;
-import java.util.function.Consumer;
-import java.util.function.Predicate;
-import java.util.stream.Collectors;
-
-import org.onlab.util.Match;
-import org.onlab.util.Tools;
-import org.onosproject.cluster.PartitionId;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Maps;
-
-/**
- * {@link AsyncConsistentMap} that has its entries partitioned horizontally across
- * several {@link AsyncConsistentMap maps}.
- *
- * @param <K> key type
- * @param <V> value type
- */
-public class PartitionedAsyncConsistentMap<K, V> implements AsyncConsistentMap<K, V> {
-
-    private final String name;
-    private final TreeMap<PartitionId, AsyncConsistentMap<K, V>> partitions = Maps.newTreeMap();
-    private final Hasher<K> keyHasher;
-
-    public PartitionedAsyncConsistentMap(String name,
-            Map<PartitionId, AsyncConsistentMap<K, V>> partitions,
-            Hasher<K> keyHasher) {
-        this.name = name;
-        this.partitions.putAll(checkNotNull(partitions));
-        this.keyHasher = checkNotNull(keyHasher);
-    }
-
-    @Override
-    public String name() {
-        return name;
-    }
-
-    @Override
-    public CompletableFuture<Integer> size() {
-        return Tools.allOf(getMaps().stream().map(m -> m.size()).collect(Collectors.toList()),
-                            Math::addExact,
-                            0);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> isEmpty() {
-        return size().thenApply(size -> size == 0);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsKey(K key) {
-        return getMap(key).containsKey(key);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsValue(V value) {
-        return Tools.firstOf(getMaps().stream().map(m -> m.containsValue(value)).collect(Collectors.toList()),
-                            Match.ifValue(true),
-                            false);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> get(K key) {
-        return getMap(key).get(key);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> getOrDefault(K key, V defaultValue) {
-        return getMap(key).getOrDefault(key, defaultValue);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> computeIf(K key,
-            Predicate<? super V> condition,
-            BiFunction<? super K, ? super V, ? extends V> remappingFunction) {
-        return getMap(key).computeIf(key, condition, remappingFunction);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> put(K key, V value) {
-        return getMap(key).put(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> putAndGet(K key, V value) {
-        return getMap(key).putAndGet(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> remove(K key) {
-        return getMap(key).remove(key);
-    }
-
-    @Override
-    public CompletableFuture<Void> clear() {
-        return CompletableFuture.allOf(getMaps().stream()
-                                                .map(map -> map.clear())
-                                                .toArray(CompletableFuture[]::new));
-    }
-
-    @Override
-    public CompletableFuture<Set<K>> keySet() {
-        return Tools.allOf(getMaps().stream().map(m -> m.keySet()).collect(Collectors.toList()),
-                    (s1, s2) -> ImmutableSet.<K>builder().addAll(s1).addAll(s2).build(),
-                    ImmutableSet.of());
-    }
-
-    @Override
-    public CompletableFuture<Collection<Versioned<V>>> values() {
-        return Tools.allOf(getMaps().stream().map(m -> m.values()).collect(Collectors.toList()),
-                    (c1, c2) -> ImmutableList.<Versioned<V>>builder().addAll(c1).addAll(c2).build(),
-                    ImmutableList.of());
-    }
-
-    @Override
-    public CompletableFuture<Set<Entry<K, Versioned<V>>>> entrySet() {
-        return Tools.allOf(getMaps().stream().map(m -> m.entrySet()).collect(Collectors.toList()),
-                (s1, s2) -> ImmutableSet.<Entry<K, Versioned<V>>>builder().addAll(s1).addAll(s2).build(),
-                ImmutableSet.of());
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> putIfAbsent(K key, V value) {
-        return getMap(key).putIfAbsent(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(K key, V value) {
-        return getMap(key).remove(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(K key, long version) {
-        return getMap(key).remove(key, version);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> replace(K key, V value) {
-        return getMap(key).replace(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(K key, V oldValue, V newValue) {
-        return getMap(key).replace(key, oldValue, newValue);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(K key, long oldVersion, V newValue) {
-        return getMap(key).replace(key, oldVersion, newValue);
-    }
-
-    @Override
-    public CompletableFuture<AsyncIterator<Entry<K, Versioned<V>>>> iterator() {
-        return Tools.allOf(getMaps().stream().map(m -> m.iterator()).collect(Collectors.toList()))
-            .thenApply(PartitionedMultimapIterator::new);
-    }
-
-    @Override
-    public CompletableFuture<Void> addListener(MapEventListener<K, V> listener, Executor executor) {
-        return CompletableFuture.allOf(getMaps().stream()
-                                                .map(map -> map.addListener(listener, executor))
-                                                .toArray(CompletableFuture[]::new));
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(MapEventListener<K, V> listener) {
-        return CompletableFuture.allOf(getMaps().stream()
-                                                .map(map -> map.removeListener(listener))
-                                                .toArray(CompletableFuture[]::new));
-    }
-
-    @Override
-    public CompletableFuture<Version> begin(TransactionId transactionId) {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepare(TransactionLog<MapUpdate<K, V>> transactionLog) {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<MapUpdate<K, V>> transactionLog) {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public CompletableFuture<Void> commit(TransactionId transactionId) {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public CompletableFuture<Void> rollback(TransactionId transactionId) {
-        throw new UnsupportedOperationException();
-    }
-
-    @Override
-    public void addStatusChangeListener(Consumer<Status> listener) {
-        partitions.values().forEach(map -> map.addStatusChangeListener(listener));
-    }
-
-    @Override
-    public void removeStatusChangeListener(Consumer<Status> listener) {
-        partitions.values().forEach(map -> map.removeStatusChangeListener(listener));
-    }
-
-    @Override
-    public Collection<Consumer<Status>> statusChangeListeners() {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * Returns the map (partition) to which the specified key maps.
-     * @param key key
-     * @return AsyncConsistentMap to which key maps
-     */
-    private AsyncConsistentMap<K, V> getMap(K key) {
-        return partitions.get(keyHasher.hash(key));
-    }
-
-    /**
-     * Returns all the constituent maps.
-     * @return collection of maps.
-     */
-    private Collection<AsyncConsistentMap<K, V>> getMaps() {
-        return partitions.values();
-    }
-
-    private class PartitionedMultimapIterator<K, V> implements AsyncIterator<Map.Entry<K, Versioned<V>>> {
-        private final Iterator<AsyncIterator<Entry<K, Versioned<V>>>> iterators;
-        private volatile AsyncIterator<Entry<K, Versioned<V>>> iterator;
-
-        public PartitionedMultimapIterator(List<AsyncIterator<Entry<K, Versioned<V>>>> iterators) {
-            this.iterators = iterators.iterator();
-        }
-
-        @Override
-        public CompletableFuture<Boolean> hasNext() {
-            if (iterator == null && iterators.hasNext()) {
-                iterator = iterators.next();
-            }
-            if (iterator == null) {
-                return CompletableFuture.completedFuture(false);
-            }
-            return iterator.hasNext()
-                .thenCompose(hasNext -> {
-                    if (!hasNext) {
-                        iterator = null;
-                        return hasNext();
-                    }
-                    return CompletableFuture.completedFuture(true);
-                });
-        }
-
-        @Override
-        public CompletableFuture<Entry<K, Versioned<V>>> next() {
-            if (iterator == null && iterators.hasNext()) {
-                iterator = iterators.next();
-            }
-            if (iterator == null) {
-                return Tools.exceptionalFuture(new NoSuchElementException());
-            }
-            return iterator.next();
-        }
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncConsistentMultimap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncConsistentMultimap.java
deleted file mode 100644
index a4e0bf9..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncConsistentMultimap.java
+++ /dev/null
@@ -1,277 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * 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.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.NoSuchElementException;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.Consumer;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.HashMultiset;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Iterables;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Multiset;
-import com.google.common.collect.Multisets;
-import org.onlab.util.Match;
-import org.onlab.util.Tools;
-import org.onosproject.cluster.PartitionId;
-import org.onosproject.store.service.AsyncConsistentMultimap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MultimapEventListener;
-import org.onosproject.store.service.Versioned;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * {@link AsyncConsistentMultimap} that has its entries partitioned horizontally across
- * several {@link AsyncConsistentMultimap maps}.
- *
- * @param <K> key type
- * @param <V> value type
- */
-public class PartitionedAsyncConsistentMultimap<K, V> implements AsyncConsistentMultimap<K, V> {
-
-    private final String name;
-    private final TreeMap<PartitionId, AsyncConsistentMultimap<K, V>> partitions = Maps.newTreeMap();
-    private final Hasher<K> keyHasher;
-
-    public PartitionedAsyncConsistentMultimap(String name,
-        Map<PartitionId, AsyncConsistentMultimap<K, V>> partitions,
-        Hasher<K> keyHasher) {
-        this.name = name;
-        this.partitions.putAll(checkNotNull(partitions));
-        this.keyHasher = checkNotNull(keyHasher);
-    }
-
-    @Override
-    public String name() {
-        return name;
-    }
-
-    @Override
-    public CompletableFuture<Integer> size() {
-        return Tools.allOf(getMultimaps().stream().map(m -> m.size()).collect(Collectors.toList()),
-            Math::addExact,
-            0);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> isEmpty() {
-        return size().thenApply(size -> size == 0);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsKey(K key) {
-        return getMultimap(key).containsKey(key);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsValue(V value) {
-        return Tools.firstOf(getMultimaps().stream().map(m -> m.containsValue(value)).collect(Collectors.toList()),
-            Match.ifValue(true),
-            false);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends V>>> get(K key) {
-        return getMultimap(key).get(key);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsEntry(K key, V value) {
-        return getMultimap(key).containsEntry(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> put(K key, V value) {
-        return getMultimap(key).put(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends V>>> putAndGet(K key, V value) {
-        return getMultimap(key).putAndGet(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> removeAll(K key, Collection<? extends V> values) {
-        return getMultimap(key).removeAll(key, values);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends V>>> removeAll(K key) {
-        return getMultimap(key).removeAll(key);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> putAll(K key, Collection<? extends V> values) {
-        return getMultimap(key).putAll(key, values);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends V>>> replaceValues(K key, Collection<V> values) {
-        return getMultimap(key).replaceValues(key, values);
-    }
-
-    @Override
-    public CompletableFuture<Map<K, Collection<V>>> asMap() {
-        throw new UnsupportedOperationException("Expensive operation.");
-    }
-
-    @Override
-    public CompletableFuture<Void> clear() {
-        return CompletableFuture.allOf(getMultimaps().stream()
-            .map(map -> map.clear())
-            .toArray(CompletableFuture[]::new));
-    }
-
-    @Override
-    public CompletableFuture<Set<K>> keySet() {
-        return Tools.allOf(getMultimaps().stream().map(m -> m.keySet()).collect(Collectors.toList()),
-            (s1, s2) -> ImmutableSet.<K>builder().addAll(s1).addAll(s2).build(),
-            ImmutableSet.of());
-    }
-
-    @Override
-    public CompletableFuture<Multiset<K>> keys() {
-        return Tools.allOf(getMultimaps().stream().map(m -> m.keys()).collect(Collectors.toList()))
-            .thenApply(results -> results.stream().reduce(Multisets::sum).orElse(HashMultiset.create()));
-    }
-
-    @Override
-    public CompletableFuture<Multiset<V>> values() {
-        return Tools.allOf(getMultimaps().stream().map(m -> m.values()).collect(Collectors.toList()))
-            .thenApply(results -> results.stream().reduce(Multisets::sum).orElse(HashMultiset.create()));
-    }
-
-    @Override
-    public CompletableFuture<Collection<Entry<K, V>>> entries() {
-        return Tools.allOf(getMultimaps().stream().map(m -> m.entries()).collect(Collectors.toList()))
-            .thenApply(results -> results.stream().reduce((s1, s2) -> ImmutableList.copyOf(Iterables.concat(s1, s2)))
-                .orElse(ImmutableList.of()));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(K key, V value) {
-        return getMultimap(key).remove(key, value);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends V>>> removeAndGet(K key, V value) {
-        return getMultimap(key).removeAndGet(key, value);
-    }
-
-    @Override
-    public CompletableFuture<AsyncIterator<Entry<K, V>>> iterator() {
-        return Tools.allOf(getMultimaps().stream().map(m -> m.iterator()).collect(Collectors.toList()))
-            .thenApply(PartitionedMultimapIterator::new);
-    }
-
-    @Override
-    public CompletableFuture<Void> addListener(MultimapEventListener<K, V> listener, Executor executor) {
-        return CompletableFuture.allOf(getMultimaps().stream()
-            .map(map -> map.addListener(listener, executor))
-            .toArray(CompletableFuture[]::new));
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(MultimapEventListener<K, V> listener) {
-        return CompletableFuture.allOf(getMultimaps().stream()
-            .map(map -> map.removeListener(listener))
-            .toArray(CompletableFuture[]::new));
-    }
-
-    @Override
-    public void addStatusChangeListener(Consumer<Status> listener) {
-        partitions.values().forEach(map -> map.addStatusChangeListener(listener));
-    }
-
-    @Override
-    public void removeStatusChangeListener(Consumer<Status> listener) {
-        partitions.values().forEach(map -> map.removeStatusChangeListener(listener));
-    }
-
-    @Override
-    public Collection<Consumer<Status>> statusChangeListeners() {
-        throw new UnsupportedOperationException();
-    }
-
-    /**
-     * Returns the map (partition) to which the specified key maps.
-     *
-     * @param key key
-     * @return AsyncConsistentMap to which key maps
-     */
-    private AsyncConsistentMultimap<K, V> getMultimap(K key) {
-        return partitions.get(keyHasher.hash(key));
-    }
-
-    /**
-     * Returns all the constituent maps.
-     *
-     * @return collection of maps.
-     */
-    private Collection<AsyncConsistentMultimap<K, V>> getMultimaps() {
-        return partitions.values();
-    }
-
-    private class PartitionedMultimapIterator<K, V> implements AsyncIterator<Map.Entry<K, V>> {
-        private final Iterator<AsyncIterator<Entry<K, V>>> iterators;
-        private volatile AsyncIterator<Entry<K, V>> iterator;
-
-        public PartitionedMultimapIterator(List<AsyncIterator<Entry<K, V>>> iterators) {
-            this.iterators = iterators.iterator();
-        }
-
-        @Override
-        public CompletableFuture<Boolean> hasNext() {
-            if (iterator == null && iterators.hasNext()) {
-                iterator = iterators.next();
-            }
-            if (iterator == null) {
-                return CompletableFuture.completedFuture(false);
-            }
-            return iterator.hasNext()
-                .thenCompose(hasNext -> {
-                    if (!hasNext) {
-                        iterator = null;
-                        return hasNext();
-                    }
-                    return CompletableFuture.completedFuture(true);
-                });
-        }
-
-        @Override
-        public CompletableFuture<Entry<K, V>> next() {
-            if (iterator == null && iterators.hasNext()) {
-                iterator = iterators.next();
-            }
-            if (iterator == null) {
-                return Tools.exceptionalFuture(new NoSuchElementException());
-            }
-            return iterator.next();
-        }
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncDocumentTree.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncDocumentTree.java
deleted file mode 100644
index 2d42d4f..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncDocumentTree.java
+++ /dev/null
@@ -1,217 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.Collection;
-import java.util.List;
-import java.util.Map;
-import java.util.Objects;
-import java.util.TreeMap;
-import java.util.concurrent.CompletableFuture;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Maps;
-import org.onlab.util.Tools;
-import org.onosproject.cluster.PartitionId;
-import org.onosproject.store.primitives.NodeUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.AsyncDocumentTree;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTreeListener;
-import org.onosproject.store.service.NoSuchDocumentPathException;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * Partitioned asynchronous document tree.
- */
-public class PartitionedAsyncDocumentTree<V> implements AsyncDocumentTree<V> {
-
-    private final String name;
-    private final TreeMap<PartitionId, AsyncDocumentTree<V>> partitions = Maps.newTreeMap();
-    private final Hasher<DocumentPath> pathHasher;
-
-    public PartitionedAsyncDocumentTree(
-            String name,
-            Map<PartitionId, AsyncDocumentTree<V>> partitions,
-            Hasher<DocumentPath> pathHasher) {
-        this.name = name;
-        this.partitions.putAll(checkNotNull(partitions));
-        this.pathHasher = checkNotNull(pathHasher);
-    }
-
-    @Override
-    public String name() {
-        return name;
-    }
-
-    @Override
-    public DocumentPath root() {
-        return DocumentPath.ROOT;
-    }
-
-    /**
-     * Returns the document tree (partition) to which the specified path maps.
-     *
-     * @param path path
-     * @return AsyncConsistentMap to which path maps
-     */
-    private AsyncDocumentTree<V> partition(DocumentPath path) {
-        return partitions.get(pathHasher.hash(path));
-    }
-
-    /**
-     * Returns all the constituent trees.
-     *
-     * @return collection of partitions.
-     */
-    private Collection<AsyncDocumentTree<V>> partitions() {
-        return partitions.values();
-    }
-
-    @Override
-    public CompletableFuture<Map<String, Versioned<V>>> getChildren(DocumentPath path) {
-        return Tools.allOf(partitions().stream()
-                .map(partition -> partition.getChildren(path).exceptionally(r -> null))
-                .collect(Collectors.toList())).thenApply(allChildren -> {
-            Map<String, Versioned<V>> children = Maps.newLinkedHashMap();
-            allChildren.stream().filter(Objects::nonNull).forEach(children::putAll);
-            return children;
-        });
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> get(DocumentPath path) {
-        return partition(path).get(path);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> set(DocumentPath path, V value) {
-        return partition(path).set(path, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> create(DocumentPath path, V value) {
-        if (path.parent() == null) {
-            // create value on root
-            return partition(path).createRecursive(path, value);
-        }
-        // TODO: This operation is not atomic
-        return partition(path.parent()).get(path.parent()).thenCompose(parentValue -> {
-            if (parentValue == null) {
-                return Tools.exceptionalFuture(new NoSuchDocumentPathException(String.valueOf(path.parent())));
-            } else {
-                // not atomic: parent did exist at some point, so moving forward
-                return partition(path).createRecursive(path, value);
-            }
-        });
-    }
-
-    @Override
-    public CompletableFuture<Boolean> createRecursive(DocumentPath path, V value) {
-        return partition(path).createRecursive(path, value);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(DocumentPath path, V newValue, long version) {
-        return partition(path).replace(path, newValue, version);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(DocumentPath path, V newValue, V currentValue) {
-        return partition(path).replace(path, newValue, currentValue);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> removeNode(DocumentPath path) {
-        return partition(path).removeNode(path);
-    }
-
-    @Override
-    public CompletableFuture<Void> addListener(DocumentPath path, DocumentTreeListener<V> listener) {
-        return CompletableFuture.allOf(partitions().stream()
-                .map(map -> map.addListener(path, listener))
-                .toArray(CompletableFuture[]::new));
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(DocumentTreeListener<V> listener) {
-        return CompletableFuture.allOf(partitions().stream()
-                .map(map -> map.removeListener(listener))
-                .toArray(CompletableFuture[]::new));
-    }
-
-    @Override
-    public CompletableFuture<Version> begin(TransactionId transactionId) {
-        return partitions().stream()
-            .map(p -> p.begin(transactionId))
-            // returning lowest Version
-            .reduce((f1, f2) -> f1.thenCombine(f2, Tools::min))
-            .orElse(Tools.exceptionalFuture(new IllegalStateException("Empty partitions?")));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepare(TransactionLog<NodeUpdate<V>> transactionLog) {
-        Map<AsyncDocumentTree<V>, List<NodeUpdate<V>>> perPart =
-                transactionLog.records().stream()
-                    .collect(Collectors.groupingBy(nu -> partition(nu.path())));
-
-        // must walk all partitions to ensure empty TransactionLog will
-        // be issued against no-op partitions in order for commit to succeed
-        return partitions().stream()
-                .map(p -> p.prepare(new TransactionLog<>(transactionLog.transactionId(),
-                                    transactionLog.version(),
-                                    perPart.getOrDefault(p, ImmutableList.of()))))
-                .reduce((f1, f2) -> f1.thenCombine(f2, Boolean::logicalAnd))
-                .orElse(CompletableFuture.completedFuture(true));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<NodeUpdate<V>> transactionLog) {
-        // Note: cannot call prepareAndCommit on each partition,
-        // must check all partitions are prepare()-ed first.
-        return prepare(transactionLog)
-                .thenApply(prepOk -> {
-                    if (prepOk) {
-                        commit(transactionLog.transactionId());
-                        return true;
-                    } else {
-                        return false;
-                    }
-                });
-
-    }
-
-    @Override
-    public CompletableFuture<Void> commit(TransactionId transactionId) {
-        return CompletableFuture.allOf(partitions().stream()
-                                           .map(p -> p.commit(transactionId))
-                                           .toArray(CompletableFuture[]::new)
-                                       );
-    }
-
-    @Override
-    public CompletableFuture<Void> rollback(TransactionId transactionId) {
-        return CompletableFuture.allOf(partitions().stream()
-                                       .map(p -> p.rollback(transactionId))
-                                       .toArray(CompletableFuture[]::new)
-                                   );
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncLeaderElector.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncLeaderElector.java
deleted file mode 100644
index 78bcf82..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedAsyncLeaderElector.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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 com.google.common.base.Preconditions.checkNotNull;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.TreeMap;
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Consumer;
-
-import org.onosproject.cluster.Leadership;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.cluster.PartitionId;
-import org.onosproject.event.Change;
-import org.onosproject.store.service.AsyncLeaderElector;
-import com.google.common.collect.Maps;
-
-/**
- * {@link AsyncLeaderElector} that has its topics partitioned horizontally across
- * several {@link AsyncLeaderElector leader electors}.
- */
-public class PartitionedAsyncLeaderElector implements AsyncLeaderElector {
-
-    private final String name;
-    private final TreeMap<PartitionId, AsyncLeaderElector> partitions = Maps.newTreeMap();
-    private final Hasher<String> topicHasher;
-
-    public PartitionedAsyncLeaderElector(String name,
-            Map<PartitionId, AsyncLeaderElector> partitions,
-            Hasher<String> topicHasher) {
-        this.name = name;
-        this.partitions.putAll(checkNotNull(partitions));
-        this.topicHasher = checkNotNull(topicHasher);
-    }
-
-    @Override
-    public String name() {
-        return name;
-    }
-
-    @Override
-    public CompletableFuture<Leadership> run(String topic, NodeId nodeId) {
-        return getLeaderElector(topic).run(topic, nodeId);
-    }
-
-    @Override
-    public CompletableFuture<Void> withdraw(String topic) {
-        return getLeaderElector(topic).withdraw(topic);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> anoint(String topic, NodeId nodeId) {
-        return getLeaderElector(topic).anoint(topic, nodeId);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> promote(String topic, NodeId nodeId) {
-        return getLeaderElector(topic).promote(topic, nodeId);
-    }
-
-    @Override
-    public CompletableFuture<Void> evict(NodeId nodeId) {
-        return CompletableFuture.allOf(getLeaderElectors().stream()
-                                                          .map(le -> le.evict(nodeId))
-                                                          .toArray(CompletableFuture[]::new));
-    }
-
-    @Override
-    public CompletableFuture<Leadership> getLeadership(String topic) {
-        return getLeaderElector(topic).getLeadership(topic);
-    }
-
-    @Override
-    public CompletableFuture<Map<String, Leadership>> getLeaderships() {
-        Map<String, Leadership> leaderships = Maps.newConcurrentMap();
-        return CompletableFuture.allOf(getLeaderElectors().stream()
-                                                          .map(le -> le.getLeaderships()
-                                                                       .thenAccept(m -> leaderships.putAll(m)))
-                                                          .toArray(CompletableFuture[]::new))
-                                .thenApply(v -> leaderships);
-    }
-
-    @Override
-    public CompletableFuture<Void> addChangeListener(Consumer<Change<Leadership>> listener) {
-        return CompletableFuture.allOf(getLeaderElectors().stream()
-                                                          .map(map -> map.addChangeListener(listener))
-                                                          .toArray(CompletableFuture[]::new));
-    }
-
-    @Override
-    public CompletableFuture<Void> removeChangeListener(Consumer<Change<Leadership>> listener) {
-        return CompletableFuture.allOf(getLeaderElectors().stream()
-                                                          .map(map -> map.removeChangeListener(listener))
-                                                          .toArray(CompletableFuture[]::new));
-    }
-
-    /**
-     * Returns the leaderElector (partition) to which the specified topic maps.
-     * @param topic topic name
-     * @return AsyncLeaderElector to which topic maps
-     */
-    private AsyncLeaderElector getLeaderElector(String topic) {
-        return partitions.get(topicHasher.hash(topic));
-    }
-
-    /**
-     * Returns all the constituent leader electors.
-     * @return collection of leader electors.
-     */
-    private Collection<AsyncLeaderElector> getLeaderElectors() {
-        return partitions.values();
-    }
-
-    @Override
-    public void addStatusChangeListener(Consumer<Status> listener) {
-        partitions.values().forEach(elector -> elector.addStatusChangeListener(listener));
-    }
-
-    @Override
-    public void removeStatusChangeListener(Consumer<Status> listener) {
-        partitions.values().forEach(elector -> elector.removeStatusChangeListener(listener));
-    }
-
-    @Override
-    public Collection<Consumer<Status>> statusChangeListeners() {
-        throw new UnsupportedOperationException();
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedTransactionalMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedTransactionalMap.java
deleted file mode 100644
index 4f6d5b6..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/PartitionedTransactionalMap.java
+++ /dev/null
@@ -1,99 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.Collection;
-import java.util.Map;
-
-import com.google.common.base.MoreObjects;
-import org.onosproject.cluster.PartitionId;
-import org.onosproject.store.service.TransactionalMap;
-
-/**
- * Partitioned transactional map.
- */
-public class PartitionedTransactionalMap<K, V> implements TransactionalMap<K, V> {
-    protected final Map<PartitionId, TransactionalMapParticipant<K, V>> partitions;
-    protected final Hasher<K> hasher;
-
-    public PartitionedTransactionalMap(
-            Map<PartitionId, TransactionalMapParticipant<K, V>> partitions, Hasher<K> hasher) {
-        this.partitions = partitions;
-        this.hasher = hasher;
-    }
-
-    /**
-     * Returns the collection of map partitions.
-     *
-     * @return a collection of map partitions
-     */
-    @SuppressWarnings("unchecked")
-    Collection<TransactionParticipant> participants() {
-        return (Collection) partitions.values();
-    }
-
-    /**
-     * Returns the partition for the given key.
-     *
-     * @param key the key for which to return the partition
-     * @return the partition for the given key
-     */
-    private TransactionalMap<K, V> partition(K key) {
-        return partitions.get(hasher.hash(key));
-    }
-
-    @Override
-    public V get(K key) {
-        return partition(key).get(key);
-    }
-
-    @Override
-    public boolean containsKey(K key) {
-        return partition(key).containsKey(key);
-    }
-
-    @Override
-    public V put(K key, V value) {
-        return partition(key).put(key, value);
-    }
-
-    @Override
-    public V remove(K key) {
-        return partition(key).remove(key);
-    }
-
-    @Override
-    public V putIfAbsent(K key, V value) {
-        return partition(key).putIfAbsent(key, value);
-    }
-
-    @Override
-    public boolean remove(K key, V value) {
-        return partition(key).remove(key, value);
-    }
-
-    @Override
-    public boolean replace(K key, V oldValue, V newValue) {
-        return partition(key).replace(key, oldValue, newValue);
-    }
-
-    @Override
-    public String toString() {
-        return MoreObjects.toStringHelper(this)
-                .add("partitions", partitions.values())
-                .toString();
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftClientCommunicator.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftClientCommunicator.java
deleted file mode 100644
index 45f40ae..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftClientCommunicator.java
+++ /dev/null
@@ -1,124 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.Collection;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.Consumer;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-
-import io.atomix.protocols.raft.cluster.MemberId;
-import io.atomix.protocols.raft.protocol.CloseSessionRequest;
-import io.atomix.protocols.raft.protocol.CloseSessionResponse;
-import io.atomix.protocols.raft.protocol.CommandRequest;
-import io.atomix.protocols.raft.protocol.CommandResponse;
-import io.atomix.protocols.raft.protocol.HeartbeatRequest;
-import io.atomix.protocols.raft.protocol.HeartbeatResponse;
-import io.atomix.protocols.raft.protocol.KeepAliveRequest;
-import io.atomix.protocols.raft.protocol.KeepAliveResponse;
-import io.atomix.protocols.raft.protocol.MetadataRequest;
-import io.atomix.protocols.raft.protocol.MetadataResponse;
-import io.atomix.protocols.raft.protocol.OpenSessionRequest;
-import io.atomix.protocols.raft.protocol.OpenSessionResponse;
-import io.atomix.protocols.raft.protocol.PublishRequest;
-import io.atomix.protocols.raft.protocol.QueryRequest;
-import io.atomix.protocols.raft.protocol.QueryResponse;
-import io.atomix.protocols.raft.protocol.RaftClientProtocol;
-import io.atomix.protocols.raft.protocol.ResetRequest;
-import io.atomix.protocols.raft.session.SessionId;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
-import org.onosproject.store.service.Serializer;
-
-/**
- * Raft client protocol that uses a cluster communicator.
- */
-public class RaftClientCommunicator extends RaftCommunicator implements RaftClientProtocol {
-
-    public RaftClientCommunicator(
-            String prefix,
-            Serializer serializer,
-            ClusterCommunicationService clusterCommunicator) {
-        super(new RaftMessageContext(prefix), serializer, clusterCommunicator);
-    }
-
-    @Override
-    public CompletableFuture<OpenSessionResponse> openSession(MemberId memberId, OpenSessionRequest request) {
-        return sendAndReceive(context.openSessionSubject, request, memberId);
-    }
-
-    @Override
-    public CompletableFuture<CloseSessionResponse> closeSession(MemberId memberId, CloseSessionRequest request) {
-        return sendAndReceive(context.closeSessionSubject, request, memberId);
-    }
-
-    @Override
-    public CompletableFuture<KeepAliveResponse> keepAlive(MemberId memberId, KeepAliveRequest request) {
-        return sendAndReceive(context.keepAliveSubject, request, memberId);
-    }
-
-    @Override
-    public CompletableFuture<QueryResponse> query(MemberId memberId, QueryRequest request) {
-        return sendAndReceive(context.querySubject, request, memberId);
-    }
-
-    @Override
-    public CompletableFuture<CommandResponse> command(MemberId memberId, CommandRequest request) {
-        return sendAndReceive(context.commandSubject, request, memberId);
-    }
-
-    @Override
-    public CompletableFuture<MetadataResponse> metadata(MemberId memberId, MetadataRequest request) {
-        return sendAndReceive(context.metadataSubject, request, memberId);
-    }
-
-    @Override
-    public void registerHeartbeatHandler(Function<HeartbeatRequest, CompletableFuture<HeartbeatResponse>> function) {
-        clusterCommunicator.addSubscriber(context.heartbeatSubject, serializer::decode, function, serializer::encode);
-    }
-
-    @Override
-    public void unregisterHeartbeatHandler() {
-        clusterCommunicator.removeSubscriber(context.heartbeatSubject);
-    }
-
-    @Override
-    public void reset(Collection<MemberId> members, ResetRequest request) {
-        Set<NodeId> nodes = members.stream().map(m -> NodeId.nodeId(m.id())).collect(Collectors.toSet());
-        clusterCommunicator.multicast(
-                request,
-                context.resetSubject(request.session()),
-                serializer::encode,
-                nodes);
-    }
-
-    @Override
-    public void registerPublishListener(SessionId sessionId, Consumer<PublishRequest> listener, Executor executor) {
-        clusterCommunicator.addSubscriber(
-                context.publishSubject(sessionId.id()),
-                serializer::decode,
-                listener,
-                executor);
-    }
-
-    @Override
-    public void unregisterPublishListener(SessionId sessionId) {
-        clusterCommunicator.removeSubscriber(context.publishSubject(sessionId.id()));
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftCommunicator.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftCommunicator.java
deleted file mode 100644
index 1117ab9..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftCommunicator.java
+++ /dev/null
@@ -1,71 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.net.ConnectException;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.CompletionException;
-
-import io.atomix.protocols.raft.RaftException;
-import io.atomix.protocols.raft.cluster.MemberId;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
-import org.onosproject.store.cluster.messaging.MessageSubject;
-import org.onosproject.store.cluster.messaging.MessagingException;
-import org.onosproject.store.service.Serializer;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * Abstract base class for Raft protocol client/server.
- */
-public abstract class RaftCommunicator {
-    protected final RaftMessageContext context;
-    protected final Serializer serializer;
-    protected final ClusterCommunicationService clusterCommunicator;
-
-    public RaftCommunicator(
-            RaftMessageContext context,
-            Serializer serializer,
-            ClusterCommunicationService clusterCommunicator) {
-        this.context = checkNotNull(context, "context cannot be null");
-        this.serializer = checkNotNull(serializer, "serializer cannot be null");
-        this.clusterCommunicator = checkNotNull(clusterCommunicator, "clusterCommunicator cannot be null");
-    }
-
-    protected <T, U> CompletableFuture<U> sendAndReceive(MessageSubject subject, T request, MemberId memberId) {
-        CompletableFuture<U> future = new CompletableFuture<>();
-        clusterCommunicator.<T, U>sendAndReceive(
-                request, subject, serializer::encode, serializer::decode, NodeId.nodeId(memberId.id()))
-                .whenComplete((result, error) -> {
-                    if (error == null) {
-                        future.complete(result);
-                    } else {
-                        if (error instanceof CompletionException) {
-                            error = error.getCause();
-                        }
-                        if (error instanceof MessagingException.NoRemoteHandler) {
-                            error = new ConnectException(error.getMessage());
-                        } else if (error instanceof MessagingException.RemoteHandlerFailure
-                                || error instanceof MessagingException.ProtocolException) {
-                            error = new RaftException.ProtocolException(error.getMessage());
-                        }
-                        future.completeExceptionally(error);
-                    }
-                });
-        return future;
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftMessageContext.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftMessageContext.java
deleted file mode 100644
index 9f1f1bc..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftMessageContext.java
+++ /dev/null
@@ -1,97 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.cluster.messaging.MessageSubject;
-
-/**
- * Protocol message context.
- */
-final class RaftMessageContext {
-    private final String prefix;
-    final MessageSubject heartbeatSubject;
-    final MessageSubject openSessionSubject;
-    final MessageSubject closeSessionSubject;
-    final MessageSubject keepAliveSubject;
-    final MessageSubject querySubject;
-    final MessageSubject commandSubject;
-    final MessageSubject metadataSubject;
-    final MessageSubject joinSubject;
-    final MessageSubject leaveSubject;
-    final MessageSubject configureSubject;
-    final MessageSubject reconfigureSubject;
-    final MessageSubject installSubject;
-    final MessageSubject pollSubject;
-    final MessageSubject voteSubject;
-    final MessageSubject appendSubject;
-    final MessageSubject transferSubject;
-
-    RaftMessageContext(String prefix) {
-        this.prefix = prefix;
-        this.heartbeatSubject = getSubject(prefix, "heartbeat");
-        this.openSessionSubject = getSubject(prefix, "open");
-        this.closeSessionSubject = getSubject(prefix, "close");
-        this.keepAliveSubject = getSubject(prefix, "keep-alive");
-        this.querySubject = getSubject(prefix, "query");
-        this.commandSubject = getSubject(prefix, "command");
-        this.metadataSubject = getSubject(prefix, "metadata");
-        this.joinSubject = getSubject(prefix, "join");
-        this.leaveSubject = getSubject(prefix, "leave");
-        this.configureSubject = getSubject(prefix, "configure");
-        this.reconfigureSubject = getSubject(prefix, "reconfigure");
-        this.installSubject = getSubject(prefix, "install");
-        this.pollSubject = getSubject(prefix, "poll");
-        this.voteSubject = getSubject(prefix, "vote");
-        this.appendSubject = getSubject(prefix, "append");
-        this.transferSubject = getSubject(prefix, "transfer");
-    }
-
-    private static MessageSubject getSubject(String prefix, String type) {
-        if (prefix == null) {
-            return new MessageSubject(type);
-        } else {
-            return new MessageSubject(String.format("%s-%s", prefix, type));
-        }
-    }
-
-    /**
-     * Returns the publish subject for the given session.
-     *
-     * @param sessionId the session for which to return the publish subject
-     * @return the publish subject for the given session
-     */
-    MessageSubject publishSubject(long sessionId) {
-        if (prefix == null) {
-            return new MessageSubject(String.format("publish-%d", sessionId));
-        } else {
-            return new MessageSubject(String.format("%s-publish-%d", prefix, sessionId));
-        }
-    }
-
-    /**
-     * Returns the reset subject for the given session.
-     *
-     * @param sessionId the session for which to return the reset subject
-     * @return the reset subject for the given session
-     */
-    MessageSubject resetSubject(long sessionId) {
-        if (prefix == null) {
-            return new MessageSubject(String.format("reset-%d", sessionId));
-        } else {
-            return new MessageSubject(String.format("%s-reset-%d", prefix, sessionId));
-        }
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftServerCommunicator.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftServerCommunicator.java
deleted file mode 100644
index f58db45..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/RaftServerCommunicator.java
+++ /dev/null
@@ -1,324 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.Consumer;
-import java.util.function.Function;
-
-import io.atomix.protocols.raft.cluster.MemberId;
-import io.atomix.protocols.raft.protocol.AppendRequest;
-import io.atomix.protocols.raft.protocol.AppendResponse;
-import io.atomix.protocols.raft.protocol.CloseSessionRequest;
-import io.atomix.protocols.raft.protocol.CloseSessionResponse;
-import io.atomix.protocols.raft.protocol.CommandRequest;
-import io.atomix.protocols.raft.protocol.CommandResponse;
-import io.atomix.protocols.raft.protocol.ConfigureRequest;
-import io.atomix.protocols.raft.protocol.ConfigureResponse;
-import io.atomix.protocols.raft.protocol.HeartbeatRequest;
-import io.atomix.protocols.raft.protocol.HeartbeatResponse;
-import io.atomix.protocols.raft.protocol.InstallRequest;
-import io.atomix.protocols.raft.protocol.InstallResponse;
-import io.atomix.protocols.raft.protocol.JoinRequest;
-import io.atomix.protocols.raft.protocol.JoinResponse;
-import io.atomix.protocols.raft.protocol.KeepAliveRequest;
-import io.atomix.protocols.raft.protocol.KeepAliveResponse;
-import io.atomix.protocols.raft.protocol.LeaveRequest;
-import io.atomix.protocols.raft.protocol.LeaveResponse;
-import io.atomix.protocols.raft.protocol.MetadataRequest;
-import io.atomix.protocols.raft.protocol.MetadataResponse;
-import io.atomix.protocols.raft.protocol.OpenSessionRequest;
-import io.atomix.protocols.raft.protocol.OpenSessionResponse;
-import io.atomix.protocols.raft.protocol.PollRequest;
-import io.atomix.protocols.raft.protocol.PollResponse;
-import io.atomix.protocols.raft.protocol.PublishRequest;
-import io.atomix.protocols.raft.protocol.QueryRequest;
-import io.atomix.protocols.raft.protocol.QueryResponse;
-import io.atomix.protocols.raft.protocol.RaftServerProtocol;
-import io.atomix.protocols.raft.protocol.ReconfigureRequest;
-import io.atomix.protocols.raft.protocol.ReconfigureResponse;
-import io.atomix.protocols.raft.protocol.ResetRequest;
-import io.atomix.protocols.raft.protocol.TransferRequest;
-import io.atomix.protocols.raft.protocol.TransferResponse;
-import io.atomix.protocols.raft.protocol.VoteRequest;
-import io.atomix.protocols.raft.protocol.VoteResponse;
-import io.atomix.protocols.raft.session.SessionId;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
-import org.onosproject.store.service.Serializer;
-
-/**
- * Raft server protocol that uses a {@link ClusterCommunicationService}.
- */
-public class RaftServerCommunicator extends RaftCommunicator implements RaftServerProtocol {
-
-    public RaftServerCommunicator(
-            String prefix,
-            Serializer serializer,
-            ClusterCommunicationService clusterCommunicator) {
-        super(new RaftMessageContext(prefix), serializer, clusterCommunicator);
-    }
-
-    @Override
-    public CompletableFuture<OpenSessionResponse> openSession(MemberId memberId, OpenSessionRequest request) {
-        return sendAndReceive(context.openSessionSubject, request, memberId);
-    }
-
-    @Override
-    public CompletableFuture<CloseSessionResponse> closeSession(MemberId memberId, CloseSessionRequest request) {
-        return sendAndReceive(context.closeSessionSubject, request, memberId);
-    }
-
-    @Override
-    public CompletableFuture<KeepAliveResponse> keepAlive(MemberId memberId, KeepAliveRequest request) {
-        return sendAndReceive(context.keepAliveSubject, request, memberId);
-    }
-
-    @Override
-    public CompletableFuture<QueryResponse> query(MemberId memberId, QueryRequest request) {
-        return sendAndReceive(context.querySubject, request, memberId);
-    }
-
-    @Override
-    public CompletableFuture<CommandResponse> command(MemberId memberId, CommandRequest request) {
-        return sendAndReceive(context.commandSubject, request, memberId);
-    }
-
-    @Override
-    public CompletableFuture<MetadataResponse> metadata(MemberId memberId, MetadataRequest request) {
-        return sendAndReceive(context.metadataSubject, request, memberId);
-    }
-
-    @Override
-    public CompletableFuture<JoinResponse> join(MemberId memberId, JoinRequest request) {
-        return sendAndReceive(context.joinSubject, request, memberId);
-    }
-
-    @Override
-    public CompletableFuture<LeaveResponse> leave(MemberId memberId, LeaveRequest request) {
-        return sendAndReceive(context.leaveSubject, request, memberId);
-    }
-
-    @Override
-    public CompletableFuture<ConfigureResponse> configure(MemberId memberId, ConfigureRequest request) {
-        return sendAndReceive(context.configureSubject, request, memberId);
-    }
-
-    @Override
-    public CompletableFuture<ReconfigureResponse> reconfigure(MemberId memberId, ReconfigureRequest request) {
-        return sendAndReceive(context.reconfigureSubject, request, memberId);
-    }
-
-    @Override
-    public CompletableFuture<InstallResponse> install(MemberId memberId, InstallRequest request) {
-        return sendAndReceive(context.installSubject, request, memberId);
-    }
-
-    @Override
-    public CompletableFuture<PollResponse> poll(MemberId memberId, PollRequest request) {
-        return sendAndReceive(context.pollSubject, request, memberId);
-    }
-
-    @Override
-    public CompletableFuture<VoteResponse> vote(MemberId memberId, VoteRequest request) {
-        return sendAndReceive(context.voteSubject, request, memberId);
-    }
-
-    @Override
-    public CompletableFuture<AppendResponse> append(MemberId memberId, AppendRequest request) {
-        return sendAndReceive(context.appendSubject, request, memberId);
-    }
-
-    @Override
-    public CompletableFuture<TransferResponse> transfer(MemberId memberId, TransferRequest request) {
-        return sendAndReceive(context.transferSubject, request, memberId);
-    }
-
-    @Override
-    public void publish(MemberId memberId, PublishRequest request) {
-        clusterCommunicator.unicast(request,
-                context.publishSubject(request.session()), serializer::encode, NodeId.nodeId(memberId.id()));
-    }
-
-    @Override
-    public CompletableFuture<HeartbeatResponse> heartbeat(MemberId memberId, HeartbeatRequest request) {
-        return sendAndReceive(context.heartbeatSubject, request, memberId);
-    }
-
-    @Override
-    public void registerOpenSessionHandler(
-            Function<OpenSessionRequest, CompletableFuture<OpenSessionResponse>> handler) {
-        clusterCommunicator.addSubscriber(context.openSessionSubject, serializer::decode, handler, serializer::encode);
-    }
-
-    @Override
-    public void unregisterOpenSessionHandler() {
-        clusterCommunicator.removeSubscriber(context.openSessionSubject);
-    }
-
-    @Override
-    public void registerCloseSessionHandler(
-            Function<CloseSessionRequest, CompletableFuture<CloseSessionResponse>> handler) {
-        clusterCommunicator.addSubscriber(context.closeSessionSubject, serializer::decode, handler, serializer::encode);
-    }
-
-    @Override
-    public void unregisterCloseSessionHandler() {
-        clusterCommunicator.removeSubscriber(context.closeSessionSubject);
-    }
-
-    @Override
-    public void registerKeepAliveHandler(Function<KeepAliveRequest, CompletableFuture<KeepAliveResponse>> handler) {
-        clusterCommunicator.addSubscriber(context.keepAliveSubject, serializer::decode, handler, serializer::encode);
-    }
-
-    @Override
-    public void unregisterKeepAliveHandler() {
-        clusterCommunicator.removeSubscriber(context.keepAliveSubject);
-    }
-
-    @Override
-    public void registerQueryHandler(Function<QueryRequest, CompletableFuture<QueryResponse>> handler) {
-        clusterCommunicator.addSubscriber(context.querySubject, serializer::decode, handler, serializer::encode);
-    }
-
-    @Override
-    public void unregisterQueryHandler() {
-        clusterCommunicator.removeSubscriber(context.querySubject);
-    }
-
-    @Override
-    public void registerCommandHandler(Function<CommandRequest, CompletableFuture<CommandResponse>> handler) {
-        clusterCommunicator.addSubscriber(context.commandSubject, serializer::decode, handler, serializer::encode);
-    }
-
-    @Override
-    public void unregisterCommandHandler() {
-        clusterCommunicator.removeSubscriber(context.commandSubject);
-    }
-
-    @Override
-    public void registerMetadataHandler(Function<MetadataRequest, CompletableFuture<MetadataResponse>> handler) {
-        clusterCommunicator.addSubscriber(context.metadataSubject, serializer::decode, handler, serializer::encode);
-    }
-
-    @Override
-    public void unregisterMetadataHandler() {
-        clusterCommunicator.removeSubscriber(context.metadataSubject);
-    }
-
-    @Override
-    public void registerJoinHandler(Function<JoinRequest, CompletableFuture<JoinResponse>> handler) {
-        clusterCommunicator.addSubscriber(context.joinSubject, serializer::decode, handler, serializer::encode);
-    }
-
-    @Override
-    public void unregisterJoinHandler() {
-        clusterCommunicator.removeSubscriber(context.joinSubject);
-    }
-
-    @Override
-    public void registerLeaveHandler(Function<LeaveRequest, CompletableFuture<LeaveResponse>> handler) {
-        clusterCommunicator.addSubscriber(context.leaveSubject, serializer::decode, handler, serializer::encode);
-    }
-
-    @Override
-    public void unregisterLeaveHandler() {
-        clusterCommunicator.removeSubscriber(context.leaveSubject);
-    }
-
-    @Override
-    public void registerConfigureHandler(Function<ConfigureRequest, CompletableFuture<ConfigureResponse>> handler) {
-        clusterCommunicator.addSubscriber(context.configureSubject, serializer::decode, handler, serializer::encode);
-    }
-
-    @Override
-    public void unregisterConfigureHandler() {
-        clusterCommunicator.removeSubscriber(context.configureSubject);
-    }
-
-    @Override
-    public void registerReconfigureHandler(
-            Function<ReconfigureRequest, CompletableFuture<ReconfigureResponse>> handler) {
-        clusterCommunicator.addSubscriber(context.reconfigureSubject, serializer::decode, handler, serializer::encode);
-    }
-
-    @Override
-    public void unregisterReconfigureHandler() {
-        clusterCommunicator.removeSubscriber(context.reconfigureSubject);
-    }
-
-    @Override
-    public void registerInstallHandler(Function<InstallRequest, CompletableFuture<InstallResponse>> handler) {
-        clusterCommunicator.addSubscriber(context.installSubject, serializer::decode, handler, serializer::encode);
-    }
-
-    @Override
-    public void unregisterInstallHandler() {
-        clusterCommunicator.removeSubscriber(context.installSubject);
-    }
-
-    @Override
-    public void registerPollHandler(Function<PollRequest, CompletableFuture<PollResponse>> handler) {
-        clusterCommunicator.addSubscriber(context.pollSubject, serializer::decode, handler, serializer::encode);
-    }
-
-    @Override
-    public void unregisterPollHandler() {
-        clusterCommunicator.removeSubscriber(context.pollSubject);
-    }
-
-    @Override
-    public void registerVoteHandler(Function<VoteRequest, CompletableFuture<VoteResponse>> handler) {
-        clusterCommunicator.addSubscriber(context.voteSubject, serializer::decode, handler, serializer::encode);
-    }
-
-    @Override
-    public void unregisterVoteHandler() {
-        clusterCommunicator.removeSubscriber(context.voteSubject);
-    }
-
-    @Override
-    public void registerAppendHandler(Function<AppendRequest, CompletableFuture<AppendResponse>> handler) {
-        clusterCommunicator.addSubscriber(context.appendSubject, serializer::decode, handler, serializer::encode);
-    }
-
-    @Override
-    public void unregisterAppendHandler() {
-        clusterCommunicator.removeSubscriber(context.appendSubject);
-    }
-
-    @Override
-    public void registerTransferHandler(Function<TransferRequest, CompletableFuture<TransferResponse>> handler) {
-        clusterCommunicator.addSubscriber(context.transferSubject, serializer::decode, handler, serializer::encode);
-    }
-
-    @Override
-    public void unregisterTransferHandler() {
-        clusterCommunicator.removeSubscriber(context.transferSubject);
-    }
-
-    @Override
-    public void registerResetListener(SessionId sessionId, Consumer<ResetRequest> listener, Executor executor) {
-        clusterCommunicator.addSubscriber(context.resetSubject(sessionId.id()), serializer::decode, listener, executor);
-    }
-
-    @Override
-    public void unregisterResetListener(SessionId sessionId) {
-        clusterCommunicator.removeSubscriber(context.resetSubject(sessionId.id()));
-    }
-}
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
index 719a96e..dd71e38 100644
--- 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
@@ -18,11 +18,17 @@
 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 com.google.common.collect.Maps;
+import io.atomix.core.Atomix;
+import io.atomix.core.counter.AtomicCounter;
+import io.atomix.core.counter.AtomicCounterType;
+import io.atomix.core.map.AtomicMapType;
+import io.atomix.core.workqueue.WorkQueueType;
+import io.atomix.primitive.partition.PartitionGroup;
+import io.atomix.protocols.raft.MultiRaftProtocol;
 import org.apache.felix.scr.annotations.Activate;
 import org.apache.felix.scr.annotations.Component;
 import org.apache.felix.scr.annotations.Deactivate;
@@ -34,15 +40,12 @@
 import org.onosproject.cluster.Member;
 import org.onosproject.cluster.MembershipService;
 import org.onosproject.cluster.NodeId;
-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.impl.AtomixManager;
 import org.onosproject.store.primitives.PartitionAdminService;
-import org.onosproject.store.primitives.PartitionService;
 import org.onosproject.store.primitives.TransactionId;
 import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncAtomicValue;
 import org.onosproject.store.service.AsyncConsistentMultimap;
 import org.onosproject.store.service.AsyncConsistentTreeMap;
 import org.onosproject.store.service.AsyncDocumentTree;
@@ -50,7 +53,6 @@
 import org.onosproject.store.service.AtomicCounterMapBuilder;
 import org.onosproject.store.service.AtomicIdGeneratorBuilder;
 import org.onosproject.store.service.AtomicValueBuilder;
-import org.onosproject.store.service.ConsistentMap;
 import org.onosproject.store.service.ConsistentMapBuilder;
 import org.onosproject.store.service.ConsistentMultimapBuilder;
 import org.onosproject.store.service.ConsistentTreeMapBuilder;
@@ -68,6 +70,7 @@
 import org.onosproject.store.service.TopicBuilder;
 import org.onosproject.store.service.TransactionContextBuilder;
 import org.onosproject.store.service.WorkQueue;
+import org.onosproject.store.service.WorkQueueBuilder;
 import org.onosproject.store.service.WorkQueueStats;
 import org.slf4j.Logger;
 
@@ -82,8 +85,6 @@
 @Component(immediate = true)
 public class StorageManager implements StorageService, StorageAdminService {
 
-    private static final int BUCKETS = 128;
-
     private final Logger log = getLogger(getClass());
 
     @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
@@ -96,26 +97,21 @@
     protected PersistenceService persistenceService;
 
     @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
-    protected PartitionService partitionService;
-
-    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
     protected PartitionAdminService partitionAdminService;
 
     @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
     protected MembershipService membershipService;
 
-    private final Supplier<TransactionId> transactionIdGenerator =
-            () -> TransactionId.from(UUID.randomUUID().toString());
-    private DistributedPrimitiveCreator federatedPrimitiveCreator;
-    private TransactionManager transactionManager;
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected AtomixManager atomixManager;
+
+    private Atomix atomix;
+    private PartitionGroup group;
 
     @Activate
     public void activate() {
-        Map<PartitionId, DistributedPrimitiveCreator> partitionMap = Maps.newHashMap();
-        partitionService.getAllPartitionIds().stream()
-            .forEach(id -> partitionMap.put(id, partitionService.getDistributedPrimitiveCreator(id)));
-        federatedPrimitiveCreator = new FederatedDistributedPrimitiveCreator(partitionMap, BUCKETS);
-        transactionManager = new TransactionManager(this, partitionService, BUCKETS);
+        atomix = atomixManager.getAtomix();
+        group = atomix.getPartitionService().getPartitionGroup(MultiRaftProtocol.TYPE);
         log.info("Started");
     }
 
@@ -137,175 +133,212 @@
 
         // Use the MembershipService to provide peers for the map that are isolated within the current version.
         Supplier<List<NodeId>> peersSupplier = () -> membershipService.getMembers().stream()
-                .map(Member::nodeId)
-                .filter(nodeId -> !nodeId.equals(localNodeId))
-                .filter(id -> clusterService.getState(id).isActive())
-                .collect(Collectors.toList());
+            .map(Member::nodeId)
+            .filter(nodeId -> !nodeId.equals(localNodeId))
+            .filter(id -> clusterService.getState(id).isActive())
+            .collect(Collectors.toList());
 
         // If this is the first node in its version, bootstrap from the previous version. Otherwise, bootstrap the
         // map from members isolated within the current version.
         Supplier<List<NodeId>> bootstrapPeersSupplier = () -> {
             if (membershipService.getMembers().size() == 1) {
                 return clusterService.getNodes()
-                        .stream()
-                        .map(ControllerNode::id)
-                        .filter(id -> !localNodeId.equals(id))
-                        .filter(id -> clusterService.getState(id).isActive())
-                        .collect(Collectors.toList());
+                    .stream()
+                    .map(ControllerNode::id)
+                    .filter(id -> !localNodeId.equals(id))
+                    .filter(id -> clusterService.getState(id).isActive())
+                    .collect(Collectors.toList());
             } else {
                 return membershipService.getMembers()
-                        .stream()
-                        .map(Member::nodeId)
-                        .filter(id -> !localNodeId.equals(id))
-                        .filter(id -> clusterService.getState(id).isActive())
-                        .collect(Collectors.toList());
+                    .stream()
+                    .map(Member::nodeId)
+                    .filter(id -> !localNodeId.equals(id))
+                    .filter(id -> clusterService.getState(id).isActive())
+                    .collect(Collectors.toList());
             }
         };
 
         return new EventuallyConsistentMapBuilderImpl<>(
-                localNodeId,
-                clusterCommunicator,
-                persistenceService,
-                peersSupplier,
-                bootstrapPeersSupplier
+            localNodeId,
+            clusterCommunicator,
+            persistenceService,
+            peersSupplier,
+            bootstrapPeersSupplier
         );
     }
 
     @Override
     public <K, V> ConsistentMapBuilder<K, V> consistentMapBuilder() {
         checkPermission(STORAGE_WRITE);
-        return new DefaultConsistentMapBuilder<>(federatedPrimitiveCreator);
+        return new AtomixConsistentMapBuilder<>(atomix, group.name());
     }
 
     @Override
     public <V> DocumentTreeBuilder<V> documentTreeBuilder() {
         checkPermission(STORAGE_WRITE);
-        return new DefaultDocumentTreeBuilder<V>(federatedPrimitiveCreator);
+        return new AtomixDocumentTreeBuilder<V>(atomix, group.name());
     }
 
     @Override
     public <V> ConsistentTreeMapBuilder<V> consistentTreeMapBuilder() {
-        return new DefaultConsistentTreeMapBuilder<V>(
-                federatedPrimitiveCreator);
+        return new AtomixConsistentTreeMapBuilder<>(atomix, group.name());
     }
 
     @Override
     public <K, V> ConsistentMultimapBuilder<K, V> consistentMultimapBuilder() {
         checkPermission(STORAGE_WRITE);
-        return new DefaultConsistentMultimapBuilder<K, V>(
-                federatedPrimitiveCreator);
+        return new AtomixConsistentMultimapBuilder<>(atomix, group.name());
     }
 
     @Override
     public <K> AtomicCounterMapBuilder<K> atomicCounterMapBuilder() {
         checkPermission(STORAGE_WRITE);
-        return new DefaultAtomicCounterMapBuilder<>(federatedPrimitiveCreator);
+        return new AtomixAtomicCounterMapBuilder<>(atomix, group.name());
     }
 
     @Override
     public <E> DistributedSetBuilder<E> setBuilder() {
         checkPermission(STORAGE_WRITE);
-        return new DefaultDistributedSetBuilder<>(() -> this.<E, Boolean>consistentMapBuilder());
+        return new AtomixDistributedSetBuilder<>(atomix, group.name());
     }
 
     @Override
     public AtomicCounterBuilder atomicCounterBuilder() {
         checkPermission(STORAGE_WRITE);
-        return new DefaultAtomicCounterBuilder(federatedPrimitiveCreator);
+        return new AtomixAtomicCounterBuilder(atomix, group.name());
     }
 
     @Override
     public AtomicIdGeneratorBuilder atomicIdGeneratorBuilder() {
         checkPermission(STORAGE_WRITE);
-        return new DefaultAtomicIdGeneratorBuilder(federatedPrimitiveCreator);
+        return new AtomixAtomicIdGeneratorBuilder(atomix, group.name());
     }
 
     @Override
     public <V> AtomicValueBuilder<V> atomicValueBuilder() {
         checkPermission(STORAGE_WRITE);
-        Supplier<ConsistentMapBuilder<String, byte[]>> mapBuilderSupplier =
-                () -> this.<String, byte[]>consistentMapBuilder()
-                          .withName("onos-atomic-values")
-                          .withSerializer(Serializer.using(KryoNamespaces.BASIC));
-        return new DefaultAtomicValueBuilder<>(mapBuilderSupplier);
+        return new AtomixAtomicValueBuilder<>(atomix, group.name());
     }
 
     @Override
     public TransactionContextBuilder transactionContextBuilder() {
         checkPermission(STORAGE_WRITE);
-        return new DefaultTransactionContextBuilder(transactionIdGenerator.get(), transactionManager);
+        return new AtomixTransactionContextBuilder(atomix, group.name());
     }
 
     @Override
     public DistributedLockBuilder lockBuilder() {
         checkPermission(STORAGE_WRITE);
-        return new DefaultDistributedLockBuilder(federatedPrimitiveCreator);
+        return new AtomixDistributedLockBuilder(atomix, group.name());
     }
 
     @Override
     public LeaderElectorBuilder leaderElectorBuilder() {
         checkPermission(STORAGE_WRITE);
-        return new DefaultLeaderElectorBuilder(federatedPrimitiveCreator);
+        return new AtomixLeaderElectorBuilder(atomix, group.name(), clusterService.getLocalNode().id());
     }
 
     @Override
     public <T> TopicBuilder<T> topicBuilder() {
         checkPermission(STORAGE_WRITE);
-        return new DefaultDistributedTopicBuilder<>(atomicValueBuilder());
+        return new AtomixDistributedTopicBuilder<>(atomix, group.name());
+    }
+
+    @Override
+    public <E> WorkQueueBuilder<E> workQueueBuilder() {
+        checkPermission(STORAGE_WRITE);
+        return new AtomixWorkQueueBuilder<>(atomix, group.name());
     }
 
     @Override
     public <E> WorkQueue<E> getWorkQueue(String name, Serializer serializer) {
         checkPermission(STORAGE_WRITE);
-        return federatedPrimitiveCreator.newWorkQueue(name, serializer);
+        return this.<E>workQueueBuilder()
+            .withName(name)
+            .withSerializer(serializer)
+            .build();
     }
 
     @Override
     public <V> AsyncDocumentTree<V> getDocumentTree(String name, Serializer serializer) {
         checkPermission(STORAGE_WRITE);
-        return federatedPrimitiveCreator.newAsyncDocumentTree(name, serializer);
+        return this.<V>documentTreeBuilder()
+            .withName(name)
+            .withSerializer(serializer)
+            .build();
     }
 
     @Override
-    public <K, V> AsyncConsistentMultimap<K, V> getAsyncSetMultimap(
-            String name, Serializer serializer) {
+    public <K, V> AsyncConsistentMultimap<K, V> getAsyncSetMultimap(String name, Serializer serializer) {
         checkPermission(STORAGE_WRITE);
-        return federatedPrimitiveCreator.newAsyncConsistentSetMultimap(name,
-                                                                serializer);
+        return new AtomixConsistentMultimapBuilder<K, V>(atomix, group.name())
+            .withName(name)
+            .withSerializer(serializer)
+            .buildMultimap();
     }
 
     @Override
-    public <V> AsyncConsistentTreeMap<V> getAsyncTreeMap(
-            String name, Serializer serializer) {
+    public <V> AsyncConsistentTreeMap<V> getAsyncTreeMap(String name, Serializer serializer) {
         checkPermission(STORAGE_WRITE);
-        return federatedPrimitiveCreator.newAsyncConsistentTreeMap(name,
-                                                                   serializer);
+        return this.<V>consistentTreeMapBuilder()
+            .withName(name)
+            .withSerializer(serializer)
+            .buildTreeMap();
+    }
+
+    @Override
+    public <T> Topic<T> getTopic(String name, Serializer serializer) {
+        checkPermission(STORAGE_WRITE);
+        return this.<T>topicBuilder()
+            .withName(name)
+            .withSerializer(serializer)
+            .build();
     }
 
     @Override
     public List<MapInfo> getMapInfo() {
-        return listMapInfo(federatedPrimitiveCreator);
+        Serializer serializer = Serializer.using(KryoNamespaces.BASIC);
+        return atomix.getPrimitives(AtomicMapType.instance())
+            .stream()
+            .map(info -> {
+                io.atomix.core.map.AtomicMap<String, byte[]> map =
+                    atomix.<String, byte[]>atomicMapBuilder(info.name())
+                        .withSerializer(new AtomixSerializerAdapter(serializer))
+                        .build();
+                int size = map.size();
+                map.close();
+                return new MapInfo(info.name(), size);
+            }).collect(Collectors.toList());
     }
 
     @Override
     public Map<String, Long> getCounters() {
-        Map<String, Long> counters = Maps.newConcurrentMap();
-        federatedPrimitiveCreator.getAsyncAtomicCounterNames()
-               .forEach(name -> counters.put(name,
-                       federatedPrimitiveCreator.newAsyncCounter(name).asAtomicCounter().get()));
-        return counters;
+        return atomix.getPrimitives(AtomicCounterType.instance())
+            .stream()
+            .map(info -> {
+                AtomicCounter counter = atomix.atomicCounterBuilder(info.name()).build();
+                long value = counter.get();
+                counter.close();
+                return Maps.immutableEntry(info.name(), value);
+            }).collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue()));
     }
 
     @Override
     public Map<String, WorkQueueStats> getQueueStats() {
-        Map<String, WorkQueueStats> workQueueStats = Maps.newConcurrentMap();
-        federatedPrimitiveCreator.getWorkQueueNames()
-               .forEach(name -> workQueueStats.put(name,
-                       federatedPrimitiveCreator.newWorkQueue(name,
-                                                              Serializer.using(KryoNamespaces.BASIC))
-                                                .stats()
-                                                .join()));
-        return workQueueStats;
+        Serializer serializer = Serializer.using(KryoNamespaces.BASIC);
+        return atomix.getPrimitives(WorkQueueType.instance())
+            .stream()
+            .map(info -> {
+                io.atomix.core.workqueue.WorkQueue queue = atomix.workQueueBuilder(info.name())
+                    .withSerializer(new AtomixSerializerAdapter(serializer))
+                    .build();
+                io.atomix.core.workqueue.WorkQueueStats stats = queue.stats();
+                return Maps.immutableEntry(info.name(), WorkQueueStats.builder()
+                    .withTotalCompleted(stats.totalCompleted())
+                    .withTotalInProgress(stats.totalInProgress())
+                    .withTotalPending(stats.totalPending())
+                    .build());
+            }).collect(Collectors.toMap(e -> e.getKey(), e -> e.getValue()));
     }
 
     @Override
@@ -315,27 +348,9 @@
 
     @Override
     public Collection<TransactionId> getPendingTransactions() {
-        return transactionManager.getPendingTransactions();
-    }
-
-    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());
-    }
-
-    @Override
-    public <T> Topic<T> getTopic(String name, Serializer serializer) {
-        AsyncAtomicValue<T> atomicValue = this.<T>atomicValueBuilder()
-                                              .withName("topic-" + name)
-                                              .withSerializer(serializer)
-                                              .build();
-        return new DefaultDistributedTopic<>(atomicValue);
+        return atomix.getTransactionService().getActiveTransactions()
+            .stream()
+            .map(transactionId -> TransactionId.from(transactionId.id()))
+            .collect(Collectors.toList());
     }
 }
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StorageNamespaces.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StorageNamespaces.java
deleted file mode 100644
index 4eaac99..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StorageNamespaces.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.time.Instant;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashSet;
-import java.util.LinkedList;
-
-import io.atomix.protocols.raft.RaftError;
-import io.atomix.protocols.raft.ReadConsistency;
-import io.atomix.protocols.raft.cluster.MemberId;
-import io.atomix.protocols.raft.cluster.RaftMember;
-import io.atomix.protocols.raft.cluster.impl.DefaultRaftMember;
-import io.atomix.protocols.raft.event.RaftEvent;
-import io.atomix.protocols.raft.event.impl.DefaultEventType;
-import io.atomix.protocols.raft.operation.OperationType;
-import io.atomix.protocols.raft.operation.RaftOperation;
-import io.atomix.protocols.raft.operation.impl.DefaultOperationId;
-import io.atomix.protocols.raft.protocol.AppendRequest;
-import io.atomix.protocols.raft.protocol.AppendResponse;
-import io.atomix.protocols.raft.protocol.CloseSessionRequest;
-import io.atomix.protocols.raft.protocol.CloseSessionResponse;
-import io.atomix.protocols.raft.protocol.CommandRequest;
-import io.atomix.protocols.raft.protocol.CommandResponse;
-import io.atomix.protocols.raft.protocol.ConfigureRequest;
-import io.atomix.protocols.raft.protocol.ConfigureResponse;
-import io.atomix.protocols.raft.protocol.HeartbeatRequest;
-import io.atomix.protocols.raft.protocol.HeartbeatResponse;
-import io.atomix.protocols.raft.protocol.InstallRequest;
-import io.atomix.protocols.raft.protocol.InstallResponse;
-import io.atomix.protocols.raft.protocol.JoinRequest;
-import io.atomix.protocols.raft.protocol.JoinResponse;
-import io.atomix.protocols.raft.protocol.KeepAliveRequest;
-import io.atomix.protocols.raft.protocol.KeepAliveResponse;
-import io.atomix.protocols.raft.protocol.LeaveRequest;
-import io.atomix.protocols.raft.protocol.LeaveResponse;
-import io.atomix.protocols.raft.protocol.MetadataRequest;
-import io.atomix.protocols.raft.protocol.MetadataResponse;
-import io.atomix.protocols.raft.protocol.OpenSessionRequest;
-import io.atomix.protocols.raft.protocol.OpenSessionResponse;
-import io.atomix.protocols.raft.protocol.PollRequest;
-import io.atomix.protocols.raft.protocol.PollResponse;
-import io.atomix.protocols.raft.protocol.PublishRequest;
-import io.atomix.protocols.raft.protocol.QueryRequest;
-import io.atomix.protocols.raft.protocol.QueryResponse;
-import io.atomix.protocols.raft.protocol.RaftResponse;
-import io.atomix.protocols.raft.protocol.ReconfigureRequest;
-import io.atomix.protocols.raft.protocol.ReconfigureResponse;
-import io.atomix.protocols.raft.protocol.ResetRequest;
-import io.atomix.protocols.raft.protocol.VoteRequest;
-import io.atomix.protocols.raft.protocol.VoteResponse;
-import io.atomix.protocols.raft.service.PropagationStrategy;
-import io.atomix.protocols.raft.session.RaftSessionMetadata;
-import io.atomix.protocols.raft.session.SessionId;
-import io.atomix.protocols.raft.storage.log.entry.CloseSessionEntry;
-import io.atomix.protocols.raft.storage.log.entry.CommandEntry;
-import io.atomix.protocols.raft.storage.log.entry.ConfigurationEntry;
-import io.atomix.protocols.raft.storage.log.entry.InitializeEntry;
-import io.atomix.protocols.raft.storage.log.entry.KeepAliveEntry;
-import io.atomix.protocols.raft.storage.log.entry.MetadataEntry;
-import io.atomix.protocols.raft.storage.log.entry.OpenSessionEntry;
-import io.atomix.protocols.raft.storage.log.entry.QueryEntry;
-import io.atomix.protocols.raft.storage.system.Configuration;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-
-/**
- * Storage serializer namespaces.
- */
-public final class StorageNamespaces {
-
-    /**
-     * Raft protocol namespace.
-     */
-    public static final KryoNamespace RAFT_PROTOCOL = KryoNamespace.newBuilder()
-            .register(KryoNamespaces.BASIC)
-            .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
-            .register(OpenSessionRequest.class)
-            .register(OpenSessionResponse.class)
-            .register(CloseSessionRequest.class)
-            .register(CloseSessionResponse.class)
-            .register(KeepAliveRequest.class)
-            .register(KeepAliveResponse.class)
-            .register(HeartbeatRequest.class)
-            .register(HeartbeatResponse.class)
-            .register(QueryRequest.class)
-            .register(QueryResponse.class)
-            .register(CommandRequest.class)
-            .register(CommandResponse.class)
-            .register(MetadataRequest.class)
-            .register(MetadataResponse.class)
-            .register(JoinRequest.class)
-            .register(JoinResponse.class)
-            .register(LeaveRequest.class)
-            .register(LeaveResponse.class)
-            .register(ConfigureRequest.class)
-            .register(ConfigureResponse.class)
-            .register(ReconfigureRequest.class)
-            .register(ReconfigureResponse.class)
-            .register(InstallRequest.class)
-            .register(InstallResponse.class)
-            .register(PollRequest.class)
-            .register(PollResponse.class)
-            .register(VoteRequest.class)
-            .register(VoteResponse.class)
-            .register(AppendRequest.class)
-            .register(AppendResponse.class)
-            .register(PublishRequest.class)
-            .register(ResetRequest.class)
-            .register(RaftResponse.Status.class)
-            .register(RaftError.class)
-            .register(RaftError.Type.class)
-            .register(RaftOperation.class)
-            .register(DefaultOperationId.class)
-            .register(OperationType.class)
-            .register(RaftEvent.class)
-            .register(DefaultEventType.class)
-            .register(RaftSessionMetadata.class)
-            .register(CloseSessionEntry.class)
-            .register(CommandEntry.class)
-            .register(ConfigurationEntry.class)
-            .register(InitializeEntry.class)
-            .register(KeepAliveEntry.class)
-            .register(MetadataEntry.class)
-            .register(OpenSessionEntry.class)
-            .register(QueryEntry.class)
-            .register(ReadConsistency.class)
-            .register(ArrayList.class)
-            .register(LinkedList.class)
-            .register(Collections.emptyList().getClass())
-            .register(HashSet.class)
-            .register(DefaultRaftMember.class)
-            .register(MemberId.class)
-            .register(SessionId.class)
-            .register(RaftMember.Type.class)
-            .register(Instant.class)
-            .register(Configuration.class)
-            .register(PropagationStrategy.class)
-            .build("RaftProtocol");
-
-    /**
-     * Raft storage namespace.
-     */
-    public static final KryoNamespace RAFT_STORAGE = KryoNamespace.newBuilder()
-            .register(KryoNamespaces.BASIC)
-            .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 100)
-            .register(CloseSessionEntry.class)
-            .register(CommandEntry.class)
-            .register(ConfigurationEntry.class)
-            .register(InitializeEntry.class)
-            .register(KeepAliveEntry.class)
-            .register(MetadataEntry.class)
-            .register(OpenSessionEntry.class)
-            .register(QueryEntry.class)
-            .register(RaftOperation.class)
-            .register(DefaultOperationId.class)
-            .register(OperationType.class)
-            .register(ReadConsistency.class)
-            .register(ArrayList.class)
-            .register(HashSet.class)
-            .register(DefaultRaftMember.class)
-            .register(MemberId.class)
-            .register(RaftMember.Type.class)
-            .register(Instant.class)
-            .register(Configuration.class)
-            .register(PropagationStrategy.class)
-            .build("RaftStorage");
-
-    private StorageNamespaces() {
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartition.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartition.java
deleted file mode 100644
index 2a29114..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartition.java
+++ /dev/null
@@ -1,284 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.io.File;
-import java.util.Collection;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.Collections2;
-import com.google.common.collect.ImmutableMap;
-import io.atomix.protocols.raft.cluster.MemberId;
-import io.atomix.protocols.raft.service.RaftService;
-import org.onosproject.cluster.ClusterService;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.cluster.Partition;
-import org.onosproject.cluster.PartitionId;
-import org.onosproject.core.Version;
-import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapService;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapService;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapService;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapService;
-import org.onosproject.store.primitives.resources.impl.AtomixCounterService;
-import org.onosproject.store.primitives.resources.impl.AtomixDistributedLockService;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeService;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorService;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueService;
-import org.onosproject.store.service.DistributedPrimitive;
-import org.onosproject.store.service.Ordering;
-import org.onosproject.store.service.PartitionInfo;
-import org.onosproject.store.service.Serializer;
-
-/**
- * Storage partition.
- */
-public class StoragePartition implements Managed<StoragePartition> {
-
-    static final String PARTITIONS_DIR =
-            System.getProperty("karaf.data") + "/db/partitions/";
-
-    protected final AtomicBoolean isOpened = new AtomicBoolean(false);
-    protected final ClusterCommunicationService clusterCommunicator;
-    protected Partition partition;
-    protected NodeId localNodeId;
-    protected StoragePartitionServer server;
-    protected StoragePartitionClient client;
-
-    public static final Map<String, Supplier<RaftService>> RAFT_SERVICES =
-            ImmutableMap.<String, Supplier<RaftService>>builder()
-                    .put(DistributedPrimitive.Type.CONSISTENT_MAP.name(), AtomixConsistentMapService::new)
-                    .put(DistributedPrimitive.Type.CONSISTENT_TREEMAP.name(), AtomixConsistentTreeMapService::new)
-                    .put(DistributedPrimitive.Type.CONSISTENT_MULTIMAP.name(), AtomixConsistentSetMultimapService::new)
-                    .put(DistributedPrimitive.Type.COUNTER_MAP.name(), AtomixAtomicCounterMapService::new)
-                    .put(DistributedPrimitive.Type.COUNTER.name(), AtomixCounterService::new)
-                    .put(DistributedPrimitive.Type.LEADER_ELECTOR.name(), AtomixLeaderElectorService::new)
-                    .put(DistributedPrimitive.Type.WORK_QUEUE.name(), AtomixWorkQueueService::new)
-                    .put(DistributedPrimitive.Type.DOCUMENT_TREE.name(),
-                            () -> new AtomixDocumentTreeService(Ordering.NATURAL))
-                    .put(String.format("%s-%s", DistributedPrimitive.Type.DOCUMENT_TREE.name(), Ordering.NATURAL),
-                            () -> new AtomixDocumentTreeService(Ordering.NATURAL))
-                    .put(String.format("%s-%s", DistributedPrimitive.Type.DOCUMENT_TREE.name(), Ordering.INSERTION),
-                            () -> new AtomixDocumentTreeService(Ordering.INSERTION))
-                    .put(DistributedPrimitive.Type.LOCK.name(), AtomixDistributedLockService::new)
-                    .build();
-
-    public StoragePartition(
-            Partition partition,
-            ClusterCommunicationService clusterCommunicator,
-            ClusterService clusterService) {
-        this.partition = partition;
-        this.clusterCommunicator = clusterCommunicator;
-        this.localNodeId = clusterService.getLocalNode().id();
-    }
-
-    /**
-     * Returns the partition client instance.
-     * @return client
-     */
-    public StoragePartitionClient client() {
-        return client;
-    }
-
-    @Override
-    public CompletableFuture<Void> open() {
-        if (partition.getMembers().contains(localNodeId)) {
-            return openServer()
-                    .thenCompose(v -> openClient())
-                    .thenAccept(v -> isOpened.set(true))
-                    .thenApply(v -> null);
-        }
-        return openClient()
-                .thenAccept(v -> isOpened.set(true))
-                .thenApply(v -> null);
-    }
-
-    @Override
-    public CompletableFuture<Void> close() {
-        // We do not explicitly close the server and instead let the cluster
-        // deal with this as an unclean exit.
-        return closeClient();
-    }
-
-    /**
-     * Deletes the partition.
-     *
-     * @return future to be completed once the partition has been deleted
-     */
-    public CompletableFuture<Void> delete() {
-        return closeServer().thenCompose(v -> closeClient()).thenRun(() -> deleteServer());
-    }
-
-    /**
-     * Returns the partition data folder.
-     *
-     * @return the partition data folder
-     */
-    public File getDataFolder() {
-        return new File(PARTITIONS_DIR + partition.getId());
-    }
-
-    /**
-     * Returns the partition name.
-     *
-     * @return the partition name
-     */
-    public String getName() {
-        return partition.getId().toString();
-    }
-
-    /**
-     * Returns the identifier of the {@link Partition partition} associated with this instance.
-     *
-     * @return partition identifier
-     */
-    public PartitionId getId() {
-        return partition.getId();
-    }
-
-    /**
-     * Returns the partition version.
-     *
-     * @return the partition version
-     */
-    public Version getVersion() {
-        return partition.getVersion();
-    }
-
-    /**
-     * Returns the identifiers of partition members.
-     * @return partition member instance ids
-     */
-    public Collection<NodeId> getMembers() {
-        return partition.getMembers();
-    }
-
-    /**
-     * Returns the {@link MemberId identifiers} of partition members.
-     * @return partition member identifiers
-     */
-    public Collection<MemberId> getMemberIds() {
-        return Collections2.transform(getMembers(), n -> MemberId.from(n.id()));
-    }
-
-    /**
-     * Attempts to rejoin the partition.
-     * @return future that is completed after the operation is complete
-     */
-    protected CompletableFuture<Void> openServer() {
-        StoragePartitionServer server = new StoragePartitionServer(
-            this,
-            MemberId.from(localNodeId.id()),
-            clusterCommunicator);
-        return server.open().thenRun(() -> this.server = server);
-    }
-
-    /**
-     * Attempts to join the partition as a new member.
-     * @return future that is completed after the operation is complete
-     */
-    private CompletableFuture<Void> joinCluster() {
-        Set<NodeId> otherMembers = partition.getMembers()
-                 .stream()
-                 .filter(nodeId -> !nodeId.equals(localNodeId))
-                 .collect(Collectors.toSet());
-        StoragePartitionServer server = new StoragePartitionServer(this,
-                MemberId.from(localNodeId.id()),
-                clusterCommunicator);
-        return server.join(Collections2.transform(otherMembers, n -> MemberId.from(n.id())))
-                .thenRun(() -> this.server = server);
-    }
-
-    private CompletableFuture<StoragePartitionClient> openClient() {
-        client = new StoragePartitionClient(this,
-                MemberId.from(localNodeId.id()),
-                new RaftClientCommunicator(
-                        String.format("partition-%s-%s", partition.getId(), partition.getVersion()),
-                        Serializer.using(StorageNamespaces.RAFT_PROTOCOL),
-                        clusterCommunicator));
-        return client.open().thenApply(v -> client);
-    }
-
-    /**
-     * Closes the partition server if it was previously opened.
-     * @return future that is completed when the operation completes
-     */
-    public CompletableFuture<Void> leaveCluster() {
-        return server != null
-                ? server.closeAndExit().thenRun(() -> server.delete())
-                : CompletableFuture.completedFuture(null);
-    }
-
-    @Override
-    public boolean isOpen() {
-        return isOpened.get();
-    }
-
-    private CompletableFuture<Void> closeServer() {
-        if (server != null) {
-            return server.close();
-        }
-        return CompletableFuture.completedFuture(null);
-    }
-
-    private void deleteServer() {
-        if (server != null) {
-            server.delete();
-        }
-    }
-
-    private CompletableFuture<Void> closeClient() {
-        if (client != null) {
-            return client.close();
-        }
-        return CompletableFuture.completedFuture(null);
-    }
-
-    /**
-     * Returns the partition information if this partition is locally managed i.e.
-     * this node is a active member of the partition.
-     * @return partition info
-     */
-    public Optional<PartitionInfo> info() {
-        return server != null && server.isOpen() ? Optional.of(server.info()) : Optional.empty();
-    }
-
-    /**
-     * Process updates to partitions and handles joining or leaving a partition.
-     * @param newValue new Partition
-     */
-    void onUpdate(Partition newValue) {
-        boolean wasPresent = partition.getMembers().contains(localNodeId);
-        boolean isPresent = newValue.getMembers().contains(localNodeId);
-        this.partition = newValue;
-        if ((wasPresent && isPresent) || (!wasPresent && !isPresent)) {
-            // no action needed
-            return;
-        }
-        // Only need to do action if our membership changed
-        if (wasPresent) {
-            leaveCluster();
-        } else {
-            joinCluster();
-        }
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartitionClient.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartitionClient.java
deleted file mode 100644
index 5fa7383..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartitionClient.java
+++ /dev/null
@@ -1,436 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.time.Duration;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.stream.Collectors;
-
-import com.google.common.base.Suppliers;
-import io.atomix.protocols.raft.RaftClient;
-import io.atomix.protocols.raft.ReadConsistency;
-import io.atomix.protocols.raft.cluster.MemberId;
-import io.atomix.protocols.raft.protocol.RaftClientProtocol;
-import io.atomix.protocols.raft.proxy.CommunicationStrategy;
-import io.atomix.protocols.raft.service.PropagationStrategy;
-import io.atomix.protocols.raft.session.RaftSessionMetadata;
-import org.onlab.util.HexString;
-import org.onosproject.store.primitives.DistributedPrimitiveCreator;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMap;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMap;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimap;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMap;
-import org.onosproject.store.primitives.resources.impl.AtomixCounter;
-import org.onosproject.store.primitives.resources.impl.AtomixDistributedLock;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTree;
-import org.onosproject.store.primitives.resources.impl.AtomixIdGenerator;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElector;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueue;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncAtomicCounter;
-import org.onosproject.store.service.AsyncAtomicCounterMap;
-import org.onosproject.store.service.AsyncAtomicIdGenerator;
-import org.onosproject.store.service.AsyncAtomicValue;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.AsyncConsistentMultimap;
-import org.onosproject.store.service.AsyncConsistentTreeMap;
-import org.onosproject.store.service.AsyncDistributedLock;
-import org.onosproject.store.service.AsyncDistributedSet;
-import org.onosproject.store.service.AsyncDocumentTree;
-import org.onosproject.store.service.AsyncLeaderElector;
-import org.onosproject.store.service.AtomicCounterMapOptions;
-import org.onosproject.store.service.AtomicCounterOptions;
-import org.onosproject.store.service.AtomicIdGeneratorOptions;
-import org.onosproject.store.service.AtomicValueOptions;
-import org.onosproject.store.service.ConsistentMapOptions;
-import org.onosproject.store.service.ConsistentMultimapOptions;
-import org.onosproject.store.service.ConsistentTreeMapOptions;
-import org.onosproject.store.service.DistributedLockOptions;
-import org.onosproject.store.service.DistributedPrimitive;
-import org.onosproject.store.service.DistributedSetOptions;
-import org.onosproject.store.service.DocumentTreeOptions;
-import org.onosproject.store.service.LeaderElectorOptions;
-import org.onosproject.store.service.PartitionClientInfo;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.WorkQueue;
-import org.onosproject.store.service.WorkQueueOptions;
-import org.slf4j.Logger;
-
-import static org.slf4j.LoggerFactory.getLogger;
-
-/**
- * StoragePartition client.
- */
-public class StoragePartitionClient implements DistributedPrimitiveCreator, Managed<StoragePartitionClient> {
-
-    private static final int MAX_RETRIES = 8;
-    private static final String ATOMIC_VALUES_CONSISTENT_MAP_NAME = "onos-atomic-values";
-
-    private static final String MIN_TIMEOUT_PROPERTY = "onos.cluster.raft.client.minTimeoutMillis";
-    private static final String MAX_TIMEOUT_PROPERTY = "onos.cluster.raft.client.maxTimeoutMillis";
-
-    private static final Duration MIN_TIMEOUT;
-    private static final Duration MAX_TIMEOUT;
-
-    private static final long DEFAULT_MIN_TIMEOUT_MILLIS = 5000;
-    private static final long DEFAULT_MAX_TIMEOUT_MILLIS = 30000;
-
-    static {
-        Duration minTimeout;
-        try {
-            minTimeout = Duration.ofMillis(Long.parseLong(
-                System.getProperty(MIN_TIMEOUT_PROPERTY,
-                    String.valueOf(DEFAULT_MIN_TIMEOUT_MILLIS))));
-        } catch (NumberFormatException e) {
-            minTimeout = Duration.ofMillis(DEFAULT_MIN_TIMEOUT_MILLIS);
-        }
-        MIN_TIMEOUT = minTimeout;
-
-        Duration maxTimeout;
-        try {
-            maxTimeout = Duration.ofMillis(Long.parseLong(
-                System.getProperty(MAX_TIMEOUT_PROPERTY,
-                    String.valueOf(DEFAULT_MAX_TIMEOUT_MILLIS))));
-        } catch (NumberFormatException e) {
-            maxTimeout = Duration.ofMillis(DEFAULT_MAX_TIMEOUT_MILLIS);
-        }
-        MAX_TIMEOUT = maxTimeout;
-    }
-
-    private final Logger log = getLogger(getClass());
-
-    private final StoragePartition partition;
-    private final MemberId localMemberId;
-    private final RaftClientProtocol protocol;
-    private RaftClient client;
-    private final com.google.common.base.Supplier<AsyncConsistentMap<String, byte[]>> onosAtomicValuesMap =
-            Suppliers.memoize(() -> newAsyncConsistentMap(ATOMIC_VALUES_CONSISTENT_MAP_NAME,
-                                                          Serializer.using(KryoNamespaces.BASIC)));
-
-    public StoragePartitionClient(StoragePartition partition, MemberId localMemberId, RaftClientProtocol protocol) {
-        this.partition = partition;
-        this.localMemberId = localMemberId;
-        this.protocol = protocol;
-    }
-
-    @Override
-    public CompletableFuture<Void> open() {
-        synchronized (StoragePartitionClient.this) {
-            client = newRaftClient(protocol);
-        }
-        return client.connect(partition.getMemberIds()).whenComplete((r, e) -> {
-            if (e == null) {
-                log.info("Successfully started client for partition {}", partition.getId());
-            } else {
-                log.info("Failed to start client for partition {}", partition.getId(), e);
-            }
-        }).thenApply(v -> null);
-    }
-
-    @Override
-    public CompletableFuture<Void> close() {
-        return client != null ? client.close() : CompletableFuture.completedFuture(null);
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public <K, V> AsyncConsistentMap<K, V> newAsyncConsistentMap(ConsistentMapOptions options) {
-        AtomixConsistentMap rawMap =
-                new AtomixConsistentMap(client.newProxyBuilder()
-                        .withName(options.name())
-                        .withServiceType(DistributedPrimitive.Type.CONSISTENT_MAP.name())
-                        .withReadConsistency(ReadConsistency.SEQUENTIAL)
-                        .withCommunicationStrategy(CommunicationStrategy.ANY)
-                        .withMinTimeout(MIN_TIMEOUT)
-                        .withMaxTimeout(MAX_TIMEOUT)
-                        .withMaxRetries(MAX_RETRIES)
-                        .withRevision(options.version() != null && options.revisionType() != null
-                            ? options.version().toInt() : 1)
-                        .withPropagationStrategy(options.revisionType() != null
-                            ? PropagationStrategy.valueOf(options.revisionType().name())
-                            : PropagationStrategy.NONE)
-                        .build()
-                        .open()
-                        .join());
-
-        if (options.serializer() != null) {
-            return DistributedPrimitives.newTranscodingMap(rawMap,
-                    key -> HexString.toHexString(options.serializer().encode(key)),
-                    string -> options.serializer().decode(HexString.fromHexString(string)),
-                    value -> value == null ? null : options.serializer().encode(value),
-                    bytes -> options.serializer().decode(bytes));
-        }
-        return (AsyncConsistentMap<K, V>) rawMap;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public <V> AsyncConsistentTreeMap<V> newAsyncConsistentTreeMap(ConsistentTreeMapOptions options) {
-        AtomixConsistentTreeMap rawMap =
-                new AtomixConsistentTreeMap(client.newProxyBuilder()
-                        .withName(options.name())
-                        .withServiceType(DistributedPrimitive.Type.CONSISTENT_TREEMAP.name())
-                        .withReadConsistency(ReadConsistency.SEQUENTIAL)
-                        .withCommunicationStrategy(CommunicationStrategy.ANY)
-                        .withMinTimeout(MIN_TIMEOUT)
-                        .withMaxTimeout(MAX_TIMEOUT)
-                        .withMaxRetries(MAX_RETRIES)
-                        .withRevision(options.version() != null && options.revisionType() != null
-                            ? options.version().toInt() : 1)
-                        .withPropagationStrategy(options.revisionType() != null
-                            ? PropagationStrategy.valueOf(options.revisionType().name())
-                            : PropagationStrategy.NONE)
-                        .build()
-                        .open()
-                        .join());
-
-        if (options.serializer() != null) {
-            return DistributedPrimitives.newTranscodingTreeMap(
-                            rawMap,
-                            value -> value == null ? null : options.serializer().encode(value),
-                            bytes -> options.serializer().decode(bytes));
-        }
-        return (AsyncConsistentTreeMap<V>) rawMap;
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public <K, V> AsyncConsistentMultimap<K, V> newAsyncConsistentSetMultimap(ConsistentMultimapOptions options) {
-        AtomixConsistentSetMultimap rawMap =
-                new AtomixConsistentSetMultimap(client.newProxyBuilder()
-                        .withName(options.name())
-                        .withServiceType(DistributedPrimitive.Type.CONSISTENT_MULTIMAP.name())
-                        .withReadConsistency(ReadConsistency.SEQUENTIAL)
-                        .withCommunicationStrategy(CommunicationStrategy.ANY)
-                        .withMinTimeout(MIN_TIMEOUT)
-                        .withMaxTimeout(MAX_TIMEOUT)
-                        .withMaxRetries(MAX_RETRIES)
-                        .withRevision(options.version() != null && options.revisionType() != null
-                            ? options.version().toInt() : 1)
-                        .withPropagationStrategy(options.revisionType() != null
-                            ? PropagationStrategy.valueOf(options.revisionType().name())
-                            : PropagationStrategy.NONE)
-                        .build()
-                        .open()
-                        .join());
-
-        if (options.serializer() != null) {
-            return DistributedPrimitives.newTranscodingMultimap(
-                            rawMap,
-                            key -> HexString.toHexString(options.serializer().encode(key)),
-                            string -> options.serializer().decode(HexString.fromHexString(string)),
-                            value -> options.serializer().encode(value),
-                            bytes -> options.serializer().decode(bytes));
-        }
-        return (AsyncConsistentMultimap<K, V>) rawMap;
-    }
-
-    @Override
-    public <E> AsyncDistributedSet<E> newAsyncDistributedSet(DistributedSetOptions options) {
-        return DistributedPrimitives.newSetFromMap(newAsyncConsistentMap(options.name(), options.serializer()));
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public <K> AsyncAtomicCounterMap<K> newAsyncAtomicCounterMap(AtomicCounterMapOptions options) {
-        AtomixAtomicCounterMap rawMap = new AtomixAtomicCounterMap(client.newProxyBuilder()
-                .withName(options.name())
-                .withServiceType(DistributedPrimitive.Type.COUNTER_MAP.name())
-                .withReadConsistency(ReadConsistency.LINEARIZABLE_LEASE)
-                .withCommunicationStrategy(CommunicationStrategy.LEADER)
-                .withMinTimeout(MIN_TIMEOUT)
-                .withMaxTimeout(MAX_TIMEOUT)
-                .withMaxRetries(MAX_RETRIES)
-                .withRevision(options.version() != null && options.revisionType() != null
-                    ? options.version().toInt() : 1)
-                .withPropagationStrategy(options.revisionType() != null
-                    ? PropagationStrategy.valueOf(options.revisionType().name())
-                    : PropagationStrategy.NONE)
-                .build()
-                .open()
-                .join());
-
-        if (options.serializer() != null) {
-            return DistributedPrimitives.newTranscodingAtomicCounterMap(
-                            rawMap,
-                            key -> HexString.toHexString(options.serializer().encode(key)),
-                            string -> options.serializer().decode(HexString.fromHexString(string)));
-        }
-        return (AsyncAtomicCounterMap<K>) rawMap;
-    }
-
-    @Override
-    public AsyncAtomicCounter newAsyncCounter(AtomicCounterOptions options) {
-        return new AtomixCounter(client.newProxyBuilder()
-                .withName(options.name())
-                .withServiceType(DistributedPrimitive.Type.COUNTER.name())
-                .withReadConsistency(ReadConsistency.LINEARIZABLE_LEASE)
-                .withCommunicationStrategy(CommunicationStrategy.LEADER)
-                .withMinTimeout(MIN_TIMEOUT)
-                .withMaxTimeout(MAX_TIMEOUT)
-                .withMaxRetries(MAX_RETRIES)
-                .withRevision(options.version() != null && options.revisionType() != null
-                    ? options.version().toInt() : 1)
-                .withPropagationStrategy(options.revisionType() != null
-                    ? PropagationStrategy.valueOf(options.revisionType().name())
-                    : PropagationStrategy.NONE)
-                .build()
-                .open()
-                .join());
-    }
-
-    @Override
-    public AsyncAtomicIdGenerator newAsyncIdGenerator(AtomicIdGeneratorOptions options) {
-        return new AtomixIdGenerator(newAsyncCounter(options.name()));
-    }
-
-    @Override
-    public <V> AsyncAtomicValue<V> newAsyncAtomicValue(AtomicValueOptions options) {
-        return new DefaultAsyncAtomicValue<>(options.name(), options.serializer(), onosAtomicValuesMap.get());
-    }
-
-    @Override
-    public <E> WorkQueue<E> newWorkQueue(WorkQueueOptions options) {
-        AtomixWorkQueue atomixWorkQueue = new AtomixWorkQueue(client.newProxyBuilder()
-                .withName(options.name())
-                .withServiceType(DistributedPrimitive.Type.WORK_QUEUE.name())
-                .withReadConsistency(ReadConsistency.LINEARIZABLE_LEASE)
-                .withCommunicationStrategy(CommunicationStrategy.LEADER)
-                .withMinTimeout(MIN_TIMEOUT)
-                .withMaxTimeout(MAX_TIMEOUT)
-                .withMaxRetries(MAX_RETRIES)
-                .withRevision(options.version() != null && options.revisionType() != null
-                    ? options.version().toInt() : 1)
-                .withPropagationStrategy(options.revisionType() != null
-                    ? PropagationStrategy.valueOf(options.revisionType().name())
-                    : PropagationStrategy.NONE)
-                .build()
-                .open()
-                .join());
-        return new DefaultDistributedWorkQueue<>(atomixWorkQueue, options.serializer());
-    }
-
-    @Override
-    public <V> AsyncDocumentTree<V> newAsyncDocumentTree(DocumentTreeOptions options) {
-        String serviceType = String.format("%s-%s", DistributedPrimitive.Type.DOCUMENT_TREE.name(), options.ordering());
-        AtomixDocumentTree atomixDocumentTree = new AtomixDocumentTree(client.newProxyBuilder()
-                .withName(options.name())
-                .withServiceType(serviceType)
-                .withReadConsistency(ReadConsistency.SEQUENTIAL)
-                .withCommunicationStrategy(CommunicationStrategy.ANY)
-                .withMinTimeout(MIN_TIMEOUT)
-                .withMaxTimeout(MAX_TIMEOUT)
-                .withMaxRetries(MAX_RETRIES)
-                .withRevision(options.version() != null && options.revisionType() != null
-                    ? options.version().toInt() : 1)
-                .withPropagationStrategy(options.revisionType() != null
-                    ? PropagationStrategy.valueOf(options.revisionType().name())
-                    : PropagationStrategy.NONE)
-                .build()
-                .open()
-                .join());
-        return new DefaultDistributedDocumentTree<>(options.name(), atomixDocumentTree, options.serializer());
-    }
-
-    @Override
-    public AsyncDistributedLock newAsyncDistributedLock(DistributedLockOptions options) {
-        return new AtomixDistributedLock(client.newProxyBuilder()
-                .withName(options.name())
-                .withServiceType(DistributedPrimitive.Type.LOCK.name())
-                .withReadConsistency(ReadConsistency.LINEARIZABLE)
-                .withCommunicationStrategy(CommunicationStrategy.LEADER)
-                .withMinTimeout(MIN_TIMEOUT)
-                .withMaxTimeout(MIN_TIMEOUT)
-                .withMaxRetries(MAX_RETRIES)
-                .withRevision(options.version() != null && options.revisionType() != null
-                    ? options.version().toInt() : 1)
-                .withPropagationStrategy(options.revisionType() != null
-                    ? PropagationStrategy.valueOf(options.revisionType().name())
-                    : PropagationStrategy.NONE)
-                .build()
-                .open()
-                .join());
-    }
-
-    @Override
-    public AsyncLeaderElector newAsyncLeaderElector(LeaderElectorOptions options) {
-        return new AtomixLeaderElector(client.newProxyBuilder()
-                .withName(options.name())
-                .withServiceType(DistributedPrimitive.Type.LEADER_ELECTOR.name())
-                .withReadConsistency(ReadConsistency.LINEARIZABLE)
-                .withCommunicationStrategy(CommunicationStrategy.LEADER)
-                .withMinTimeout(Duration.ofMillis(options.electionTimeoutMillis()))
-                .withMaxTimeout(MIN_TIMEOUT)
-                .withMaxRetries(MAX_RETRIES)
-                .withRevision(options.version() != null && options.revisionType() != null
-                    ? options.version().toInt() : 1)
-                .withPropagationStrategy(options.revisionType() != null
-                    ? PropagationStrategy.valueOf(options.revisionType().name())
-                    : PropagationStrategy.NONE)
-                .build()
-                .open()
-                .join());
-    }
-
-    @Override
-    public Set<String> getAsyncConsistentMapNames() {
-        return client.metadata().getSessions(DistributedPrimitive.Type.CONSISTENT_MAP.name())
-                .join()
-                .stream()
-                .map(RaftSessionMetadata::serviceName)
-                .collect(Collectors.toSet());
-    }
-
-    @Override
-    public Set<String> getAsyncAtomicCounterNames() {
-        return client.metadata().getSessions(DistributedPrimitive.Type.COUNTER.name())
-                .join()
-                .stream()
-                .map(RaftSessionMetadata::serviceName)
-                .collect(Collectors.toSet());
-    }
-
-    @Override
-    public Set<String> getWorkQueueNames() {
-        return client.metadata().getSessions(DistributedPrimitive.Type.WORK_QUEUE.name())
-                .join()
-                .stream()
-                .map(RaftSessionMetadata::serviceName)
-                .collect(Collectors.toSet());
-    }
-
-    @Override
-    public boolean isOpen() {
-        return client != null;
-    }
-
-    /**
-     * Returns the {@link PartitionClientInfo information} for this client.
-     * @return partition client information
-     */
-    public PartitionClientInfo clientInfo() {
-        return new PartitionClientInfo(partition.getId(), partition.getMembers());
-    }
-
-    private RaftClient newRaftClient(RaftClientProtocol protocol) {
-        return RaftClient.newBuilder()
-                .withClientId("partition-" + partition.getId())
-                .withMemberId(localMemberId)
-                .withProtocol(protocol)
-                .build();
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartitionDetails.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartitionDetails.java
deleted file mode 100644
index 1bed89d..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartitionDetails.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.Collection;
-import java.util.Set;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-
-import io.atomix.protocols.raft.cluster.RaftMember;
-import org.onosproject.cluster.PartitionId;
-import org.onosproject.store.service.PartitionInfo;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.ImmutableSet;
-
-/**
- * Operational details for a {@code StoragePartition}.
- */
-public class StoragePartitionDetails {
-
-    private final PartitionId partitionId;
-    private final Set<RaftMember> activeMembers;
-    private final Set<RaftMember> configuredMembers;
-    private final RaftMember leader;
-    private final long leaderTerm;
-
-    public StoragePartitionDetails(PartitionId partitionId,
-            Collection<RaftMember> activeMembers,
-            Collection<RaftMember> configuredMembers,
-            RaftMember leader,
-            long leaderTerm) {
-        this.partitionId = partitionId;
-        this.activeMembers = ImmutableSet.copyOf(activeMembers);
-        this.configuredMembers = ImmutableSet.copyOf(configuredMembers);
-        this.leader = leader;
-        this.leaderTerm = leaderTerm;
-    }
-
-    /**
-     * Returns the set of active members.
-     * @return active members
-     */
-    public Set<RaftMember> activeMembers() {
-        return activeMembers;
-    }
-
-    /**
-     * Returns the set of configured members.
-     * @return configured members
-     */
-    public Set<RaftMember> configuredMembers() {
-        return configuredMembers;
-    }
-
-    /**
-     * Returns the partition leader.
-     * @return leader
-     */
-    public RaftMember leader() {
-        return leader;
-    }
-
-    /**
-     * Returns the partition leader term.
-     * @return leader term
-     */
-    public long leaderTerm() {
-        return leaderTerm;
-    }
-
-    @Override
-    public String toString() {
-        return MoreObjects.toStringHelper(getClass())
-                .add("activeMembers", activeMembers)
-                .add("configuredMembers", configuredMembers)
-                .add("leader", leader)
-                .add("leaderTerm", leaderTerm)
-                .toString();
-    }
-
-    /**
-     * Returns the details as an instance of {@code PartitionInfo}.
-     * @return partition info
-     */
-    public PartitionInfo toPartitionInfo() {
-        Function<RaftMember, String> memberToString =
-                m -> m == null ? "none" : m.memberId().toString();
-        return new PartitionInfo(partitionId,
-                leaderTerm,
-                activeMembers.stream().map(memberToString).collect(Collectors.toList()),
-                memberToString.apply(leader));
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartitionServer.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartitionServer.java
deleted file mode 100644
index f2600e8..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/StoragePartitionServer.java
+++ /dev/null
@@ -1,340 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.io.File;
-import java.io.IOException;
-import java.nio.file.FileVisitResult;
-import java.nio.file.Files;
-import java.nio.file.Path;
-import java.nio.file.SimpleFileVisitor;
-import java.nio.file.attribute.BasicFileAttributes;
-import java.time.Duration;
-import java.util.Collection;
-import java.util.concurrent.CompletableFuture;
-import java.util.stream.Collectors;
-
-import io.atomix.protocols.raft.RaftServer;
-import io.atomix.protocols.raft.cluster.MemberId;
-import io.atomix.protocols.raft.storage.RaftStorage;
-import io.atomix.storage.StorageLevel;
-import org.onosproject.cluster.Partition;
-import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
-import org.onosproject.store.primitives.resources.impl.AtomixSerializerAdapter;
-import org.onosproject.store.service.PartitionInfo;
-import org.onosproject.store.service.Serializer;
-import org.slf4j.Logger;
-
-import static org.slf4j.LoggerFactory.getLogger;
-
-/**
- * {@link StoragePartition} server.
- */
-public class StoragePartitionServer implements Managed<StoragePartitionServer> {
-
-    private final Logger log = getLogger(getClass());
-
-    private static final String ELECTION_TIMEOUT_MILLIS_PROPERTY = "onos.cluster.raft.electionTimeoutMillis";
-    private static final String ELECTION_THRESHOLD_PROPERTY = "onos.cluster.raft.electionFailureThreshold";
-    private static final String SESSION_THRESHOLD_PROPERTY = "onos.cluster.raft.sessionFailureThreshold";
-    private static final String HEARTBEAT_INTERVAL_MILLIS_PROPERTY = "onos.cluster.raft.heartbeatIntervalMillis";
-    private static final String MAX_SEGMENT_SIZE_PROPERTY = "onos.cluster.raft.storage.maxSegmentSize";
-    private static final String STORAGE_LEVEL_PROPERTY = "onos.cluster.raft.storage.level";
-    private static final String FLUSH_ON_COMMIT_PROPERTY = "onos.cluster.raft.storage.flushOnCommit";
-
-    private static final long ELECTION_TIMEOUT_MILLIS;
-    private static final int ELECTION_THRESHOLD;
-    private static final int SESSION_THRESHOLD;
-    private static final long HEARTBEAT_INTERVAL_MILLIS;
-    private static final int MAX_SEGMENT_SIZE;
-    private static final StorageLevel STORAGE_LEVEL;
-    private static final boolean FLUSH_ON_COMMIT;
-
-    private static final int DEFAULT_MAX_SEGMENT_SIZE = 1024 * 1024 * 64;
-    private static final long DEFAULT_ELECTION_TIMEOUT_MILLIS = 2500;
-    private static final int DEFAULT_ELECTION_THRESHOLD = 12;
-    private static final int DEFAULT_SESSION_THRESHOLD = 10;
-    private static final long DEFAULT_HEARTBEAT_INTERVAL_MILLIS = 500;
-    private static final StorageLevel DEFAULT_STORAGE_LEVEL = StorageLevel.MAPPED;
-    private static final boolean DEFAULT_FLUSH_ON_COMMIT = false;
-
-    static {
-        int maxSegmentSize;
-        try {
-            maxSegmentSize = Integer.parseInt(System.getProperty(
-                MAX_SEGMENT_SIZE_PROPERTY,
-                String.valueOf(DEFAULT_MAX_SEGMENT_SIZE)));
-        } catch (NumberFormatException e) {
-            maxSegmentSize = DEFAULT_MAX_SEGMENT_SIZE;
-        }
-        MAX_SEGMENT_SIZE = maxSegmentSize;
-
-        long electionTimeoutMillis;
-        try {
-            electionTimeoutMillis = Long.parseLong(System.getProperty(
-                ELECTION_TIMEOUT_MILLIS_PROPERTY,
-                String.valueOf(DEFAULT_ELECTION_TIMEOUT_MILLIS)));
-        } catch (NumberFormatException e) {
-            electionTimeoutMillis = DEFAULT_ELECTION_TIMEOUT_MILLIS;
-        }
-        ELECTION_TIMEOUT_MILLIS = electionTimeoutMillis;
-
-        int electionFailureThreshold;
-        try {
-            electionFailureThreshold = Integer.parseInt(System.getProperty(
-                ELECTION_THRESHOLD_PROPERTY,
-                String.valueOf(DEFAULT_ELECTION_THRESHOLD)));
-        } catch (NumberFormatException e) {
-            electionFailureThreshold = DEFAULT_ELECTION_THRESHOLD;
-        }
-        ELECTION_THRESHOLD = electionFailureThreshold;
-
-        int sessionFailureThreshold;
-        try {
-            sessionFailureThreshold = Integer.parseInt(System.getProperty(
-                SESSION_THRESHOLD_PROPERTY,
-                String.valueOf(DEFAULT_SESSION_THRESHOLD)));
-        } catch (NumberFormatException e) {
-            sessionFailureThreshold = DEFAULT_SESSION_THRESHOLD;
-        }
-        SESSION_THRESHOLD = sessionFailureThreshold;
-
-        long heartbeatIntervalMillis;
-        try {
-            heartbeatIntervalMillis = Long.parseLong(System.getProperty(
-                HEARTBEAT_INTERVAL_MILLIS_PROPERTY,
-                String.valueOf(DEFAULT_HEARTBEAT_INTERVAL_MILLIS)));
-        } catch (NumberFormatException e) {
-            heartbeatIntervalMillis = DEFAULT_HEARTBEAT_INTERVAL_MILLIS;
-        }
-        HEARTBEAT_INTERVAL_MILLIS = heartbeatIntervalMillis;
-
-        StorageLevel storageLevel;
-        try {
-            storageLevel = StorageLevel.valueOf(System.getProperty(
-                STORAGE_LEVEL_PROPERTY,
-                DEFAULT_STORAGE_LEVEL.name()).toUpperCase());
-        } catch (IllegalArgumentException e) {
-            storageLevel = DEFAULT_STORAGE_LEVEL;
-        }
-        STORAGE_LEVEL = storageLevel;
-
-        boolean flushOnCommit;
-        try {
-            flushOnCommit = Boolean.parseBoolean(System.getProperty(
-                FLUSH_ON_COMMIT_PROPERTY,
-                String.valueOf(DEFAULT_FLUSH_ON_COMMIT)));
-        } catch (Exception e) {
-            flushOnCommit = DEFAULT_FLUSH_ON_COMMIT;
-        }
-        FLUSH_ON_COMMIT = flushOnCommit;
-    }
-
-    private final MemberId localMemberId;
-    private final StoragePartition partition;
-    private final ClusterCommunicationService clusterCommunicator;
-    private RaftServer server;
-
-    public StoragePartitionServer(
-            StoragePartition partition,
-            MemberId localMemberId,
-            ClusterCommunicationService clusterCommunicator) {
-        this.partition = partition;
-        this.localMemberId = localMemberId;
-        this.clusterCommunicator = clusterCommunicator;
-    }
-
-    @Override
-    public CompletableFuture<Void> open() {
-        log.info("Starting server for partition {} ({})", partition.getId(), partition.getVersion());
-        CompletableFuture<RaftServer> serverOpenFuture;
-        if (partition.getMemberIds().contains(localMemberId)) {
-            if (server != null && server.isRunning()) {
-                return CompletableFuture.completedFuture(null);
-            }
-            synchronized (this) {
-                server = buildServer();
-            }
-            serverOpenFuture = server.bootstrap(partition.getMemberIds());
-        } else {
-            serverOpenFuture = CompletableFuture.completedFuture(null);
-        }
-        return serverOpenFuture.whenComplete((r, e) -> {
-            if (e == null) {
-                log.info("Successfully started server for partition {} ({})",
-                        partition.getId(), partition.getVersion());
-            } else {
-                log.info("Failed to start server for partition {} ({})",
-                        partition.getId(), partition.getVersion(), e);
-            }
-        }).thenApply(v -> null);
-    }
-
-    @Override
-    public CompletableFuture<Void> close() {
-        return server.shutdown();
-    }
-
-    /**
-     * Closes the server and exits the partition.
-     * @return future that is completed when the operation is complete
-     */
-    public CompletableFuture<Void> closeAndExit() {
-        return server.leave();
-    }
-
-    /**
-     * Deletes the server.
-     */
-    public void delete() {
-        try {
-            Files.walkFileTree(partition.getDataFolder().toPath(), new SimpleFileVisitor<Path>() {
-                @Override
-                public FileVisitResult visitFile(Path file, BasicFileAttributes attrs) throws IOException {
-                    Files.delete(file);
-                    return FileVisitResult.CONTINUE;
-                }
-                @Override
-                public FileVisitResult postVisitDirectory(Path dir, IOException exc) throws IOException {
-                    Files.delete(dir);
-                    return FileVisitResult.CONTINUE;
-                }
-            });
-        } catch (IOException e) {
-            log.error("Failed to delete partition: {}", e);
-        }
-    }
-
-    /**
-     * Forks the existing partition into a new partition.
-     *
-     * @param fromPartition the partition from which to fork the server
-     * @return future to be completed once the fork operation is complete
-     */
-    public CompletableFuture<Void> fork(Partition fromPartition) {
-        log.info("Forking server for partition {} ({}->{})",
-                partition.getId(), fromPartition.getVersion(), partition.getVersion());
-        RaftServer.Builder builder = RaftServer.newBuilder(localMemberId)
-                .withName(String.format("partition-%s", fromPartition.getId()))
-                .withProtocol(new RaftServerCommunicator(
-                        String.format("partition-%s-%s", fromPartition.getId(), fromPartition.getVersion()),
-                        Serializer.using(StorageNamespaces.RAFT_PROTOCOL),
-                        clusterCommunicator))
-                .withElectionTimeout(Duration.ofMillis(ELECTION_TIMEOUT_MILLIS))
-                .withHeartbeatInterval(Duration.ofMillis(HEARTBEAT_INTERVAL_MILLIS))
-                .withElectionThreshold(ELECTION_THRESHOLD)
-                .withSessionFailureThreshold(SESSION_THRESHOLD)
-                .withStorage(RaftStorage.newBuilder()
-                        .withPrefix(String.format("partition-%s", partition.getId()))
-                        .withStorageLevel(STORAGE_LEVEL)
-                        .withFlushOnCommit(FLUSH_ON_COMMIT)
-                        .withSerializer(new AtomixSerializerAdapter(Serializer.using(StorageNamespaces.RAFT_STORAGE)))
-                        .withDirectory(partition.getDataFolder())
-                        .withMaxSegmentSize(MAX_SEGMENT_SIZE)
-                        .build());
-        StoragePartition.RAFT_SERVICES.forEach(builder::addService);
-        RaftServer server = builder.build();
-
-        // Create a collection of members currently in the source partition.
-        Collection<MemberId> members = fromPartition.getMembers()
-                .stream()
-                .map(id -> MemberId.from(id.id()))
-                .collect(Collectors.toList());
-
-        // If this node is a member of the partition, join the partition. Otherwise, listen to the partition.
-        CompletableFuture<RaftServer> future = members.contains(localMemberId)
-                ? server.bootstrap(members) : server.listen(members);
-
-        // TODO: We should leave the cluster for nodes that aren't normally members to ensure the source
-        // cluster's configuration is kept consistent for rolling back upgrades, but Atomix deletes configuration
-        // files when a node leaves the cluster so we can't do that here.
-        return future.thenCompose(v -> server.shutdown())
-                .thenCompose(v -> {
-                    // Delete the cluster configuration file from the forked partition.
-                    try {
-                        Files.delete(new File(
-                                partition.getDataFolder(),
-                                String.format("partition-%s.conf", partition.getId())).toPath());
-                    } catch (IOException e) {
-                        log.error("Failed to delete partition configuration: {}", e);
-                    }
-
-                    // Build and bootstrap a new server.
-                    this.server = buildServer();
-                    return this.server.bootstrap();
-                }).whenComplete((r, e) -> {
-                    if (e == null) {
-                        log.info("Successfully forked server for partition {} ({}->{})",
-                                partition.getId(), fromPartition.getVersion(), partition.getVersion());
-                    } else {
-                        log.info("Failed to fork server for partition {} ({}->{})",
-                                partition.getId(), fromPartition.getVersion(), partition.getVersion(), e);
-                    }
-                }).thenApply(v -> null);
-    }
-
-    private RaftServer buildServer() {
-        RaftServer.Builder builder = RaftServer.newBuilder(localMemberId)
-                .withName(String.format("partition-%s", partition.getId()))
-                .withProtocol(new RaftServerCommunicator(
-                        String.format("partition-%s-%s", partition.getId(), partition.getVersion()),
-                        Serializer.using(StorageNamespaces.RAFT_PROTOCOL),
-                        clusterCommunicator))
-                .withElectionTimeout(Duration.ofMillis(ELECTION_TIMEOUT_MILLIS))
-                .withHeartbeatInterval(Duration.ofMillis(HEARTBEAT_INTERVAL_MILLIS))
-                .withElectionThreshold(ELECTION_THRESHOLD)
-                .withSessionFailureThreshold(SESSION_THRESHOLD)
-                .withStorage(RaftStorage.newBuilder()
-                        .withPrefix(String.format("partition-%s", partition.getId()))
-                        .withStorageLevel(STORAGE_LEVEL)
-                        .withFlushOnCommit(FLUSH_ON_COMMIT)
-                        .withSerializer(new AtomixSerializerAdapter(Serializer.using(StorageNamespaces.RAFT_STORAGE)))
-                        .withDirectory(partition.getDataFolder())
-                        .withMaxSegmentSize(MAX_SEGMENT_SIZE)
-                        .build());
-        StoragePartition.RAFT_SERVICES.forEach(builder::addService);
-        return builder.build();
-    }
-
-    public CompletableFuture<Void> join(Collection<MemberId> otherMembers) {
-        log.info("Joining partition {} ({})", partition.getId(), partition.getName());
-        server = buildServer();
-        return server.join(otherMembers).whenComplete((r, e) -> {
-            if (e == null) {
-                log.info("Successfully joined partition {} ({})", partition.getId(), partition.getName());
-            } else {
-                log.info("Failed to join partition {} ({})", partition.getId(), partition.getName(), e);
-            }
-        }).thenApply(v -> null);
-    }
-
-    @Override
-    public boolean isOpen() {
-        return server.isRunning();
-    }
-
-    /**
-     * Returns the partition information.
-     * @return partition info
-     */
-    public PartitionInfo info() {
-        return new StoragePartitionDetails(partition.getId(),
-                server.cluster().getMembers(),
-                server.cluster().getMembers(),
-                server.cluster().getLeader(),
-                server.cluster().getTerm()).toPartitionInfo();
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/Transaction.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/Transaction.java
deleted file mode 100644
index 28074e2..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/Transaction.java
+++ /dev/null
@@ -1,296 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.List;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Transactional;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.TransactionContext;
-import org.onosproject.store.service.TransactionException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static com.google.common.base.MoreObjects.toStringHelper;
-import static com.google.common.base.Preconditions.checkState;
-
-/**
- * Manages a transaction within the context of a single primitive.
- * <p>
- * The {@code Transaction} object is used to manage the transaction for a single partition primitive that implements
- * the {@link Transactional} interface. It's used as a proxy for {@link TransactionContext}s to manage the transaction
- * as it relates to a single piece of atomic state.
- */
-public class Transaction<T> {
-
-    /**
-     * Transaction state.
-     * <p>
-     * The transaction state is used to indicate the phase within which the transaction is currently running.
-     */
-    enum State {
-
-        /**
-         * Active transaction state.
-         * <p>
-         * The {@code ACTIVE} state represents a transaction in progress. Active transactions may or may not affect
-         * concurrently running transactions depending on the transaction's isolation level.
-         */
-        ACTIVE,
-
-        /**
-         * Preparing transaction state.
-         * <p>
-         * Once a transaction commitment begins, it enters the {@code PREPARING} phase of the two-phase commit protocol.
-         */
-        PREPARING,
-
-        /**
-         * Prepared transaction state.
-         * <p>
-         * Once the first phase of the two-phase commit protocol is complete, the transaction's state is set to
-         * {@code PREPARED}.
-         */
-        PREPARED,
-
-        /**
-         * Committing transaction state.
-         * <p>
-         * The {@code COMMITTING} state represents a transaction within the second phase of the two-phase commit
-         * protocol.
-         */
-        COMMITTING,
-
-        /**
-         * Committed transaction state.
-         * <p>
-         * Once the second phase of the two-phase commit protocol is complete, the transaction's state is set to
-         * {@code COMMITTED}.
-         */
-        COMMITTED,
-
-        /**
-         * Rolling back transaction state.
-         * <p>
-         * In the event of a two-phase lock failure, when the transaction is rolled back it will enter the
-         * {@code ROLLING_BACK} state while the rollback is in progress.
-         */
-        ROLLING_BACK,
-
-        /**
-         * Rolled back transaction state.
-         * <p>
-         * Once a transaction has been rolled back, it will enter the {@code ROLLED_BACK} state.
-         */
-        ROLLED_BACK,
-    }
-
-    private static final String TX_OPEN_ERROR = "transaction already open";
-    private static final String TX_CLOSED_ERROR = "transaction not open";
-    private static final String TX_INACTIVE_ERROR = "transaction is not active";
-    private static final String TX_UNPREPARED_ERROR = "transaction has not been prepared";
-
-    protected final Logger log = LoggerFactory.getLogger(getClass());
-    protected final TransactionId transactionId;
-    protected final Transactional<T> transactionalObject;
-    private final AtomicBoolean open = new AtomicBoolean();
-    private volatile State state = State.ACTIVE;
-    private volatile Version lock;
-
-    public Transaction(TransactionId transactionId, Transactional<T> transactionalObject) {
-        this.transactionId = transactionId;
-        this.transactionalObject = transactionalObject;
-    }
-
-    /**
-     * Returns the transaction identifier.
-     *
-     * @return the transaction identifier
-     */
-    public TransactionId transactionId() {
-        return transactionId;
-    }
-
-    /**
-     * Returns the current transaction state.
-     *
-     * @return the current transaction state
-     */
-    public State state() {
-        return state;
-    }
-
-    /**
-     * Returns a boolean indicating whether the transaction is open.
-     *
-     * @return indicates whether the transaction is open
-     */
-    public boolean isOpen() {
-        return open.get();
-    }
-
-    /**
-     * Opens the transaction, throwing an {@link IllegalStateException} if it's already open.
-     */
-    protected void open() {
-        if (!open.compareAndSet(false, true)) {
-            throw new IllegalStateException(TX_OPEN_ERROR);
-        }
-    }
-
-    /**
-     * Checks that the transaction is open and throws an {@link IllegalStateException} if not.
-     */
-    protected void checkOpen() {
-        checkState(isOpen(), TX_CLOSED_ERROR);
-    }
-
-    /**
-     * Checks that the transaction state is {@code ACTIVE} and throws an {@link IllegalStateException} if not.
-     */
-    protected void checkActive() {
-        checkState(state == State.ACTIVE, TX_INACTIVE_ERROR);
-    }
-
-    /**
-     * Checks that the transaction state is {@code PREPARED} and throws an {@link IllegalStateException} if not.
-     */
-    protected void checkPrepared() {
-        checkState(state == State.PREPARED, TX_UNPREPARED_ERROR);
-    }
-
-    /**
-     * Updates the transaction state.
-     *
-     * @param state the updated transaction state
-     */
-    protected void setState(State state) {
-        this.state = state;
-    }
-
-    /**
-     * Begins the transaction.
-     * <p>
-     * Locks are acquired when the transaction is begun to prevent concurrent transactions from operating on the shared
-     * resource to which this transaction relates.
-     *
-     * @return a completable future to be completed once the transaction has been started
-     */
-    public CompletableFuture<Version> begin() {
-        log.debug("Beginning transaction {} for {}", transactionId, transactionalObject);
-        open();
-        return transactionalObject.begin(transactionId).thenApply(lock -> {
-            this.lock = lock;
-            log.trace("Transaction lock acquired: {}", lock);
-            return lock;
-        });
-    }
-
-    /**
-     * Prepares the transaction.
-     * <p>
-     * When preparing the transaction, the given list of updates for the shared resource will be prepared, and
-     * concurrent modification checks will be performed. The returned future may be completed with a
-     * {@link TransactionException} if a concurrent modification is detected for an isolation level that does
-     * not allow such modifications.
-     *
-     * @param updates the transaction updates
-     * @return a completable future to be completed once the transaction has been prepared
-     */
-    public CompletableFuture<Boolean> prepare(List<T> updates) {
-        checkOpen();
-        checkActive();
-        log.debug("Preparing transaction {} for {}", transactionId, transactionalObject);
-        Version lock = this.lock;
-        checkState(lock != null, TX_INACTIVE_ERROR);
-        setState(State.PREPARING);
-        return transactionalObject.prepare(new TransactionLog<T>(transactionId, lock.value(), updates))
-                .thenApply(succeeded -> {
-                    setState(State.PREPARED);
-                    return succeeded;
-                });
-    }
-
-    /**
-     * Prepares and commits the transaction in a single atomic operation.
-     * <p>
-     * Both the prepare and commit phases of the protocol must be executed within a single atomic operation. This method
-     * is used to optimize committing transactions that operate only on a single partition within a single primitive.
-     *
-     * @param updates the transaction updates
-     * @return a completable future to be completed once the transaction has been prepared
-     */
-    public CompletableFuture<Boolean> prepareAndCommit(List<T> updates) {
-        checkOpen();
-        checkActive();
-        log.debug("Preparing and committing transaction {} for {}", transactionId, transactionalObject);
-        Version lock = this.lock;
-        checkState(lock != null, TX_INACTIVE_ERROR);
-        setState(State.PREPARING);
-        return transactionalObject.prepareAndCommit(new TransactionLog<T>(transactionId, lock.value(), updates))
-                .thenApply(succeeded -> {
-                    setState(State.COMMITTED);
-                    return succeeded;
-                });
-    }
-
-    /**
-     * Commits the transaction.
-     * <p>
-     * Performs the second phase of the two-phase commit protocol, committing the previously
-     * {@link #prepare(List) prepared} updates.
-     *
-     * @return a completable future to be completed once the transaction has been committed
-     */
-    public CompletableFuture<Void> commit() {
-        checkOpen();
-        checkPrepared();
-        log.debug("Committing transaction {} for {}", transactionId, transactionalObject);
-        setState(State.COMMITTING);
-        return transactionalObject.commit(transactionId).thenRun(() -> {
-            setState(State.COMMITTED);
-        });
-    }
-
-    /**
-     * Rolls back the transaction.
-     * <p>
-     * Rolls back the first phase of the two-phase commit protocol, cancelling prepared updates.
-     *
-     * @return a completable future to be completed once the transaction has been rolled back
-     */
-    public CompletableFuture<Void> rollback() {
-        checkOpen();
-        checkPrepared();
-        log.debug("Rolling back transaction {} for {}", transactionId, transactionalObject);
-        setState(State.ROLLING_BACK);
-        return transactionalObject.rollback(transactionId).thenRun(() -> {
-            setState(State.ROLLED_BACK);
-        });
-    }
-
-    @Override
-    public String toString() {
-        return toStringHelper(this)
-                .add("transactionId", transactionId)
-                .add("state", state)
-                .toString();
-    }
-}
\ 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
deleted file mode 100644
index 23d4399..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionCoordinator.java
+++ /dev/null
@@ -1,150 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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 com.google.common.collect.Sets;
-import org.onlab.util.Tools;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.CommitStatus;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionalMap;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static com.google.common.base.MoreObjects.toStringHelper;
-
-/**
- * Transaction coordinator.
- */
-public class TransactionCoordinator {
-    private final Logger log = LoggerFactory.getLogger(getClass());
-    protected final TransactionId transactionId;
-    protected final TransactionManager transactionManager;
-    protected final Set<TransactionParticipant> transactionParticipants = Sets.newConcurrentHashSet();
-
-    public TransactionCoordinator(TransactionId transactionId, TransactionManager transactionManager) {
-        this.transactionId = transactionId;
-        this.transactionManager = transactionManager;
-    }
-
-    /**
-     * Returns a transactional map for this transaction.
-     *
-     * @param name the transactional map name
-     * @param serializer the serializer
-     * @param <K> key type
-     * @param <V> value type
-     * @return a transactional map for this transaction
-     */
-    public <K, V> TransactionalMap<K, V> getTransactionalMap(String name, Serializer serializer) {
-        PartitionedTransactionalMap<K, V> map = transactionManager.getTransactionalMap(name, serializer, this);
-        transactionParticipants.addAll(map.participants());
-        return map;
-    }
-
-    /**
-     * Commits the transaction.
-     *
-     * @return the transaction commit status
-     */
-    public CompletableFuture<CommitStatus> commit() {
-        long totalParticipants = transactionParticipants.stream()
-                .filter(TransactionParticipant::hasPendingUpdates)
-                .count();
-
-        if (totalParticipants == 0) {
-            log.debug("No transaction participants, skipping commit", totalParticipants);
-            return CompletableFuture.completedFuture(CommitStatus.SUCCESS);
-        } else if (totalParticipants == 1) {
-            log.debug("Committing transaction {} via 1 participant", transactionId);
-            return transactionParticipants.stream()
-                    .filter(TransactionParticipant::hasPendingUpdates)
-                    .findFirst()
-                    .get()
-                    .prepareAndCommit()
-                    .thenApply(v -> v ? CommitStatus.SUCCESS : CommitStatus.FAILURE);
-        } else {
-            log.debug("Committing transaction {} via {} participants", transactionId, totalParticipants);
-            Set<TransactionParticipant> transactionParticipants = this.transactionParticipants.stream()
-                    .filter(TransactionParticipant::hasPendingUpdates)
-                    .collect(Collectors.toSet());
-
-            CompletableFuture<CommitStatus> status = transactionManager.updateState(
-                    transactionId, Transaction.State.PREPARING)
-                    .thenCompose(v -> prepare(transactionParticipants))
-                    .thenCompose(result -> result
-                            ? transactionManager.updateState(transactionId, Transaction.State.COMMITTING)
-                            .thenCompose(v -> commit(transactionParticipants))
-                            .thenApply(v -> CommitStatus.SUCCESS)
-                            : transactionManager.updateState(transactionId, Transaction.State.ROLLING_BACK)
-                            .thenCompose(v -> rollback(transactionParticipants))
-                            .thenApply(v -> CommitStatus.FAILURE));
-            return status.thenCompose(v -> transactionManager.remove(transactionId).thenApply(u -> v));
-        }
-    }
-
-    /**
-     * Performs the prepare phase of the two-phase commit protocol for the given transaction participants.
-     *
-     * @param transactionParticipants the transaction participants for which to prepare the transaction
-     * @return a completable future indicating whether <em>all</em> prepares succeeded
-     */
-    protected CompletableFuture<Boolean> prepare(Set<TransactionParticipant> transactionParticipants) {
-        log.trace("Preparing transaction {} via {}", transactionId, transactionParticipants);
-        return Tools.allOf(transactionParticipants.stream()
-                .map(TransactionParticipant::prepare)
-                .collect(Collectors.toList()))
-                .thenApply(list -> list.stream().reduce(Boolean::logicalAnd).orElse(true));
-    }
-
-    /**
-     * Performs the commit phase of the two-phase commit protocol for the given transaction participants.
-     *
-     * @param transactionParticipants the transaction participants for which to commit the transaction
-     * @return a completable future to be completed once the commits are complete
-     */
-    protected CompletableFuture<Void> commit(Set<TransactionParticipant> transactionParticipants) {
-        log.trace("Committing transaction {} via {}", transactionId, transactionParticipants);
-        return CompletableFuture.allOf(transactionParticipants.stream()
-                .map(TransactionParticipant::commit)
-                .toArray(CompletableFuture[]::new));
-    }
-
-    /**
-     * Rolls back transactions for the given participants.
-     *
-     * @param transactionParticipants the transaction participants for which to roll back the transaction
-     * @return a completable future to be completed once the rollbacks are complete
-     */
-    protected CompletableFuture<Void> rollback(Set<TransactionParticipant> transactionParticipants) {
-        log.trace("Rolling back transaction {} via {}", transactionId, transactionParticipants);
-        return CompletableFuture.allOf(transactionParticipants.stream()
-                .map(TransactionParticipant::rollback)
-                .toArray(CompletableFuture[]::new));
-    }
-
-    @Override
-    public String toString() {
-        return toStringHelper(this)
-                .add("transactionId", transactionId)
-                .add("participants", transactionParticipants)
-                .toString();
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionManager.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionManager.java
deleted file mode 100644
index 7d8d4c4..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionManager.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.Collection;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
-
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.hash.Hashing;
-import com.google.common.util.concurrent.Futures;
-import org.onosproject.cluster.PartitionId;
-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.Serializer;
-import org.onosproject.store.service.StorageService;
-import org.onosproject.store.service.TransactionException;
-
-/**
- * Transaction manager for managing state shared across multiple transactions.
- */
-public class TransactionManager {
-    private static final int DEFAULT_CACHE_SIZE = 100;
-
-    private final PartitionService partitionService;
-    private final List<PartitionId> sortedPartitions;
-    private final AsyncConsistentMap<TransactionId, Transaction.State> transactions;
-    private final long cacheSize;
-    private final int buckets;
-    private final Map<PartitionId, Cache<String, CachedMap>> partitionCache = Maps.newConcurrentMap();
-
-    public TransactionManager(StorageService storageService, PartitionService partitionService, int buckets) {
-        this(storageService, partitionService, DEFAULT_CACHE_SIZE, buckets);
-    }
-
-    public TransactionManager(
-            StorageService storageService,
-            PartitionService partitionService,
-            int cacheSize,
-            int buckets) {
-        this.partitionService = partitionService;
-        this.cacheSize = cacheSize;
-        this.buckets = buckets;
-        this.transactions = storageService.<TransactionId, Transaction.State>consistentMapBuilder()
-                .withName("onos-transactions")
-                .withSerializer(Serializer.using(KryoNamespaces.API,
-                        Transaction.class,
-                        Transaction.State.class))
-                .buildAsyncMap();
-        this.sortedPartitions = Lists.newArrayList(partitionService.getAllPartitionIds());
-        Collections.sort(sortedPartitions);
-    }
-
-    /**
-     * Returns the collection of currently pending transactions.
-     *
-     * @return a collection of currently pending transactions
-     */
-    public Collection<TransactionId> getPendingTransactions() {
-        return Futures.getUnchecked(transactions.keySet());
-    }
-
-    /**
-     * Returns a partitioned transactional map for use within a transaction context.
-     * <p>
-     * The transaction coordinator will return a map that takes advantage of caching that's shared across transaction
-     * contexts.
-     *
-     * @param name the map name
-     * @param serializer the map serializer
-     * @param transactionCoordinator the transaction coordinator for which the map is being created
-     * @param <K> key type
-     * @param <V> value type
-     * @return a partitioned transactional map
-     */
-    <K, V> PartitionedTransactionalMap<K, V> getTransactionalMap(
-            String name,
-            Serializer serializer,
-            TransactionCoordinator transactionCoordinator) {
-        Map<PartitionId, TransactionalMapParticipant<K, V>> partitions = new HashMap<>();
-        for (PartitionId partitionId : partitionService.getAllPartitionIds()) {
-            partitions.put(partitionId, getTransactionalMapPartition(
-                    name, partitionId, serializer, transactionCoordinator));
-        }
-
-        Hasher<K> hasher = key -> {
-            int bucket = Math.abs(Hashing.murmur3_32().hashBytes(serializer.encode(key)).asInt()) % buckets;
-            int partition = Hashing.consistentHash(bucket, sortedPartitions.size());
-            return sortedPartitions.get(partition);
-        };
-        return new PartitionedTransactionalMap<>(partitions, hasher);
-    }
-
-    @SuppressWarnings("unchecked")
-    private <K, V> TransactionalMapParticipant<K, V> getTransactionalMapPartition(
-            String mapName,
-            PartitionId partitionId,
-            Serializer serializer,
-            TransactionCoordinator transactionCoordinator) {
-        Cache<String, CachedMap> mapCache = partitionCache.computeIfAbsent(partitionId, p ->
-                CacheBuilder.newBuilder().maximumSize(cacheSize / partitionService.getNumberOfPartitions()).build());
-        try {
-            CachedMap<K, V> cachedMap = mapCache.get(mapName,
-                    () -> new CachedMap<>(partitionService.getDistributedPrimitiveCreator(partitionId)
-                            .newAsyncConsistentMap(mapName, serializer)));
-
-            Transaction<MapUpdate<K, V>> transaction = new Transaction<>(
-                    transactionCoordinator.transactionId,
-                    cachedMap.baseMap);
-            return new DefaultTransactionalMapParticipant<>(cachedMap.cachedMap.asConsistentMap(), transaction);
-        } catch (ExecutionException e) {
-            throw new TransactionException(e);
-        }
-    }
-
-    /**
-     * Updates the state of a transaction in the transaction registry.
-     *
-     * @param transactionId the transaction identifier
-     * @param state the state of the transaction
-     * @return a completable future to be completed once the transaction state has been updated in the registry
-     */
-    CompletableFuture<Void> updateState(TransactionId transactionId, Transaction.State state) {
-        return transactions.put(transactionId, state).thenApply(v -> null);
-    }
-
-    /**
-     * Removes the given transaction from the transaction registry.
-     *
-     * @param transactionId the transaction identifier
-     * @return a completable future to be completed once the transaction state has been removed from the registry
-     */
-    CompletableFuture<Void> remove(TransactionId transactionId) {
-        return transactions.remove(transactionId).thenApply(v -> null);
-    }
-
-    private static class CachedMap<K, V> {
-        private final AsyncConsistentMap<K, V> baseMap;
-        private final AsyncConsistentMap<K, V> cachedMap;
-
-        public CachedMap(AsyncConsistentMap<K, V> baseMap) {
-            this.baseMap = baseMap;
-            this.cachedMap = DistributedPrimitives.newCachingMap(baseMap);
-        }
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionParticipant.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionParticipant.java
deleted file mode 100644
index 02da9d9..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionParticipant.java
+++ /dev/null
@@ -1,60 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.concurrent.CompletableFuture;
-
-/**
- * Participant in a two-phase commit protocol.
- */
-public interface TransactionParticipant {
-
-    /**
-     * Returns a boolean indicating whether the participant has pending updates.
-     *
-     * @return indicates whether the participant has pending updates
-     */
-    boolean hasPendingUpdates();
-
-    /**
-     * Executes the prepare phase.
-     *
-     * @return {@code true} is successful; {@code false} otherwise
-     */
-    CompletableFuture<Boolean> prepare();
-
-    /**
-     * Attempts to execute the commit phase for previously prepared transaction.
-     *
-     * @return future that is completed when the operation completes
-     */
-    CompletableFuture<Void> commit();
-
-    /**
-     * Executes the prepare and commit phases atomically.
-     *
-     * @return {@code true} is successful; {@code false} otherwise
-     */
-    CompletableFuture<Boolean> prepareAndCommit();
-
-    /**
-     * Attempts to execute the rollback phase for previously prepared transaction.
-     *
-     * @return future that is completed when the operation completes
-     */
-    CompletableFuture<Void> rollback();
-
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionalMapParticipant.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionalMapParticipant.java
deleted file mode 100644
index 3cb87a4..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TransactionalMapParticipant.java
+++ /dev/null
@@ -1,229 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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 com.google.common.base.MoreObjects;
-import com.google.common.base.Objects;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.service.ConsistentMap;
-import org.onosproject.store.service.DistributedPrimitive;
-import org.onosproject.store.service.TransactionException;
-import org.onosproject.store.service.TransactionalMap;
-import org.onosproject.store.service.Version;
-
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.google.common.base.Preconditions.checkState;
-
-/**
- * Base class for participants within a single {@link TransactionalMap}.
- * <p>
- * This class provides the basic functionality required by transactional map participants and provides methods
- * for defining operations specific to individual isolation levels.
- *
- * @param <K> key type
- * @param <V> value type.
- */
-public abstract class TransactionalMapParticipant<K, V> implements TransactionalMap<K, V>, TransactionParticipant {
-    private static final String TX_CLOSED_ERROR = "Transaction is closed";
-    private static final String ERROR_NULL_VALUE = "Null values are not allowed";
-    private static final String ERROR_NULL_KEY = "Null key is not allowed";
-
-    protected final ConsistentMap<K, V> backingMap;
-    protected final Transaction<MapUpdate<K, V>> transaction;
-    protected final Map<K, V> writeCache = Maps.newConcurrentMap();
-    protected final Set<K> deleteSet = Sets.newConcurrentHashSet();
-    protected volatile Version lock;
-
-    protected TransactionalMapParticipant(
-            ConsistentMap<K, V> backingMap,
-            Transaction<MapUpdate<K, V>> transaction) {
-        this.backingMap = backingMap;
-        this.transaction = transaction;
-    }
-
-    /**
-     * Starts the transaction for this partition when a read occurs.
-     * <p>
-     * Acquiring a pessimistic lock at the start of the transaction ensures that underlying cached maps have been
-     * synchronized prior to a read.
-     */
-    private void beginTransaction() {
-        if (lock == null) {
-            synchronized (this) {
-                if (lock == null) {
-                    try {
-                        lock = transaction.begin()
-                                .get(DistributedPrimitive.DEFAULT_OPERATION_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS);
-                    } catch (InterruptedException e) {
-                        Thread.currentThread().interrupt();
-                        throw new TransactionException.Interrupted();
-                    } catch (TimeoutException e) {
-                        throw new TransactionException.Timeout();
-                    } catch (ExecutionException e) {
-                        throw new TransactionException(e.getCause());
-                    }
-                }
-            }
-        }
-    }
-
-    @Override
-    public V get(K key) {
-        // Start the transaction for this primitive/partition if necessary.
-        beginTransaction();
-
-        checkState(transaction.isOpen(), TX_CLOSED_ERROR);
-        checkNotNull(key, ERROR_NULL_KEY);
-
-        if (deleteSet.contains(key)) {
-            return null;
-        }
-
-        V latest = writeCache.get(key);
-        if (latest != null) {
-            return latest;
-        } else {
-            return read(key);
-        }
-    }
-
-    /**
-     * Executes a get operation based on the transaction isolation level.
-     *
-     * @param key the key to look up
-     * @return the value
-     */
-    protected abstract V read(K key);
-
-    @Override
-    public boolean containsKey(K key) {
-        return get(key) != null;
-    }
-
-    @Override
-    public V put(K key, V value) {
-        checkNotNull(value, ERROR_NULL_VALUE);
-
-        V latest = get(key);
-        writeCache.put(key, value);
-        deleteSet.remove(key);
-        return latest;
-    }
-
-    @Override
-    public V remove(K key) {
-        V latest = get(key);
-        if (latest != null) {
-            writeCache.remove(key);
-            deleteSet.add(key);
-        }
-        return latest;
-    }
-
-    @Override
-    public boolean remove(K key, V value) {
-        checkNotNull(value, ERROR_NULL_VALUE);
-
-        V latest = get(key);
-        if (Objects.equal(value, latest)) {
-            remove(key);
-            return true;
-        }
-        return false;
-    }
-
-    @Override
-    public boolean replace(K key, V oldValue, V newValue) {
-        checkNotNull(oldValue, ERROR_NULL_VALUE);
-        checkNotNull(newValue, ERROR_NULL_VALUE);
-
-        V latest = get(key);
-        if (Objects.equal(oldValue, latest)) {
-            put(key, newValue);
-            return true;
-        }
-        return false;
-    }
-
-    @Override
-    public V putIfAbsent(K key, V value) {
-        checkNotNull(value, ERROR_NULL_VALUE);
-
-        V latest = get(key);
-        if (latest == null) {
-            put(key, value);
-        }
-        return latest;
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepare() {
-        return transaction.prepare(log(lock));
-    }
-
-    @Override
-    public CompletableFuture<Void> commit() {
-        return transaction.commit();
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepareAndCommit() {
-        return transaction.prepareAndCommit(log(lock));
-    }
-
-    @Override
-    public CompletableFuture<Void> rollback() {
-        return transaction.rollback();
-    }
-
-    /**
-     * Returns a list of updates performed within this map partition.
-     *
-     * @param lockVersion the global transaction lock version
-     * @return a list of map updates
-     */
-    protected List<MapUpdate<K, V>> log(Version lockVersion) {
-        return records(lockVersion).collect(Collectors.toList());
-    }
-
-    /**
-     * Returns a stream of updates performed within this map partition.
-     *
-     * @param lockVersion the global transaction lock version
-     * @return a stream of map updates
-     */
-    protected abstract Stream<MapUpdate<K, V>> records(Version lockVersion);
-
-    @Override
-    public String toString() {
-        return MoreObjects.toStringHelper(this)
-                .add("backingMap", backingMap)
-                .toString();
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncAtomicCounterMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncAtomicCounterMap.java
deleted file mode 100644
index 550465d..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncAtomicCounterMap.java
+++ /dev/null
@@ -1,178 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.onlab.util.Tools;
-import org.onosproject.store.service.AsyncAtomicCounterMap;
-
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Function;
-
-/**
- * An {@code AsyncAtomicCounterMap} that transcodes keys.
- */
-public class TranscodingAsyncAtomicCounterMap<K1, K2> implements AsyncAtomicCounterMap<K1> {
-    private final AsyncAtomicCounterMap<K2> backingMap;
-    private final Function<K1, K2> keyEncoder;
-    private final Function<K2, K1> keyDecoder;
-
-    public TranscodingAsyncAtomicCounterMap(AsyncAtomicCounterMap<K2> backingMap,
-            Function<K1, K2> keyEncoder, Function<K2, K1> keyDecoder) {
-        this.backingMap = backingMap;
-        this.keyEncoder = k -> k == null ? null : keyEncoder.apply(k);
-        this.keyDecoder = k -> k == null ? null : keyDecoder.apply(k);
-    }
-
-    @Override
-    public String name() {
-        return backingMap.name();
-    }
-
-    @Override
-    public CompletableFuture<Long> incrementAndGet(K1 key) {
-        try {
-            return backingMap.incrementAndGet(keyEncoder.apply(key));
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Long> decrementAndGet(K1 key) {
-        try {
-            return backingMap.decrementAndGet(keyEncoder.apply(key));
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Long> getAndIncrement(K1 key) {
-        try {
-            return backingMap.getAndIncrement(keyEncoder.apply(key));
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Long> getAndDecrement(K1 key) {
-        try {
-            return backingMap.getAndDecrement(keyEncoder.apply(key));
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Long> addAndGet(K1 key, long delta) {
-        try {
-            return backingMap.addAndGet(keyEncoder.apply(key), delta);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Long> getAndAdd(K1 key, long delta) {
-        try {
-            return backingMap.getAndAdd(keyEncoder.apply(key), delta);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Long> get(K1 key) {
-        try {
-            return backingMap.get(keyEncoder.apply(key));
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Long> put(K1 key, long newValue) {
-        try {
-            return backingMap.put(keyEncoder.apply(key), newValue);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Long> putIfAbsent(K1 key, long newValue) {
-        try {
-            return backingMap.putIfAbsent(keyEncoder.apply(key), newValue);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(K1 key, long expectedOldValue, long newValue) {
-        try {
-            return backingMap.replace(keyEncoder.apply(key), expectedOldValue, newValue);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Long> remove(K1 key) {
-        try {
-            return backingMap.remove(keyEncoder.apply(key));
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(K1 key, long value) {
-        try {
-            return backingMap.remove(keyEncoder.apply(key), value);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Integer> size() {
-        try {
-            return backingMap.size();
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Boolean> isEmpty() {
-        try {
-            return backingMap.isEmpty();
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Void> clear() {
-        try {
-            return backingMap.clear();
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncAtomicValue.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncAtomicValue.java
deleted file mode 100644
index 24dd35d..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncAtomicValue.java
+++ /dev/null
@@ -1,122 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * 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.Collection;
-import java.util.Map;
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Consumer;
-import java.util.function.Function;
-
-import com.google.common.collect.Maps;
-import org.onosproject.store.service.AsyncAtomicValue;
-import org.onosproject.store.service.AtomicValueEvent;
-import org.onosproject.store.service.AtomicValueEventListener;
-
-/**
- * An {@code AsyncAtomicValue} that transcodes values.
- */
-public class TranscodingAsyncAtomicValue<V1, V2> implements AsyncAtomicValue<V1> {
-    private final AsyncAtomicValue<V2> backingValue;
-    private final Function<V1, V2> valueEncoder;
-    private final Function<V2, V1> valueDecoder;
-    private final Map<AtomicValueEventListener<V1>, InternalValueEventListener> listeners = Maps.newIdentityHashMap();
-
-    public TranscodingAsyncAtomicValue(
-        AsyncAtomicValue<V2> backingValue, Function<V1, V2> valueEncoder, Function<V2, V1> valueDecoder) {
-        this.backingValue = backingValue;
-        this.valueEncoder = k -> k == null ? null : valueEncoder.apply(k);
-        this.valueDecoder = k -> k == null ? null : valueDecoder.apply(k);
-    }
-
-    @Override
-    public String name() {
-        return backingValue.name();
-    }
-
-    @Override
-    public CompletableFuture<Boolean> compareAndSet(V1 expect, V1 update) {
-        return backingValue.compareAndSet(valueEncoder.apply(expect), valueEncoder.apply(update));
-    }
-
-    @Override
-    public CompletableFuture<V1> get() {
-        return backingValue.get().thenApply(valueDecoder);
-    }
-
-    @Override
-    public CompletableFuture<V1> getAndSet(V1 value) {
-        return backingValue.getAndSet(valueEncoder.apply(value)).thenApply(valueDecoder);
-    }
-
-    @Override
-    public CompletableFuture<Void> set(V1 value) {
-        return backingValue.set(valueEncoder.apply(value));
-    }
-
-    @Override
-    public CompletableFuture<Void> addListener(AtomicValueEventListener<V1> listener) {
-        synchronized (listeners) {
-            InternalValueEventListener backingMapListener =
-                listeners.computeIfAbsent(listener, k -> new InternalValueEventListener(listener));
-            return backingValue.addListener(backingMapListener);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(AtomicValueEventListener<V1> listener) {
-        synchronized (listeners) {
-            InternalValueEventListener backingMapListener = listeners.remove(listener);
-            if (backingMapListener != null) {
-                return backingValue.removeListener(backingMapListener);
-            } else {
-                return CompletableFuture.completedFuture(null);
-            }
-        }
-    }
-
-    @Override
-    public void addStatusChangeListener(Consumer<Status> listener) {
-        backingValue.addStatusChangeListener(listener);
-    }
-
-    @Override
-    public void removeStatusChangeListener(Consumer<Status> listener) {
-        backingValue.removeStatusChangeListener(listener);
-    }
-
-    @Override
-    public Collection<Consumer<Status>> statusChangeListeners() {
-        return backingValue.statusChangeListeners();
-    }
-
-    private class InternalValueEventListener implements AtomicValueEventListener<V2> {
-        private final AtomicValueEventListener<V1> listener;
-
-        InternalValueEventListener(AtomicValueEventListener<V1> listener) {
-            this.listener = listener;
-        }
-
-        @Override
-        public void event(AtomicValueEvent<V2> event) {
-            listener.event(new AtomicValueEvent<>(
-                event.name(),
-                event.newValue() != null ? valueDecoder.apply(event.newValue()) : null,
-                event.oldValue() != null ? valueDecoder.apply(event.oldValue()) : null));
-        }
-    }
-
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentMap.java
deleted file mode 100644
index d68ada5..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentMap.java
+++ /dev/null
@@ -1,374 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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 com.google.common.collect.Maps;
-import org.onlab.util.Tools;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MapEvent;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.BiFunction;
-import java.util.function.Consumer;
-import java.util.function.Function;
-import java.util.function.Predicate;
-import java.util.stream.Collectors;
-
-/**
- * An {@code AsyncConsistentMap} that maps its operations to operations on a
- * differently typed {@code AsyncConsistentMap} by transcoding operation inputs and outputs.
- *
- * @param <K2> key type of other map
- * @param <V2> value type of other map
- * @param <K1> key type of this map
- * @param <V1> value type of this map
- */
-public class TranscodingAsyncConsistentMap<K1, V1, K2, V2> implements AsyncConsistentMap<K1, V1> {
-
-    private final AsyncConsistentMap<K2, V2> backingMap;
-    private final Function<K1, K2> keyEncoder;
-    private final Function<K2, K1> keyDecoder;
-    private final Function<V2, V1> valueDecoder;
-    private final Function<V1, V2> valueEncoder;
-    private final Function<Versioned<V2>, Versioned<V1>> versionedValueTransform;
-    private final Map<MapEventListener<K1, V1>, InternalBackingMapEventListener> listeners =
-            Maps.newIdentityHashMap();
-
-    public TranscodingAsyncConsistentMap(AsyncConsistentMap<K2, V2> backingMap,
-                                   Function<K1, K2> keyEncoder,
-                                   Function<K2, K1> keyDecoder,
-                                   Function<V1, V2> valueEncoder,
-                                   Function<V2, V1> valueDecoder) {
-        this.backingMap = backingMap;
-        this.keyEncoder = k -> k == null ? null : keyEncoder.apply(k);
-        this.keyDecoder = k -> k == null ? null : keyDecoder.apply(k);
-        this.valueEncoder = v -> v == null ? null : valueEncoder.apply(v);
-        this.valueDecoder = v -> v == null ? null : valueDecoder.apply(v);
-        this.versionedValueTransform = v -> v == null ? null : v.map(valueDecoder);
-    }
-
-    @Override
-    public String name() {
-        return backingMap.name();
-    }
-
-    @Override
-    public CompletableFuture<Integer> size() {
-        return backingMap.size();
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsKey(K1 key) {
-        try {
-            return backingMap.containsKey(keyEncoder.apply(key));
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsValue(V1 value) {
-        try {
-            return backingMap.containsValue(valueEncoder.apply(value));
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V1>> get(K1 key) {
-        try {
-            return backingMap.get(keyEncoder.apply(key)).thenApply(versionedValueTransform);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V1>> getOrDefault(K1 key, V1 defaultValue) {
-        try {
-            return backingMap.getOrDefault(keyEncoder.apply(key), valueEncoder.apply(defaultValue))
-                    .thenApply(versionedValueTransform);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V1>> computeIf(K1 key,
-            Predicate<? super V1> condition,
-            BiFunction<? super K1, ? super V1, ? extends V1> remappingFunction) {
-        try {
-            return backingMap.computeIf(keyEncoder.apply(key),
-                    v -> condition.test(valueDecoder.apply(v)),
-                    (k, v) -> valueEncoder.apply(remappingFunction.apply(keyDecoder.apply(k),
-                            valueDecoder.apply(v))))
-                            .thenApply(versionedValueTransform);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V1>> put(K1 key, V1 value) {
-        try {
-            return backingMap.put(keyEncoder.apply(key), valueEncoder.apply(value))
-                             .thenApply(versionedValueTransform);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V1>> putAndGet(K1 key, V1 value) {
-        try {
-            return backingMap.putAndGet(keyEncoder.apply(key), valueEncoder.apply(value))
-                             .thenApply(versionedValueTransform);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V1>> remove(K1 key) {
-        try {
-            return backingMap.remove(keyEncoder.apply(key)).thenApply(versionedValueTransform);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Void> clear() {
-        return backingMap.clear();
-    }
-
-    @Override
-    public CompletableFuture<Set<K1>> keySet() {
-        return backingMap.keySet()
-                         .thenApply(s -> s.stream().map(keyDecoder).collect(Collectors.toSet()));
-    }
-
-    @Override
-    public CompletableFuture<Collection<Versioned<V1>>> values() {
-        return backingMap.values()
-                .thenApply(c -> c.stream().map(versionedValueTransform).collect(Collectors.toList()));
-    }
-
-    @Override
-    public CompletableFuture<Set<Entry<K1, Versioned<V1>>>> entrySet() {
-        return backingMap.entrySet()
-                         .thenApply(s -> s.stream()
-                                          .map(e -> Maps.immutableEntry(keyDecoder.apply(e.getKey()),
-                                                                        versionedValueTransform.apply(e.getValue())))
-                                          .collect(Collectors.toSet()));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V1>> putIfAbsent(K1 key, V1 value) {
-        try {
-            return backingMap.putIfAbsent(keyEncoder.apply(key), valueEncoder.apply(value))
-                             .thenApply(versionedValueTransform);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(K1 key, V1 value) {
-        try {
-            return backingMap.remove(keyEncoder.apply(key), valueEncoder.apply(value));
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(K1 key, long version) {
-        try {
-            return backingMap.remove(keyEncoder.apply(key), version);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V1>> replace(K1 key, V1 value) {
-        try {
-            return backingMap.replace(keyEncoder.apply(key), valueEncoder.apply(value))
-                             .thenApply(versionedValueTransform);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(K1 key, V1 oldValue, V1 newValue) {
-        try {
-            return backingMap.replace(keyEncoder.apply(key),
-                                      valueEncoder.apply(oldValue),
-                                      valueEncoder.apply(newValue));
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(K1 key, long oldVersion, V1 newValue) {
-        try {
-            return backingMap.replace(keyEncoder.apply(key), oldVersion, valueEncoder.apply(newValue));
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<AsyncIterator<Entry<K1, Versioned<V1>>>> iterator() {
-        return backingMap.iterator().thenApply(TranscodingIterator::new);
-    }
-
-    @Override
-    public CompletableFuture<Void> addListener(MapEventListener<K1, V1> listener, Executor executor) {
-        synchronized (listeners) {
-            InternalBackingMapEventListener backingMapListener =
-                    listeners.computeIfAbsent(listener, k -> new InternalBackingMapEventListener(listener));
-            return backingMap.addListener(backingMapListener, executor);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(MapEventListener<K1, V1> listener) {
-        synchronized (listeners) {
-            InternalBackingMapEventListener backingMapListener = listeners.remove(listener);
-            if (backingMapListener != null) {
-                return backingMap.removeListener(backingMapListener);
-            } else {
-                return CompletableFuture.completedFuture(null);
-            }
-        }
-    }
-
-    @Override
-    public CompletableFuture<Version> begin(TransactionId transactionId) {
-        try {
-            return backingMap.begin(transactionId);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepare(TransactionLog<MapUpdate<K1, V1>> transactionLog) {
-        try {
-            return backingMap.prepare(transactionLog.map(record -> record.map(keyEncoder, valueEncoder)));
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<MapUpdate<K1, V1>> transactionLog) {
-        try {
-            return backingMap.prepareAndCommit(transactionLog.map(record -> record.map(keyEncoder, valueEncoder)));
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Void> commit(TransactionId transactionId) {
-        try {
-            return backingMap.commit(transactionId);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Void> rollback(TransactionId transactionId) {
-        try {
-            return backingMap.rollback(transactionId);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public void addStatusChangeListener(Consumer<Status> listener) {
-        backingMap.addStatusChangeListener(listener);
-    }
-
-    @Override
-    public void removeStatusChangeListener(Consumer<Status> listener) {
-        backingMap.removeStatusChangeListener(listener);
-    }
-
-    @Override
-    public Collection<Consumer<Status>> statusChangeListeners() {
-        return backingMap.statusChangeListeners();
-    }
-
-    private class TranscodingIterator implements AsyncIterator<Entry<K1, Versioned<V1>>> {
-        private final AsyncIterator<Map.Entry<K2, Versioned<V2>>> iterator;
-
-        public TranscodingIterator(AsyncIterator<Map.Entry<K2, Versioned<V2>>> iterator) {
-            this.iterator = iterator;
-        }
-
-        @Override
-        public CompletableFuture<Boolean> hasNext() {
-            return iterator.hasNext();
-        }
-
-        @Override
-        public CompletableFuture<Map.Entry<K1, Versioned<V1>>> next() {
-            return iterator.next().thenApply(entry ->
-                Maps.immutableEntry(keyDecoder.apply(entry.getKey()), entry.getValue().map(valueDecoder)));
-        }
-    }
-
-    private class InternalBackingMapEventListener implements MapEventListener<K2, V2> {
-
-        private final MapEventListener<K1, V1> listener;
-
-        InternalBackingMapEventListener(MapEventListener<K1, V1> listener) {
-            this.listener = listener;
-        }
-
-        @Override
-        public void event(MapEvent<K2, V2> event) {
-            listener.event(new MapEvent<K1, V1>(
-                    event.type(),
-                    event.name(),
-                    keyDecoder.apply(event.key()),
-                    event.newValue() != null ? event.newValue().map(valueDecoder) : null,
-                    event.oldValue() != null ? event.oldValue().map(valueDecoder) : null));
-        }
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentMultimap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentMultimap.java
deleted file mode 100644
index 6fab8c8..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentMultimap.java
+++ /dev/null
@@ -1,381 +0,0 @@
-/*
- * Copyright 2016 Open Networking Foundation
- *
- * 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.Collection;
-import java.util.EnumSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.BiConsumer;
-import java.util.function.BinaryOperator;
-import java.util.function.Consumer;
-import java.util.function.Function;
-import java.util.function.Supplier;
-import java.util.stream.Collector;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.ImmutableMultiset;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Multiset;
-import org.onlab.util.Tools;
-import org.onosproject.store.service.AsyncConsistentMultimap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MultimapEvent;
-import org.onosproject.store.service.MultimapEventListener;
-import org.onosproject.store.service.Versioned;
-
-/**
- * An {@link AsyncConsistentMultimap} that maps its operation to operations to
- * a differently typed {@link AsyncConsistentMultimap} by transcoding operation
- * inputs and outputs while maintaining version numbers.
- *
- * @param <K2> key type of other map
- * @param <V2> value type of other map
- * @param <K1> key type of this map
- * @param <V1> value type of this map
- */
-public class TranscodingAsyncConsistentMultimap<K1, V1, K2, V2>
-        implements AsyncConsistentMultimap<K1, V1> {
-
-    private final AsyncConsistentMultimap<K2, V2> backingMap;
-    private final Function<K1, K2> keyEncoder;
-    private final Function<K2, K1> keyDecoder;
-    private final Function<V2, V1> valueDecoder;
-    private final Function<V1, V2> valueEncoder;
-    private final Function<? extends Versioned<V2>,
-            ? extends Versioned<V1>> versionedValueTransform;
-    private final Function<Versioned<Collection<? extends V2>>,
-            Versioned<Collection<? extends V1>>> versionedValueCollectionDecode;
-    private final Function<Collection<? extends V1>, Collection<V2>>
-            valueCollectionEncode;
-    private final Map<MultimapEventListener<K1, V1>, InternalBackingMultimapEventListener> listeners =
-            Maps.newIdentityHashMap();
-
-     public TranscodingAsyncConsistentMultimap(
-            AsyncConsistentMultimap<K2, V2> backingMap,
-            Function<K1, K2> keyEncoder,
-            Function<K2, K1> keyDecoder,
-            Function<V2, V1> valueDecoder,
-            Function<V1, V2> valueEncoder) {
-        this.backingMap = backingMap;
-        this.keyEncoder = k -> k == null ? null : keyEncoder.apply(k);
-        this.keyDecoder = k -> k == null ? null : keyDecoder.apply(k);
-        this.valueDecoder = v -> v == null ? null : valueDecoder.apply(v);
-        this.valueEncoder = v -> v == null ? null : valueEncoder.apply(v);
-        this.versionedValueTransform = v -> v == null ? null :
-                v.map(valueDecoder);
-        this.versionedValueCollectionDecode = v -> v == null ? null :
-                new Versioned<>(
-                        v.value()
-                                .stream()
-                                .map(valueDecoder)
-                                .collect(Collectors.toSet()),
-                        v.version(),
-                        v.creationTime());
-         this.valueCollectionEncode = v -> v == null ? null :
-                 v.stream().map(valueEncoder).collect(Collectors.toSet());
-    }
-
-    @Override
-    public CompletableFuture<Integer> size() {
-        return backingMap.size();
-    }
-
-    @Override
-    public CompletableFuture<Boolean> isEmpty() {
-        return backingMap.isEmpty();
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsKey(K1 key) {
-        try {
-            return backingMap.containsKey(keyEncoder.apply(key));
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsValue(V1 value) {
-        try {
-            return backingMap.containsValue(valueEncoder.apply(value));
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsEntry(K1 key, V1 value) {
-        try {
-            return backingMap.containsEntry(keyEncoder.apply(key),
-                                            valueEncoder.apply(value));
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Boolean> put(K1 key, V1 value) {
-        try {
-            return backingMap.put(keyEncoder.apply(key),
-                                  valueEncoder.apply(value));
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends V1>>> putAndGet(K1 key, V1 value) {
-        try {
-            return backingMap.putAndGet(keyEncoder.apply(key), valueEncoder.apply(value))
-                .thenApply(versionedValueCollectionDecode);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(K1 key, V1 value) {
-        try {
-            return backingMap.remove(keyEncoder.apply(key), valueEncoder
-                    .apply(value));
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends V1>>> removeAndGet(K1 key, V1 value) {
-        try {
-            return backingMap.removeAndGet(keyEncoder.apply(key), valueEncoder.apply(value))
-                .thenApply(versionedValueCollectionDecode);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Boolean> removeAll(
-            K1 key, Collection<? extends V1> values) {
-        try {
-            return backingMap.removeAll(
-                    keyEncoder.apply(key),
-                    values.stream().map(valueEncoder).collect(
-                            Collectors.toSet()));
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends V1>>>
-    removeAll(K1 key) {
-        try {
-            return backingMap.removeAll(keyEncoder.apply(key))
-                    .thenApply(versionedValueCollectionDecode);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Boolean>
-    putAll(K1 key, Collection<? extends V1> values) {
-        try {
-            return backingMap.putAll(keyEncoder.apply(key),
-                                     valueCollectionEncode.apply(values));
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends V1>>>
-    replaceValues(K1 key, Collection<V1> values) {
-        try {
-            return backingMap.replaceValues(keyEncoder.apply(key),
-                                     valueCollectionEncode.apply(values))
-                    .thenApply(versionedValueCollectionDecode);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Void> clear() {
-        return backingMap.clear();
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends V1>>> get(K1 key) {
-        try {
-            return backingMap.get(keyEncoder.apply(key))
-                    .thenApply(versionedValueCollectionDecode);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Set<K1>> keySet() {
-        return backingMap.keySet().thenApply(s -> s.stream()
-                .map(keyDecoder)
-                .collect(Collectors.toSet()));
-    }
-
-    @Override
-    public CompletableFuture<Multiset<K1>> keys() {
-        return backingMap.keys().thenApply(s -> s.stream().map(keyDecoder)
-                .collect(new MultisetCollector<>()));
-    }
-
-    @Override
-    public CompletableFuture<Multiset<V1>> values() {
-        return backingMap.values().thenApply(s ->
-            s.stream().map(valueDecoder).collect(new MultisetCollector<>()));
-    }
-
-    @Override
-    public CompletableFuture<Collection<Map.Entry<K1, V1>>> entries() {
-        return backingMap.entries().thenApply(s -> s.stream()
-        .map(e -> Maps.immutableEntry(keyDecoder.apply(e.getKey()),
-                                      valueDecoder.apply(e.getValue())))
-                .collect(Collectors.toSet()));
-    }
-
-    @Override
-    public CompletableFuture<AsyncIterator<Map.Entry<K1, V1>>> iterator() {
-        return backingMap.iterator().thenApply(TranscodingIterator::new);
-    }
-
-    @Override
-    public CompletableFuture<Map<K1, Collection<V1>>> asMap() {
-        throw new UnsupportedOperationException("Unsupported operation.");
-    }
-
-    @Override
-    public String name() {
-        return backingMap.name();
-    }
-
-    @Override
-    public CompletableFuture<Void> addListener(MultimapEventListener<K1, V1> listener, Executor executor) {
-        synchronized (listeners) {
-            InternalBackingMultimapEventListener backingMapListener =
-                    listeners.computeIfAbsent(listener, k -> new InternalBackingMultimapEventListener(listener));
-            return backingMap.addListener(backingMapListener, executor);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(MultimapEventListener<K1, V1> listener) {
-        synchronized (listeners) {
-            InternalBackingMultimapEventListener backingMapListener = listeners.remove(listener);
-            if (backingMapListener != null) {
-                return backingMap.removeListener(backingMapListener);
-            } else {
-                return CompletableFuture.completedFuture(null);
-            }
-        }
-    }
-
-    @Override
-    public void addStatusChangeListener(Consumer<Status> listener) {
-        backingMap.addStatusChangeListener(listener);
-    }
-
-    @Override
-    public void removeStatusChangeListener(Consumer<Status> listener) {
-        backingMap.removeStatusChangeListener(listener);
-    }
-
-    @Override
-    public Collection<Consumer<Status>> statusChangeListeners() {
-        return backingMap.statusChangeListeners();
-    }
-
-    private class MultisetCollector<T> implements Collector<T,
-            ImmutableMultiset.Builder<T>,
-            Multiset<T>> {
-
-        @Override
-        public Supplier<ImmutableMultiset.Builder<T>> supplier() {
-            return ImmutableMultiset::builder;
-        }
-
-        @Override
-        public BiConsumer<ImmutableMultiset.Builder<T>, T> accumulator() {
-            return ((builder, t) -> builder.add(t));
-        }
-
-        @Override
-        public BinaryOperator<ImmutableMultiset.Builder<T>> combiner() {
-            return (a, b) -> {
-                a.addAll(b.build());
-                return a;
-            };
-        }
-
-        @Override
-        public Function<ImmutableMultiset.Builder<T>, Multiset<T>> finisher() {
-            return ImmutableMultiset.Builder::build;
-        }
-
-        @Override
-        public Set<Characteristics> characteristics() {
-            return EnumSet.of(Characteristics.UNORDERED);
-        }
-    }
-
-    private class TranscodingIterator implements AsyncIterator<Map.Entry<K1, V1>> {
-        private final AsyncIterator<Map.Entry<K2, V2>> iterator;
-
-        public TranscodingIterator(AsyncIterator<Map.Entry<K2, V2>> iterator) {
-            this.iterator = iterator;
-        }
-
-        @Override
-        public CompletableFuture<Boolean> hasNext() {
-            return iterator.hasNext();
-        }
-
-        @Override
-        public CompletableFuture<Map.Entry<K1, V1>> next() {
-            return iterator.next().thenApply(entry ->
-                Maps.immutableEntry(keyDecoder.apply(entry.getKey()), valueDecoder.apply(entry.getValue())));
-        }
-    }
-
-    private class InternalBackingMultimapEventListener implements MultimapEventListener<K2, V2> {
-
-        private final MultimapEventListener<K1, V1> listener;
-
-        InternalBackingMultimapEventListener(MultimapEventListener<K1, V1> listener) {
-            this.listener = listener;
-        }
-
-        @Override
-        public void event(MultimapEvent<K2, V2> event) {
-            listener.event(new MultimapEvent(event.name(),
-                    keyDecoder.apply(event.key()),
-                    valueDecoder.apply(event.newValue()),
-                    valueDecoder.apply(event.oldValue())));
-        }
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentTreeMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentTreeMap.java
deleted file mode 100644
index d8e686e..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/TranscodingAsyncConsistentTreeMap.java
+++ /dev/null
@@ -1,437 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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 com.google.common.collect.Maps;
-import org.onlab.util.Tools;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.AsyncConsistentTreeMap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MapEvent;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.NavigableSet;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-import java.util.function.Predicate;
-import java.util.stream.Collectors;
-
-/**
- * An {@code AsyncConsistentTreeMap} that maps its operations to operations on
- * a differently typed {@code AsyncConsistentTreeMap} by transcoding operation
- * inputs and outputs.
- *
- * @param <V2> value type of other map
- * @param <V1> value type of this map
- */
-public class TranscodingAsyncConsistentTreeMap<V1, V2>
-        implements AsyncConsistentTreeMap<V1> {
-    private final AsyncConsistentTreeMap<V2> backingMap;
-    private final Function<V2, V1> valueDecoder;
-    private final Function<V1, V2> valueEncoder;
-    private final Function<Versioned<V2>, Versioned<V1>>
-            versionedValueTransform;
-    private final Map<MapEventListener<String, V1>,
-            TranscodingAsyncConsistentTreeMap.InternalBackingMapEventListener>
-            listeners = Maps.newIdentityHashMap();
-
-    public TranscodingAsyncConsistentTreeMap(
-            AsyncConsistentTreeMap<V2> backingMap,
-            Function<V1, V2> valueEncoder,
-            Function<V2, V1> valueDecoder) {
-        this.backingMap = backingMap;
-        this.valueEncoder = v -> v == null ? null : valueEncoder.apply(v);
-        this.valueDecoder = v -> v == null ? null : valueDecoder.apply(v);
-        this.versionedValueTransform = v -> v == null ? null :
-                v.map(valueDecoder);
-    }
-
-    @Override
-    public CompletableFuture<String> firstKey() {
-        return backingMap.firstKey();
-    }
-
-    @Override
-    public CompletableFuture<String> lastKey() {
-        return backingMap.lastKey();
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<V1>>>
-    ceilingEntry(String key) {
-        return backingMap.ceilingEntry(key)
-                .thenApply(
-                        entry ->
-                                Maps.immutableEntry(
-                                        entry.getKey(),
-                                        versionedValueTransform
-                                                .apply(entry.getValue())));
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<V1>>>
-    floorEntry(String key) {
-        return backingMap.floorEntry(key)
-                .thenApply(
-                        entry ->
-                                Maps.immutableEntry(
-                                        entry.getKey(),
-                                        versionedValueTransform
-                                                .apply(entry.getValue())));
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<V1>>>
-    higherEntry(String key) {
-        return backingMap
-                .higherEntry(key)
-                .thenApply(entry ->
-                                   Maps.immutableEntry(
-                                           entry.getKey(),
-                                           versionedValueTransform
-                                                   .apply(entry.getValue())));
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<V1>>>
-    lowerEntry(String key) {
-        return backingMap.lowerEntry(key).thenApply(
-                entry ->
-                        Maps.immutableEntry(
-                                entry.getKey(),
-                                versionedValueTransform
-                                        .apply(entry.getValue())));
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<V1>>>
-    firstEntry() {
-        return backingMap.firstEntry()
-                .thenApply(entry ->
-                                   Maps.immutableEntry(
-                                           entry.getKey(),
-                                           versionedValueTransform
-                                                   .apply(entry.getValue())));
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<V1>>>
-    lastEntry() {
-        return backingMap.lastEntry()
-                .thenApply(
-                        entry -> Maps.immutableEntry(
-                                entry.getKey(),
-                                versionedValueTransform
-                                        .apply(entry.getValue())));
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<V1>>>
-    pollFirstEntry() {
-        return backingMap.pollFirstEntry()
-                .thenApply(
-                        entry -> Maps.immutableEntry(
-                                entry.getKey(),
-                                versionedValueTransform
-                                        .apply(entry.getValue())));
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<V1>>>
-    pollLastEntry() {
-        return backingMap.pollLastEntry()
-                .thenApply(entry -> Maps.immutableEntry(
-                        entry.getKey(),
-                        versionedValueTransform.apply(entry.getValue())));
-    }
-
-    @Override
-    public CompletableFuture<String> lowerKey(String key) {
-        return backingMap.lowerKey(key);
-    }
-
-    @Override
-    public CompletableFuture<String> floorKey(String key) {
-        return backingMap.floorKey(key);
-    }
-
-    @Override
-    public CompletableFuture<String> ceilingKey(String key) {
-        return backingMap.ceilingKey(key);
-    }
-
-    @Override
-    public CompletableFuture<String> higherKey(String key) {
-        return backingMap.higherKey(key);
-    }
-
-    @Override
-    public CompletableFuture<NavigableSet<String>> navigableKeySet() {
-        return backingMap.navigableKeySet();
-    }
-
-    @Override
-    public CompletableFuture<NavigableMap<String, V1>> subMap(
-            String upperKey,
-            String lowerKey,
-            boolean inclusiveUpper,
-            boolean inclusiveLower) {
-        throw new UnsupportedOperationException("This operation is not yet" +
-                                                        "supported.");
-    }
-
-    @Override
-    public String name() {
-        return backingMap.name();
-    }
-
-    @Override
-    public CompletableFuture<Integer> size() {
-        return backingMap.size();
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsKey(String key) {
-        return backingMap.containsKey(key);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsValue(V1 value) {
-        return backingMap.containsValue(valueEncoder.apply(value));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V1>> get(String key) {
-        return backingMap.get(key).thenApply(versionedValueTransform);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V1>> getOrDefault(String key, V1 defaultValue) {
-        return backingMap.getOrDefault(key, valueEncoder.apply(defaultValue)).thenApply(versionedValueTransform);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V1>> computeIf(
-            String key, Predicate<? super V1> condition,
-            BiFunction<? super String, ? super V1, ? extends V1>
-                    remappingFunction) {
-        try {
-            return backingMap
-                    .computeIf(
-                            key,
-                            v -> condition.test(valueDecoder.apply(v)),
-                            (k, v) -> valueEncoder
-                                    .apply(
-                                            remappingFunction.apply(
-                                                    key,
-                                                    valueDecoder.apply(v))))
-                    .thenApply(versionedValueTransform);
-        } catch (Exception e) {
-            return Tools.exceptionalFuture(e);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V1>> put(String key, V1 value) {
-        return backingMap.put(key, valueEncoder.apply(value))
-                .thenApply(versionedValueTransform);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V1>> putAndGet(String key, V1 value) {
-        return backingMap.putAndGet(key, valueEncoder.apply(value))
-                .thenApply(versionedValueTransform);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V1>> remove(String key) {
-        return backingMap.remove(key).thenApply(versionedValueTransform);
-    }
-
-    @Override
-    public CompletableFuture<Void> clear() {
-        return backingMap.clear();
-    }
-
-    @Override
-    public CompletableFuture<Set<String>> keySet() {
-        return backingMap.keySet();
-    }
-
-    @Override
-    public CompletableFuture<Collection<Versioned<V1>>> values() {
-        return backingMap.values().thenApply(valueSet -> valueSet.stream()
-                .map(versionedValueTransform).collect(Collectors.toSet()));
-    }
-
-    @Override
-    public CompletableFuture<Set<Map.Entry<String, Versioned<V1>>>>
-    entrySet() {
-        return backingMap.entrySet()
-                .thenApply(
-                        entries -> entries
-                                .stream()
-                                .map(entry ->
-                                             Maps.immutableEntry(
-                                                 entry.getKey(),
-                                                 versionedValueTransform
-                                                     .apply(entry.getValue())
-                                             ))
-                                .collect(Collectors.toSet()));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V1>> putIfAbsent(String key, V1 value) {
-        return backingMap.putIfAbsent(key, valueEncoder.apply(value))
-                .thenApply(versionedValueTransform);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(String key, V1 value) {
-        return backingMap.remove(key, valueEncoder.apply(value));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(String key, long version) {
-        return backingMap.remove(key, version);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V1>> replace(String key, V1 value) {
-        return backingMap.replace(key, valueEncoder.apply(value))
-                .thenApply(versionedValueTransform);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(String key, V1 oldValue,
-                                              V1 newValue) {
-        return backingMap.replace(key, valueEncoder.apply(oldValue),
-                                  valueEncoder.apply(newValue));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(String key, long oldVersion,
-                                              V1 newValue) {
-        return backingMap.replace(key, oldVersion,
-                                  valueEncoder.apply(newValue));
-    }
-
-    @Override
-    public CompletableFuture<AsyncIterator<Map.Entry<String, Versioned<V1>>>> iterator() {
-        return backingMap.iterator().thenApply(TranscodingIterator::new);
-    }
-
-    @Override
-    public CompletableFuture<Void> addListener(
-            MapEventListener<String, V1> listener,
-            Executor executor) {
-        InternalBackingMapEventListener backingMapEventListener =
-                listeners.computeIfAbsent(
-                        listener,
-                        k -> new InternalBackingMapEventListener(listener));
-        return backingMap.addListener(backingMapEventListener, executor);
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(
-            MapEventListener<String, V1> listener) {
-        InternalBackingMapEventListener backingMapEventListener =
-                listeners.remove(listener);
-        if (backingMapEventListener == null) {
-            return CompletableFuture.completedFuture(null);
-        } else {
-            return backingMap.removeListener(backingMapEventListener);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Version> begin(TransactionId transactionId) {
-        throw new UnsupportedOperationException("This operation is not yet supported.");
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepare(TransactionLog<MapUpdate<String, V1>> transactionLog) {
-        throw new UnsupportedOperationException("This operation is not yet supported.");
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<MapUpdate<String, V1>> transactionLog) {
-        throw new UnsupportedOperationException("This operation is not yet supported.");
-    }
-
-    @Override
-    public CompletableFuture<Void> commit(TransactionId transactionId) {
-        throw new UnsupportedOperationException("This operation is not yet supported.");
-    }
-
-    @Override
-    public CompletableFuture<Void> rollback(TransactionId transactionId) {
-        throw new UnsupportedOperationException("This operation is not yet supported.");
-    }
-
-    private class TranscodingIterator implements AsyncIterator<Map.Entry<String, Versioned<V1>>> {
-        private final AsyncIterator<Map.Entry<String, Versioned<V2>>> iterator;
-
-        public TranscodingIterator(AsyncIterator<Map.Entry<String, Versioned<V2>>> iterator) {
-            this.iterator = iterator;
-        }
-
-        @Override
-        public CompletableFuture<Boolean> hasNext() {
-            return iterator.hasNext();
-        }
-
-        @Override
-        public CompletableFuture<Map.Entry<String, Versioned<V1>>> next() {
-            return iterator.next().thenApply(entry ->
-                Maps.immutableEntry(entry.getKey(), entry.getValue().map(valueDecoder)));
-        }
-    }
-
-    private class InternalBackingMapEventListener
-            implements MapEventListener<String, V2> {
-
-        private final MapEventListener<String, V1> listener;
-
-        InternalBackingMapEventListener(
-                MapEventListener<String, V1> listener) {
-            this.listener = listener;
-        }
-
-        @Override
-        public void event(MapEvent<String, V2> event) {
-            listener.event(new MapEvent<String, V1>(
-                    event.type(),
-                    event.name(),
-                    event.key(),
-                    event.newValue() != null ?
-                            event.newValue().map(valueDecoder) : null,
-                    event.oldValue() != null ?
-                            event.oldValue().map(valueDecoder) : null));
-        }
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UnmodifiableAsyncConsistentMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UnmodifiableAsyncConsistentMap.java
deleted file mode 100644
index 65378b9..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UnmodifiableAsyncConsistentMap.java
+++ /dev/null
@@ -1,100 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.concurrent.CompletableFuture;
-import java.util.function.BiFunction;
-import java.util.function.Predicate;
-
-import org.onlab.util.Tools;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.Versioned;
-
-/**
- * An unmodifiable {@link AsyncConsistentMap}.
- * <p>
- * Any attempt to update the map through this instance will cause the
- * operation to be completed with an {@link UnsupportedOperationException}.
- *
- * @param <K> key type
- * @param <V> value type
- */
-public class UnmodifiableAsyncConsistentMap<K, V> extends DelegatingAsyncConsistentMap<K, V> {
-
-    private static final String ERROR_MSG = "map updates are not allowed";
-
-    public UnmodifiableAsyncConsistentMap(AsyncConsistentMap<K, V> backingMap) {
-        super(backingMap);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> computeIf(K key,
-            Predicate<? super V> condition,
-            BiFunction<? super K, ? super V, ? extends V> remappingFunction) {
-        return Tools.exceptionalFuture(new UnsupportedOperationException(""));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> put(K key, V value) {
-        return Tools.exceptionalFuture(new UnsupportedOperationException(ERROR_MSG));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> putAndGet(K key, V value) {
-        return Tools.exceptionalFuture(new UnsupportedOperationException(ERROR_MSG));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> remove(K key) {
-        return Tools.exceptionalFuture(new UnsupportedOperationException(ERROR_MSG));
-    }
-
-    @Override
-    public CompletableFuture<Void> clear() {
-        return Tools.exceptionalFuture(new UnsupportedOperationException(ERROR_MSG));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> putIfAbsent(K key, V value) {
-        return Tools.exceptionalFuture(new UnsupportedOperationException(ERROR_MSG));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(K key, V value) {
-        return Tools.exceptionalFuture(new UnsupportedOperationException(ERROR_MSG));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(K key, long version) {
-        return Tools.exceptionalFuture(new UnsupportedOperationException(ERROR_MSG));
-    }
-
-    @Override
-    public CompletableFuture<Versioned<V>> replace(K key, V value) {
-        return Tools.exceptionalFuture(new UnsupportedOperationException(ERROR_MSG));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(K key, V oldValue, V newValue) {
-        return Tools.exceptionalFuture(new UnsupportedOperationException(ERROR_MSG));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(K key, long oldVersion, V newValue) {
-        return Tools.exceptionalFuture(new UnsupportedOperationException(ERROR_MSG));
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UpdateEntry.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UpdateEntry.java
index 42bf1f9..680795c 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UpdateEntry.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UpdateEntry.java
@@ -15,10 +15,10 @@
  */
 package org.onosproject.store.primitives.impl;
 
-import static com.google.common.base.Preconditions.checkNotNull;
-
 import com.google.common.base.MoreObjects;
 
+import static com.google.common.base.Preconditions.checkNotNull;
+
 /**
  * Describes a single update event in an EventuallyConsistentMap.
  */
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UpdateRequest.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UpdateRequest.java
index 7563469..cb3db2f 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UpdateRequest.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/UpdateRequest.java
@@ -15,12 +15,12 @@
  */
 package org.onosproject.store.primitives.impl;
 
+import java.util.Set;
+
 import com.google.common.base.MoreObjects;
 import com.google.common.collect.ImmutableSet;
 import org.onosproject.cluster.NodeId;
 
-import java.util.Set;
-
 import static com.google.common.base.Preconditions.checkNotNull;
 
 /**
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/package-info.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/package-info.java
index 21824a7..e151e54 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/package-info.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/impl/package-info.java
@@ -15,6 +15,6 @@
  */
 
 /**
- * Implementation classes for various Distributed primitives.
+ * Implementation classes for store utilities.
  */
 package org.onosproject.store.primitives.impl;
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AbstractRaftPrimitive.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AbstractRaftPrimitive.java
deleted file mode 100644
index 17b8ef8..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AbstractRaftPrimitive.java
+++ /dev/null
@@ -1,91 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.Collection;
-import java.util.Set;
-import java.util.function.Consumer;
-import java.util.function.Function;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import org.onosproject.store.service.DistributedPrimitive;
-
-import static com.google.common.base.MoreObjects.toStringHelper;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * Abstract base class for primitives that interact with Raft replicated state machines via proxy.
- */
-public abstract class AbstractRaftPrimitive implements DistributedPrimitive {
-    private final Function<RaftProxy.State, Status> mapper = state -> {
-        switch (state) {
-            case CONNECTED:
-                return Status.ACTIVE;
-            case SUSPENDED:
-                return Status.SUSPENDED;
-            case CLOSED:
-                return Status.INACTIVE;
-            default:
-                throw new IllegalStateException("Unknown state " + state);
-        }
-    };
-
-    protected final RaftProxy proxy;
-    private final Set<Consumer<Status>> statusChangeListeners = Sets.newCopyOnWriteArraySet();
-
-    public AbstractRaftPrimitive(RaftProxy proxy) {
-        this.proxy = checkNotNull(proxy, "proxy cannot be null");
-        proxy.addStateChangeListener(this::onStateChange);
-    }
-
-    @Override
-    public String name() {
-        return proxy.name();
-    }
-
-    /**
-     * Handles a Raft session state change.
-     *
-     * @param state the updated Raft session state
-     */
-    private void onStateChange(RaftProxy.State state) {
-        statusChangeListeners.forEach(listener -> listener.accept(mapper.apply(state)));
-    }
-
-    @Override
-    public void addStatusChangeListener(Consumer<Status> listener) {
-        statusChangeListeners.add(listener);
-    }
-
-    @Override
-    public void removeStatusChangeListener(Consumer<Status> listener) {
-        statusChangeListeners.remove(listener);
-    }
-
-    @Override
-    public Collection<Consumer<Status>> statusChangeListeners() {
-        return ImmutableSet.copyOf(statusChangeListeners);
-    }
-
-    @Override
-    public String toString() {
-        return toStringHelper(this)
-                .add("proxy", proxy)
-                .toString();
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMap.java
deleted file mode 100644
index 9d25e26..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMap.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.concurrent.CompletableFuture;
-
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.AddAndGet;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.DecrementAndGet;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.Get;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GetAndAdd;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GetAndDecrement;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GetAndIncrement;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.IncrementAndGet;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.Put;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.PutIfAbsent;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.Remove;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.RemoveValue;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.Replace;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncAtomicCounterMap;
-import org.onosproject.store.service.Serializer;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.ADD_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.CLEAR;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.DECREMENT_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GET_AND_ADD;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GET_AND_DECREMENT;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GET_AND_INCREMENT;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.INCREMENT_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.IS_EMPTY;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.PUT;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.PUT_IF_ABSENT;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.REMOVE;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.REMOVE_VALUE;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.REPLACE;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.SIZE;
-
-/**
- * {@code AsyncAtomicCounterMap} implementation backed by Atomix.
- */
-public class AtomixAtomicCounterMap extends AbstractRaftPrimitive implements AsyncAtomicCounterMap<String> {
-    private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
-            .register(KryoNamespaces.BASIC)
-            .register(AtomixAtomicCounterMapOperations.NAMESPACE)
-            .build());
-
-    public AtomixAtomicCounterMap(RaftProxy proxy) {
-        super(proxy);
-    }
-
-    @Override
-    public CompletableFuture<Long> incrementAndGet(String key) {
-        return proxy.invoke(INCREMENT_AND_GET, SERIALIZER::encode, new IncrementAndGet(key), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Long> decrementAndGet(String key) {
-        return proxy.invoke(DECREMENT_AND_GET, SERIALIZER::encode, new DecrementAndGet(key), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Long> getAndIncrement(String key) {
-        return proxy.invoke(GET_AND_INCREMENT, SERIALIZER::encode, new GetAndIncrement(key), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Long> getAndDecrement(String key) {
-        return proxy.invoke(GET_AND_DECREMENT, SERIALIZER::encode, new GetAndDecrement(key), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Long> addAndGet(String key, long delta) {
-        return proxy.invoke(ADD_AND_GET, SERIALIZER::encode, new AddAndGet(key, delta), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Long> getAndAdd(String key, long delta) {
-        return proxy.invoke(GET_AND_ADD, SERIALIZER::encode, new GetAndAdd(key, delta), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Long> get(String key) {
-        return proxy.invoke(GET, SERIALIZER::encode, new Get(key), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Long> put(String key, long newValue) {
-        return proxy.invoke(PUT, SERIALIZER::encode, new Put(key, newValue), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Long> putIfAbsent(String key, long newValue) {
-        return proxy.invoke(PUT_IF_ABSENT, SERIALIZER::encode, new PutIfAbsent(key, newValue), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(String key, long expectedOldValue, long newValue) {
-        return proxy.invoke(
-                REPLACE,
-                SERIALIZER::encode,
-                new Replace(key, expectedOldValue, newValue),
-                SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Long> remove(String key) {
-        return proxy.invoke(REMOVE, SERIALIZER::encode, new Remove(key), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(String key, long value) {
-        return proxy.invoke(REMOVE_VALUE, SERIALIZER::encode, new RemoveValue(key, value), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Integer> size() {
-        return proxy.invoke(SIZE, SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> isEmpty() {
-        return proxy.invoke(IS_EMPTY, SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Void> clear() {
-        return proxy.invoke(CLEAR);
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapOperations.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapOperations.java
deleted file mode 100644
index 7b2cba2..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapOperations.java
+++ /dev/null
@@ -1,246 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import io.atomix.protocols.raft.operation.OperationId;
-import io.atomix.protocols.raft.operation.OperationType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-
-/**
- * Atomic counter map commands.
- */
-public enum AtomixAtomicCounterMapOperations implements OperationId {
-    PUT(OperationType.COMMAND),
-    PUT_IF_ABSENT(OperationType.COMMAND),
-    GET(OperationType.QUERY),
-    REPLACE(OperationType.COMMAND),
-    REMOVE(OperationType.COMMAND),
-    REMOVE_VALUE(OperationType.COMMAND),
-    GET_AND_INCREMENT(OperationType.COMMAND),
-    GET_AND_DECREMENT(OperationType.COMMAND),
-    INCREMENT_AND_GET(OperationType.COMMAND),
-    DECREMENT_AND_GET(OperationType.COMMAND),
-    ADD_AND_GET(OperationType.COMMAND),
-    GET_AND_ADD(OperationType.COMMAND),
-    SIZE(OperationType.QUERY),
-    IS_EMPTY(OperationType.QUERY),
-    CLEAR(OperationType.COMMAND);
-
-    private final OperationType type;
-
-    AtomixAtomicCounterMapOperations(OperationType type) {
-        this.type = type;
-    }
-
-    @Override
-    public String id() {
-        return name();
-    }
-
-    @Override
-    public OperationType type() {
-        return type;
-    }
-
-    public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
-            .register(KryoNamespaces.BASIC)
-            .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
-            .register(IncrementAndGet.class)
-            .register(DecrementAndGet.class)
-            .register(GetAndIncrement.class)
-            .register(GetAndDecrement.class)
-            .register(AddAndGet.class)
-            .register(GetAndAdd.class)
-            .register(Get.class)
-            .register(Put.class)
-            .register(PutIfAbsent.class)
-            .register(Replace.class)
-            .register(Remove.class)
-            .register(RemoveValue.class)
-            .build("AtomixAtomicCounterMapOperations");
-
-    public abstract static class AtomicCounterMapOperation<V> {
-    }
-
-    public abstract static class KeyOperation extends AtomicCounterMapOperation {
-        private String key;
-
-        public KeyOperation() {
-        }
-
-        public KeyOperation(String key) {
-            this.key = key;
-        }
-
-        public String key() {
-            return key;
-        }
-    }
-
-    public static class KeyValueOperation extends KeyOperation {
-        private long value;
-
-        public KeyValueOperation() {
-        }
-
-        public KeyValueOperation(String key, long value) {
-            super(key);
-            this.value = value;
-        }
-
-        public long value() {
-            return value;
-        }
-    }
-
-    public static class Get extends KeyOperation {
-        public Get() {
-        }
-
-        public Get(String key) {
-            super(key);
-        }
-    }
-
-    public static class Put extends KeyValueOperation {
-        public Put() {
-        }
-
-        public Put(String key, long value) {
-            super(key, value);
-        }
-    }
-
-    public static class PutIfAbsent extends KeyValueOperation {
-        public PutIfAbsent() {
-        }
-
-        public PutIfAbsent(String key, long value) {
-            super(key, value);
-        }
-    }
-
-    public static class Replace extends KeyOperation {
-        private long replace;
-        private long value;
-
-        public Replace() {
-        }
-
-        public Replace(String key, long replace, long value) {
-            super(key);
-            this.replace = replace;
-            this.value = value;
-        }
-
-        public long replace() {
-            return replace;
-        }
-
-        public long value() {
-            return value;
-        }
-    }
-
-    public static class Remove extends KeyOperation {
-        public Remove() {
-        }
-
-        public Remove(String key) {
-            super(key);
-        }
-    }
-
-    public static class RemoveValue extends KeyValueOperation {
-        public RemoveValue() {
-        }
-
-        public RemoveValue(String key, long value) {
-            super(key, value);
-        }
-    }
-
-    public static class IncrementAndGet extends KeyOperation {
-        public IncrementAndGet() {
-        }
-
-        public IncrementAndGet(String key) {
-            super(key);
-        }
-    }
-
-    public static class DecrementAndGet extends KeyOperation {
-        public DecrementAndGet(String key) {
-            super(key);
-        }
-
-        public DecrementAndGet() {
-        }
-    }
-
-    public static class GetAndIncrement extends KeyOperation {
-        public GetAndIncrement() {
-        }
-
-        public GetAndIncrement(String key) {
-            super(key);
-        }
-    }
-
-    public static class GetAndDecrement extends KeyOperation {
-        public GetAndDecrement() {
-        }
-
-        public GetAndDecrement(String key) {
-            super(key);
-        }
-    }
-
-    public abstract static class DeltaOperation extends KeyOperation {
-        private long delta;
-
-        public DeltaOperation() {
-        }
-
-        public DeltaOperation(String key, long delta) {
-            super(key);
-            this.delta = delta;
-        }
-
-        public long delta() {
-            return delta;
-        }
-    }
-
-    public static class AddAndGet extends DeltaOperation {
-        public AddAndGet() {
-        }
-
-        public AddAndGet(String key, long delta) {
-            super(key, delta);
-        }
-    }
-
-    public static class GetAndAdd extends DeltaOperation {
-        public GetAndAdd() {
-        }
-
-        public GetAndAdd(String key, long delta) {
-            super(key, delta);
-        }
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapService.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapService.java
deleted file mode 100644
index 354db26..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapService.java
+++ /dev/null
@@ -1,303 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.HashMap;
-import java.util.Map;
-
-import io.atomix.protocols.raft.service.AbstractRaftService;
-import io.atomix.protocols.raft.service.Commit;
-import io.atomix.protocols.raft.service.RaftServiceExecutor;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.AddAndGet;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.DecrementAndGet;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.Get;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GetAndAdd;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GetAndDecrement;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GetAndIncrement;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.IncrementAndGet;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.Put;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.PutIfAbsent;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.Remove;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.RemoveValue;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.Replace;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.Serializer;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.ADD_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.CLEAR;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.DECREMENT_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GET_AND_ADD;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GET_AND_DECREMENT;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.GET_AND_INCREMENT;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.INCREMENT_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.IS_EMPTY;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.PUT;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.PUT_IF_ABSENT;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.REMOVE;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.REMOVE_VALUE;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.REPLACE;
-import static org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapOperations.SIZE;
-
-/**
- * Atomic counter map state for Atomix.
- * <p>
- * The counter map state is implemented as a snapshottable state machine. Snapshots are necessary
- * since incremental compaction is impractical for counters where the value of a counter is the sum
- * of all its increments. Note that this snapshotting large state machines may risk blocking of the
- * Raft cluster with the current implementation of snapshotting in Copycat.
- */
-public class AtomixAtomicCounterMapService extends AbstractRaftService {
-
-    private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
-            .register(KryoNamespaces.BASIC)
-            .register(AtomixAtomicCounterMapOperations.NAMESPACE)
-            .build());
-
-    private Map<String, Long> map = new HashMap<>();
-
-    @Override
-    protected void configure(RaftServiceExecutor executor) {
-        executor.register(PUT, SERIALIZER::decode, this::put, SERIALIZER::encode);
-        executor.register(PUT_IF_ABSENT, SERIALIZER::decode, this::putIfAbsent, SERIALIZER::encode);
-        executor.register(GET, SERIALIZER::decode, this::get, SERIALIZER::encode);
-        executor.register(REPLACE, SERIALIZER::decode, this::replace, SERIALIZER::encode);
-        executor.register(REMOVE, SERIALIZER::decode, this::remove, SERIALIZER::encode);
-        executor.register(REMOVE_VALUE, SERIALIZER::decode, this::removeValue, SERIALIZER::encode);
-        executor.register(GET_AND_INCREMENT, SERIALIZER::decode, this::getAndIncrement, SERIALIZER::encode);
-        executor.register(GET_AND_DECREMENT, SERIALIZER::decode, this::getAndDecrement, SERIALIZER::encode);
-        executor.register(INCREMENT_AND_GET, SERIALIZER::decode, this::incrementAndGet, SERIALIZER::encode);
-        executor.register(DECREMENT_AND_GET, SERIALIZER::decode, this::decrementAndGet, SERIALIZER::encode);
-        executor.register(ADD_AND_GET, SERIALIZER::decode, this::addAndGet, SERIALIZER::encode);
-        executor.register(GET_AND_ADD, SERIALIZER::decode, this::getAndAdd, SERIALIZER::encode);
-        executor.register(SIZE, this::size, SERIALIZER::encode);
-        executor.register(IS_EMPTY, this::isEmpty, SERIALIZER::encode);
-        executor.register(CLEAR, this::clear);
-    }
-
-    @Override
-    public void snapshot(SnapshotWriter writer) {
-        writer.writeObject(map, SERIALIZER::encode);
-    }
-
-    @Override
-    public void install(SnapshotReader reader) {
-        map = reader.readObject(SERIALIZER::decode);
-    }
-
-    /**
-     * Returns the primitive value for the given primitive wrapper.
-     */
-    private long primitive(Long value) {
-        if (value != null) {
-            return value;
-        } else {
-            return 0;
-        }
-    }
-
-    /**
-     * Handles a {@link Put} command which implements {@link AtomixAtomicCounterMap#put(String, long)}.
-     *
-     * @param commit put commit
-     * @return put result
-     */
-    protected long put(Commit<Put> commit) {
-        return primitive(map.put(commit.value().key(), commit.value().value()));
-    }
-
-    /**
-     * Handles a {@link PutIfAbsent} command which implements {@link AtomixAtomicCounterMap#putIfAbsent(String, long)}.
-     *
-     * @param commit putIfAbsent commit
-     * @return putIfAbsent result
-     */
-    protected long putIfAbsent(Commit<PutIfAbsent> commit) {
-        return primitive(map.putIfAbsent(commit.value().key(), commit.value().value()));
-    }
-
-    /**
-     * Handles a {@link Get} query which implements {@link AtomixAtomicCounterMap#get(String)}}.
-     *
-     * @param commit get commit
-     * @return get result
-     */
-    protected long get(Commit<Get> commit) {
-        return primitive(map.get(commit.value().key()));
-    }
-
-    /**
-     * Handles a {@link Replace} command which implements {@link AtomixAtomicCounterMap#replace(String, long, long)}.
-     *
-     * @param commit replace commit
-     * @return replace result
-     */
-    protected boolean replace(Commit<Replace> commit) {
-        Long value = map.get(commit.value().key());
-        if (value == null) {
-            if (commit.value().replace() == 0) {
-                map.put(commit.value().key(), commit.value().value());
-                return true;
-            } else {
-                return false;
-            }
-        } else if (value == commit.value().replace()) {
-            map.put(commit.value().key(), commit.value().value());
-            return true;
-        }
-        return false;
-    }
-
-    /**
-     * Handles a {@link Remove} command which implements {@link AtomixAtomicCounterMap#remove(String)}.
-     *
-     * @param commit remove commit
-     * @return remove result
-     */
-    protected long remove(Commit<Remove> commit) {
-        return primitive(map.remove(commit.value().key()));
-    }
-
-    /**
-     * Handles a {@link RemoveValue} command which implements {@link AtomixAtomicCounterMap#remove(String, long)}.
-     *
-     * @param commit removeValue commit
-     * @return removeValue result
-     */
-    protected boolean removeValue(Commit<RemoveValue> commit) {
-        Long value = map.get(commit.value().key());
-        if (value == null) {
-            if (commit.value().value() == 0) {
-                map.remove(commit.value().key());
-                return true;
-            }
-            return false;
-        } else if (value == commit.value().value()) {
-            map.remove(commit.value().key());
-            return true;
-        }
-        return false;
-    }
-
-    /**
-     * Handles a {@link GetAndIncrement} command which implements
-     * {@link AtomixAtomicCounterMap#getAndIncrement(String)}.
-     *
-     * @param commit getAndIncrement commit
-     * @return getAndIncrement result
-     */
-    protected long getAndIncrement(Commit<GetAndIncrement> commit) {
-        long value = primitive(map.get(commit.value().key()));
-        map.put(commit.value().key(), value + 1);
-        return value;
-    }
-
-    /**
-     * Handles a {@link GetAndDecrement} command which implements
-     * {@link AtomixAtomicCounterMap#getAndDecrement(String)}.
-     *
-     * @param commit getAndDecrement commit
-     * @return getAndDecrement result
-     */
-    protected long getAndDecrement(Commit<GetAndDecrement> commit) {
-        long value = primitive(map.get(commit.value().key()));
-        map.put(commit.value().key(), value - 1);
-        return value;
-    }
-
-    /**
-     * Handles a {@link IncrementAndGet} command which implements
-     * {@link AtomixAtomicCounterMap#incrementAndGet(String)}.
-     *
-     * @param commit incrementAndGet commit
-     * @return incrementAndGet result
-     */
-    protected long incrementAndGet(Commit<IncrementAndGet> commit) {
-        long value = primitive(map.get(commit.value().key()));
-        map.put(commit.value().key(), ++value);
-        return value;
-    }
-
-    /**
-     * Handles a {@link DecrementAndGet} command which implements
-     * {@link AtomixAtomicCounterMap#decrementAndGet(String)}.
-     *
-     * @param commit decrementAndGet commit
-     * @return decrementAndGet result
-     */
-    protected long decrementAndGet(Commit<DecrementAndGet> commit) {
-        long value = primitive(map.get(commit.value().key()));
-        map.put(commit.value().key(), --value);
-        return value;
-    }
-
-    /**
-     * Handles a {@link AddAndGet} command which implements {@link AtomixAtomicCounterMap#addAndGet(String, long)}.
-     *
-     * @param commit addAndGet commit
-     * @return addAndGet result
-     */
-    protected long addAndGet(Commit<AddAndGet> commit) {
-        long value = primitive(map.get(commit.value().key()));
-        value += commit.value().delta();
-        map.put(commit.value().key(), value);
-        return value;
-    }
-
-    /**
-     * Handles a {@link GetAndAdd} command which implements {@link AtomixAtomicCounterMap#getAndAdd(String, long)}.
-     *
-     * @param commit getAndAdd commit
-     * @return getAndAdd result
-     */
-    protected long getAndAdd(Commit<GetAndAdd> commit) {
-        long value = primitive(map.get(commit.value().key()));
-        map.put(commit.value().key(), value + commit.value().delta());
-        return value;
-    }
-
-    /**
-     * Handles a {@code Size} query which implements {@link AtomixAtomicCounterMap#size()}.
-     *
-     * @param commit size commit
-     * @return size result
-     */
-    protected int size(Commit<Void> commit) {
-        return map.size();
-    }
-
-    /**
-     * Handles an {@code IsEmpty} query which implements {@link AtomixAtomicCounterMap#isEmpty()}.
-     *
-     * @param commit isEmpty commit
-     * @return isEmpty result
-     */
-    protected boolean isEmpty(Commit<Void> commit) {
-        return map.isEmpty();
-    }
-
-    /**
-     * Handles a {@code Clear} command which implements {@link AtomixAtomicCounterMap#clear()}.
-     *
-     * @param commit clear commit
-     */
-    protected void clear(Commit<Void> commit) {
-        map.clear();
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMap.java
deleted file mode 100644
index 0514333..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMap.java
+++ /dev/null
@@ -1,530 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.ConcurrentModificationException;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.NoSuchElementException;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executor;
-import java.util.function.BiFunction;
-import java.util.function.Function;
-import java.util.function.Predicate;
-
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Tools;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ContainsKey;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ContainsValue;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.Get;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.GetOrDefault;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.Put;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.Remove;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.RemoveValue;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.RemoveVersion;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.Replace;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ReplaceValue;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ReplaceVersion;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.TransactionBegin;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.TransactionCommit;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.TransactionPrepare;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.TransactionPrepareAndCommit;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.TransactionRollback;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncConsistentMap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.ConsistentMapException;
-import org.onosproject.store.service.MapEvent;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapEvents.CHANGE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ADD_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.BEGIN;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.CLEAR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.CLOSE_ITERATOR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.COMMIT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.CONTAINS_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.CONTAINS_VALUE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ENTRY_SET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.GET_OR_DEFAULT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.IS_EMPTY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.IteratorBatch;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.IteratorPosition;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.KEY_SET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.NEXT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.OPEN_ITERATOR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PREPARE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PREPARE_AND_COMMIT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PUT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PUT_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PUT_IF_ABSENT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REMOVE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REMOVE_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REMOVE_VALUE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REMOVE_VERSION;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REPLACE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REPLACE_VALUE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REPLACE_VERSION;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ROLLBACK;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.SIZE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.VALUES;
-
-/**
- * Distributed resource providing the {@link AsyncConsistentMap} primitive.
- */
-public class AtomixConsistentMap extends AbstractRaftPrimitive implements AsyncConsistentMap<String, byte[]> {
-    private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
-        .register(KryoNamespaces.BASIC)
-        .register(AtomixConsistentMapOperations.NAMESPACE)
-        .register(AtomixConsistentMapEvents.NAMESPACE)
-        .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 100)
-        .build());
-
-    private final Map<MapEventListener<String, byte[]>, Executor> mapEventListeners = new ConcurrentHashMap<>();
-
-    public AtomixConsistentMap(RaftProxy proxy) {
-        super(proxy);
-        proxy.addEventListener(CHANGE, SERIALIZER::decode, this::handleEvent);
-        proxy.addStateChangeListener(state -> {
-            if (state == RaftProxy.State.CONNECTED && isListening()) {
-                proxy.invoke(ADD_LISTENER);
-            }
-        });
-    }
-
-    protected Serializer serializer() {
-        return SERIALIZER;
-    }
-
-    private void handleEvent(List<MapEvent<String, byte[]>> events) {
-        events.forEach(event ->
-            mapEventListeners.forEach((listener, executor) -> executor.execute(() -> listener.event(event))));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> isEmpty() {
-        return proxy.invoke(IS_EMPTY, serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<Integer> size() {
-        return proxy.invoke(SIZE, serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsKey(String key) {
-        return proxy.invoke(CONTAINS_KEY, serializer()::encode, new ContainsKey(key), serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsValue(byte[] value) {
-        return proxy.invoke(CONTAINS_VALUE, serializer()::encode, new ContainsValue(value), serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<byte[]>> get(String key) {
-        return proxy.invoke(GET, serializer()::encode, new Get(key), serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<byte[]>> getOrDefault(String key, byte[] defaultValue) {
-        return proxy.invoke(
-            GET_OR_DEFAULT,
-            serializer()::encode,
-            new GetOrDefault(key, defaultValue),
-            serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<Set<String>> keySet() {
-        return proxy.invoke(KEY_SET, serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<Collection<Versioned<byte[]>>> values() {
-        return proxy.invoke(VALUES, serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<Set<Entry<String, Versioned<byte[]>>>> entrySet() {
-        return proxy.invoke(ENTRY_SET, serializer()::decode);
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public CompletableFuture<Versioned<byte[]>> put(String key, byte[] value) {
-        return proxy.<Put, MapEntryUpdateResult<String, byte[]>>invoke(
-            PUT,
-            serializer()::encode,
-            new Put(key, value),
-            serializer()::decode)
-            .whenComplete((r, e) -> throwIfLocked(r))
-            .thenApply(v -> v.result());
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public CompletableFuture<Versioned<byte[]>> putAndGet(String key, byte[] value) {
-        return proxy.<Put, MapEntryUpdateResult<String, byte[]>>invoke(
-            PUT_AND_GET,
-            serializer()::encode,
-            new Put(key, value),
-            serializer()::decode)
-            .whenComplete((r, e) -> throwIfLocked(r))
-            .thenApply(v -> v.result());
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public CompletableFuture<Versioned<byte[]>> putIfAbsent(String key, byte[] value) {
-        return proxy.<Put, MapEntryUpdateResult<String, byte[]>>invoke(
-            PUT_IF_ABSENT,
-            serializer()::encode,
-            new Put(key, value),
-            serializer()::decode)
-            .whenComplete((r, e) -> throwIfLocked(r))
-            .thenApply(v -> v.result());
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public CompletableFuture<Versioned<byte[]>> remove(String key) {
-        return proxy.<Remove, MapEntryUpdateResult<String, byte[]>>invoke(
-            REMOVE,
-            serializer()::encode,
-            new Remove(key),
-            serializer()::decode)
-            .whenComplete((r, e) -> throwIfLocked(r))
-            .thenApply(v -> v.result());
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public CompletableFuture<Boolean> remove(String key, byte[] value) {
-        return proxy.<RemoveValue, MapEntryUpdateResult<String, byte[]>>invoke(
-            REMOVE_VALUE,
-            serializer()::encode,
-            new RemoveValue(key, value),
-            serializer()::decode)
-            .whenComplete((r, e) -> throwIfLocked(r))
-            .thenApply(v -> v.updated());
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public CompletableFuture<Boolean> remove(String key, long version) {
-        return proxy.<RemoveVersion, MapEntryUpdateResult<String, byte[]>>invoke(
-            REMOVE_VERSION,
-            serializer()::encode,
-            new RemoveVersion(key, version),
-            serializer()::decode)
-            .whenComplete((r, e) -> throwIfLocked(r))
-            .thenApply(v -> v.updated());
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public CompletableFuture<Versioned<byte[]>> replace(String key, byte[] value) {
-        return proxy.<Replace, MapEntryUpdateResult<String, byte[]>>invoke(
-            REPLACE,
-            serializer()::encode,
-            new Replace(key, value),
-            serializer()::decode)
-            .whenComplete((r, e) -> throwIfLocked(r))
-            .thenApply(v -> v.result());
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public CompletableFuture<Boolean> replace(String key, byte[] oldValue, byte[] newValue) {
-        return proxy.<ReplaceValue, MapEntryUpdateResult<String, byte[]>>invoke(
-            REPLACE_VALUE,
-            serializer()::encode,
-            new ReplaceValue(key, oldValue, newValue),
-            serializer()::decode)
-            .whenComplete((r, e) -> throwIfLocked(r))
-            .thenApply(v -> v.updated());
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public CompletableFuture<Boolean> replace(String key, long oldVersion, byte[] newValue) {
-        return proxy.<ReplaceVersion, MapEntryUpdateResult<String, byte[]>>invoke(
-            REPLACE_VERSION,
-            serializer()::encode,
-            new ReplaceVersion(key, oldVersion, newValue),
-            serializer()::decode)
-            .whenComplete((r, e) -> throwIfLocked(r))
-            .thenApply(v -> v.updated());
-    }
-
-    @Override
-    public CompletableFuture<Void> clear() {
-        return proxy.<MapEntryUpdateResult.Status>invoke(CLEAR, serializer()::decode)
-            .whenComplete((r, e) -> throwIfLocked(r))
-            .thenApply(v -> null);
-    }
-
-    @Override
-    @SuppressWarnings("unchecked")
-    public CompletableFuture<Versioned<byte[]>> computeIf(String key,
-        Predicate<? super byte[]> condition,
-        BiFunction<? super String, ? super byte[], ? extends byte[]> remappingFunction) {
-        return get(key).thenCompose(r1 -> {
-            byte[] existingValue = r1 == null ? null : r1.value();
-            // if the condition evaluates to false, return existing value.
-            if (!condition.test(existingValue)) {
-                return CompletableFuture.completedFuture(r1);
-            }
-
-            byte[] computedValue;
-            try {
-                computedValue = remappingFunction.apply(key, existingValue);
-            } catch (Exception e) {
-                return Tools.exceptionalFuture(e);
-            }
-
-            if (computedValue == null && r1 == null) {
-                return CompletableFuture.completedFuture(null);
-            }
-
-            if (r1 == null) {
-                return proxy.<Put, MapEntryUpdateResult<String, byte[]>>invoke(
-                    PUT_IF_ABSENT,
-                    serializer()::encode,
-                    new Put(key, computedValue),
-                    serializer()::decode)
-                    .whenComplete((r, e) -> throwIfLocked(r))
-                    .thenCompose(r -> checkLocked(r))
-                    .thenApply(result -> new Versioned<>(computedValue, result.version()));
-            } else if (computedValue == null) {
-                return proxy.<RemoveVersion, MapEntryUpdateResult<String, byte[]>>invoke(
-                    REMOVE_VERSION,
-                    serializer()::encode,
-                    new RemoveVersion(key, r1.version()),
-                    serializer()::decode)
-                    .whenComplete((r, e) -> throwIfLocked(r))
-                    .thenCompose(r -> checkLocked(r))
-                    .thenApply(v -> null);
-            } else {
-                return proxy.<ReplaceVersion, MapEntryUpdateResult<String, byte[]>>invoke(
-                    REPLACE_VERSION,
-                    serializer()::encode,
-                    new ReplaceVersion(key, r1.version(), computedValue),
-                    serializer()::decode)
-                    .whenComplete((r, e) -> throwIfLocked(r))
-                    .thenCompose(r -> checkLocked(r))
-                    .thenApply(result -> result.status() == MapEntryUpdateResult.Status.OK
-                        ? new Versioned(computedValue, result.version()) : result.result());
-            }
-        });
-    }
-
-    private CompletableFuture<MapEntryUpdateResult<String, byte[]>> checkLocked(
-        MapEntryUpdateResult<String, byte[]> result) {
-        if (result.status() == MapEntryUpdateResult.Status.PRECONDITION_FAILED ||
-            result.status() == MapEntryUpdateResult.Status.WRITE_LOCK) {
-            return Tools.exceptionalFuture(new ConsistentMapException.ConcurrentModification());
-        }
-        return CompletableFuture.completedFuture(result);
-    }
-
-    @Override
-    public synchronized CompletableFuture<Void> addListener(MapEventListener<String, byte[]> listener,
-        Executor executor) {
-        if (mapEventListeners.isEmpty()) {
-            return proxy.invoke(ADD_LISTENER).thenRun(() -> mapEventListeners.put(listener, executor));
-        } else {
-            mapEventListeners.put(listener, executor);
-            return CompletableFuture.completedFuture(null);
-        }
-    }
-
-    @Override
-    public synchronized CompletableFuture<Void> removeListener(MapEventListener<String, byte[]> listener) {
-        if (mapEventListeners.remove(listener) != null && mapEventListeners.isEmpty()) {
-            return proxy.invoke(REMOVE_LISTENER).thenApply(v -> null);
-        }
-        return CompletableFuture.completedFuture(null);
-    }
-
-    private void throwIfLocked(MapEntryUpdateResult<String, byte[]> result) {
-        if (result != null) {
-            throwIfLocked(result.status());
-        }
-    }
-
-    private void throwIfLocked(MapEntryUpdateResult.Status status) {
-        if (status == MapEntryUpdateResult.Status.WRITE_LOCK) {
-            throw new ConcurrentModificationException("Cannot update map: Another transaction in progress");
-        }
-    }
-
-    @Override
-    public CompletableFuture<Version> begin(TransactionId transactionId) {
-        return proxy.<TransactionBegin, Long>invoke(
-            BEGIN,
-            serializer()::encode,
-            new TransactionBegin(transactionId),
-            serializer()::decode)
-            .thenApply(Version::new);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepare(TransactionLog<MapUpdate<String, byte[]>> transactionLog) {
-        return proxy.<TransactionPrepare, PrepareResult>invoke(
-            PREPARE,
-            serializer()::encode,
-            new TransactionPrepare(transactionLog),
-            serializer()::decode)
-            .thenApply(v -> v == PrepareResult.OK);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<MapUpdate<String, byte[]>> transactionLog) {
-        return proxy.<TransactionPrepareAndCommit, PrepareResult>invoke(
-            PREPARE_AND_COMMIT,
-            serializer()::encode,
-            new TransactionPrepareAndCommit(transactionLog),
-            serializer()::decode)
-            .thenApply(v -> v == PrepareResult.OK);
-    }
-
-    @Override
-    public CompletableFuture<Void> commit(TransactionId transactionId) {
-        return proxy.<TransactionCommit, CommitResult>invoke(
-            COMMIT,
-            serializer()::encode,
-            new TransactionCommit(transactionId),
-            serializer()::decode)
-            .thenApply(v -> null);
-    }
-
-    @Override
-    public CompletableFuture<Void> rollback(TransactionId transactionId) {
-        return proxy.invoke(
-            ROLLBACK,
-            serializer()::encode,
-            new TransactionRollback(transactionId),
-            serializer()::decode)
-            .thenApply(v -> null);
-    }
-
-    private boolean isListening() {
-        return !mapEventListeners.isEmpty();
-    }
-
-    @Override
-    public CompletableFuture<AsyncIterator<Entry<String, Versioned<byte[]>>>> iterator() {
-        return proxy.<Long>invoke(OPEN_ITERATOR, SERIALIZER::decode)
-            .thenApply(ConsistentMultimapIterator::new);
-    }
-
-    /**
-     * Consistent multimap iterator.
-     */
-    private class ConsistentMultimapIterator implements AsyncIterator<Map.Entry<String, Versioned<byte[]>>> {
-        private final long id;
-        private volatile CompletableFuture<IteratorBatch> batch;
-        private volatile CompletableFuture<Void> closeFuture;
-
-        ConsistentMultimapIterator(long id) {
-            this.id = id;
-            this.batch = CompletableFuture.completedFuture(
-                new IteratorBatch(0, Collections.emptyList()));
-        }
-
-        /**
-         * Returns the current batch iterator or lazily fetches the next batch from the cluster.
-         *
-         * @return the next batch iterator
-         */
-        private CompletableFuture<Iterator<Entry<String, Versioned<byte[]>>>> batch() {
-            return batch.thenCompose(iterator -> {
-                if (iterator != null && !iterator.hasNext()) {
-                    batch = fetch(iterator.position());
-                    return batch.thenApply(Function.identity());
-                }
-                return CompletableFuture.completedFuture(iterator);
-            });
-        }
-
-        /**
-         * Fetches the next batch of entries from the cluster.
-         *
-         * @param position the position from which to fetch the next batch
-         * @return the next batch of entries from the cluster
-         */
-        private CompletableFuture<IteratorBatch> fetch(int position) {
-            return proxy.<IteratorPosition, IteratorBatch>invoke(
-                NEXT,
-                SERIALIZER::encode,
-                new IteratorPosition(id, position),
-                SERIALIZER::decode)
-                .thenCompose(batch -> {
-                    if (batch == null) {
-                        return close().thenApply(v -> null);
-                    }
-                    return CompletableFuture.completedFuture(batch);
-                });
-        }
-
-        /**
-         * Closes the iterator.
-         *
-         * @return future to be completed once the iterator has been closed
-         */
-        private CompletableFuture<Void> close() {
-            if (closeFuture == null) {
-                synchronized (this) {
-                    if (closeFuture == null) {
-                        closeFuture = proxy.invoke(CLOSE_ITERATOR, SERIALIZER::encode, id);
-                    }
-                }
-            }
-            return closeFuture;
-        }
-
-        @Override
-        public CompletableFuture<Boolean> hasNext() {
-            return batch().thenApply(iterator -> iterator != null && iterator.hasNext());
-        }
-
-        @Override
-        public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> next() {
-            return batch().thenCompose(iterator -> {
-                if (iterator == null) {
-                    return Tools.exceptionalFuture(new NoSuchElementException());
-                }
-                return CompletableFuture.completedFuture(iterator.next());
-            });
-        }
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapEvents.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapEvents.java
deleted file mode 100644
index a0429da..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapEvents.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import io.atomix.protocols.raft.event.EventType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.MapEvent;
-
-/**
- * Atomix consistent map events.
- */
-public enum AtomixConsistentMapEvents implements EventType {
-    CHANGE;
-
-    @Override
-    public String id() {
-        return name();
-    }
-
-    public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
-            .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 50)
-            .register(MapEvent.class)
-            .register(MapEvent.Type.class)
-            .register(byte[].class)
-            .build("AtomixConsistentMapEvents");
-
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapOperations.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapOperations.java
deleted file mode 100644
index 4f131ef..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapOperations.java
+++ /dev/null
@@ -1,648 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.Map;
-
-import com.google.common.collect.Maps;
-import io.atomix.protocols.raft.operation.OperationId;
-import io.atomix.protocols.raft.operation.OperationType;
-import io.atomix.utils.ArraySizeHashPrinter;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Match;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Versioned;
-
-import static com.google.common.base.MoreObjects.toStringHelper;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * {@link AtomixConsistentMap} resource state machine operations.
- */
-public enum AtomixConsistentMapOperations implements OperationId {
-    IS_EMPTY(OperationType.QUERY),
-    SIZE(OperationType.QUERY),
-    CONTAINS_KEY(OperationType.QUERY),
-    CONTAINS_VALUE(OperationType.QUERY),
-    GET(OperationType.QUERY),
-    GET_OR_DEFAULT(OperationType.QUERY),
-    KEY_SET(OperationType.QUERY),
-    VALUES(OperationType.QUERY),
-    ENTRY_SET(OperationType.QUERY),
-    PUT(OperationType.COMMAND),
-    PUT_IF_ABSENT(OperationType.COMMAND),
-    PUT_AND_GET(OperationType.COMMAND),
-    REMOVE(OperationType.COMMAND),
-    REMOVE_VALUE(OperationType.COMMAND),
-    REMOVE_VERSION(OperationType.COMMAND),
-    REPLACE(OperationType.COMMAND),
-    REPLACE_VALUE(OperationType.COMMAND),
-    REPLACE_VERSION(OperationType.COMMAND),
-    CLEAR(OperationType.COMMAND),
-    ADD_LISTENER(OperationType.COMMAND),
-    REMOVE_LISTENER(OperationType.COMMAND),
-    BEGIN(OperationType.COMMAND),
-    PREPARE(OperationType.COMMAND),
-    PREPARE_AND_COMMIT(OperationType.COMMAND),
-    COMMIT(OperationType.COMMAND),
-    ROLLBACK(OperationType.COMMAND),
-    OPEN_ITERATOR(OperationType.COMMAND),
-    NEXT(OperationType.QUERY),
-    CLOSE_ITERATOR(OperationType.COMMAND);
-
-    private final OperationType type;
-
-    AtomixConsistentMapOperations(OperationType type) {
-        this.type = type;
-    }
-
-    @Override
-    public String id() {
-        return name();
-    }
-
-    @Override
-    public OperationType type() {
-        return type;
-    }
-
-    public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
-            .register(KryoNamespaces.BASIC)
-            .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
-            .register(ContainsKey.class)
-            .register(ContainsValue.class)
-            .register(Get.class)
-            .register(GetOrDefault.class)
-            .register(Put.class)
-            .register(Remove.class)
-            .register(RemoveValue.class)
-            .register(RemoveVersion.class)
-            .register(Replace.class)
-            .register(ReplaceValue.class)
-            .register(ReplaceVersion.class)
-            .register(TransactionBegin.class)
-            .register(TransactionPrepare.class)
-            .register(TransactionPrepareAndCommit.class)
-            .register(TransactionCommit.class)
-            .register(TransactionRollback.class)
-            .register(TransactionId.class)
-            .register(TransactionLog.class)
-            .register(MapUpdate.class)
-            .register(MapUpdate.Type.class)
-            .register(PrepareResult.class)
-            .register(CommitResult.class)
-            .register(RollbackResult.class)
-            .register(Match.class)
-            .register(MapEntryUpdateResult.class)
-            .register(MapEntryUpdateResult.Status.class)
-            .register(Versioned.class)
-            .register(byte[].class)
-            .register(Maps.immutableEntry("", "").getClass())
-            .register(IteratorBatch.class)
-            .register(IteratorPosition.class)
-            .build("AtomixConsistentMapOperations");
-
-    /**
-     * Abstract map command.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class MapOperation {
-        @Override
-        public String toString() {
-            return toStringHelper(getClass())
-                    .toString();
-        }
-    }
-
-    /**
-     * Abstract key-based query.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class KeyOperation extends MapOperation {
-        protected String key;
-
-        public KeyOperation() {
-        }
-
-        public KeyOperation(String key) {
-            this.key = checkNotNull(key, "key cannot be null");
-        }
-
-        /**
-         * Returns the key.
-         * @return key
-         */
-        public String key() {
-            return key;
-        }
-
-        @Override
-        public String toString() {
-            return toStringHelper(getClass())
-                    .add("key", key)
-                    .toString();
-        }
-    }
-
-    /**
-     * Abstract value-based query.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class ValueOperation extends MapOperation {
-        protected byte[] value;
-
-        public ValueOperation() {
-        }
-
-        public ValueOperation(byte[] value) {
-            this.value = value;
-        }
-
-        /**
-         * Returns the value.
-         * @return value
-         */
-        public byte[] value() {
-            return value;
-        }
-
-        @Override
-        public String toString() {
-            return toStringHelper(getClass())
-                    .add("value", value)
-                    .toString();
-        }
-    }
-
-    /**
-     * Abstract key/value operation.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class KeyValueOperation extends KeyOperation {
-        protected byte[] value;
-
-        public KeyValueOperation() {
-        }
-
-        public KeyValueOperation(String key, byte[] value) {
-            super(key);
-            this.value = value;
-        }
-
-        /**
-         * Returns the value.
-         * @return value
-         */
-        public byte[] value() {
-            return value;
-        }
-
-        @Override
-        public String toString() {
-            return toStringHelper(getClass())
-                    .add("key", key)
-                    .add("value", ArraySizeHashPrinter.of(value))
-                    .toString();
-        }
-    }
-
-    /**
-     * Abstract key/version operation.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class KeyVersionOperation extends KeyOperation {
-        protected long version;
-
-        public KeyVersionOperation() {
-        }
-
-        public KeyVersionOperation(String key, long version) {
-            super(key);
-            this.version = version;
-        }
-
-        /**
-         * Returns the version.
-         * @return version
-         */
-        public long version() {
-            return version;
-        }
-
-        @Override
-        public String toString() {
-            return toStringHelper(getClass())
-                    .add("key", key)
-                    .add("version", version)
-                    .toString();
-        }
-    }
-
-    /**
-     * Contains key command.
-     */
-    @SuppressWarnings("serial")
-    public static class ContainsKey extends KeyOperation {
-        public ContainsKey() {
-        }
-
-        public ContainsKey(String key) {
-            super(key);
-        }
-    }
-
-    /**
-     * Contains value command.
-     */
-    @SuppressWarnings("serial")
-    public static class ContainsValue extends ValueOperation {
-        public ContainsValue() {
-        }
-
-        public ContainsValue(byte[] value) {
-            super(value);
-        }
-    }
-
-    /**
-     * Map put operation.
-     */
-    public static class Put extends KeyValueOperation {
-        public Put() {
-        }
-
-        public Put(String key, byte[] value) {
-            super(key, value);
-        }
-    }
-
-    /**
-     * Remove operation.
-     */
-    public static class Remove extends KeyOperation {
-        public Remove() {
-        }
-
-        public Remove(String key) {
-            super(key);
-        }
-    }
-
-    /**
-     * Remove if value match operation.
-     */
-    public static class RemoveValue extends KeyValueOperation {
-        public RemoveValue() {
-        }
-
-        public RemoveValue(String key, byte[] value) {
-            super(key, value);
-        }
-    }
-
-    /**
-     * Remove if version match operation.
-     */
-    public static class RemoveVersion extends KeyVersionOperation {
-        public RemoveVersion() {
-        }
-
-        public RemoveVersion(String key, long version) {
-            super(key, version);
-        }
-    }
-
-    /**
-     * Replace operation.
-     */
-    public static class Replace extends KeyValueOperation {
-        public Replace() {
-        }
-
-        public Replace(String key, byte[] value) {
-            super(key, value);
-        }
-    }
-
-    /**
-     * Replace by value operation.
-     */
-    public static class ReplaceValue extends KeyOperation {
-        private byte[] oldValue;
-        private byte[] newValue;
-
-        public ReplaceValue() {
-        }
-
-        public ReplaceValue(String key, byte[] oldValue, byte[] newValue) {
-            super(key);
-            this.oldValue = oldValue;
-            this.newValue = newValue;
-        }
-
-        public byte[] oldValue() {
-            return oldValue;
-        }
-
-        public byte[] newValue() {
-            return newValue;
-        }
-
-        @Override
-        public String toString() {
-            return toStringHelper(this)
-                    .add("key", key)
-                    .add("oldValue", ArraySizeHashPrinter.of(oldValue))
-                    .add("newValue", ArraySizeHashPrinter.of(newValue))
-                    .toString();
-        }
-    }
-
-    /**
-     * Replace by version operation.
-     */
-    public static class ReplaceVersion extends KeyOperation {
-        private long oldVersion;
-        private byte[] newValue;
-
-        public ReplaceVersion() {
-        }
-
-        public ReplaceVersion(String key, long oldVersion, byte[] newValue) {
-            super(key);
-            this.oldVersion = oldVersion;
-            this.newValue = newValue;
-        }
-
-        public long oldVersion() {
-            return oldVersion;
-        }
-
-        public byte[] newValue() {
-            return newValue;
-        }
-
-        @Override
-        public String toString() {
-            return toStringHelper(this)
-                    .add("key", key)
-                    .add("oldVersion", oldVersion)
-                    .add("newValue", ArraySizeHashPrinter.of(newValue))
-                    .toString();
-        }
-    }
-
-    /**
-     * Transaction begin command.
-     */
-    public static class TransactionBegin extends MapOperation {
-        private TransactionId transactionId;
-
-        public TransactionBegin() {
-        }
-
-        public TransactionBegin(TransactionId transactionId) {
-            this.transactionId = transactionId;
-        }
-
-        public TransactionId transactionId() {
-            return transactionId;
-        }
-    }
-
-    /**
-     * Map prepare command.
-     */
-    @SuppressWarnings("serial")
-    public static class TransactionPrepare extends MapOperation {
-        private TransactionLog<MapUpdate<String, byte[]>> transactionLog;
-
-        public TransactionPrepare() {
-        }
-
-        public TransactionPrepare(TransactionLog<MapUpdate<String, byte[]>> transactionLog) {
-            this.transactionLog = transactionLog;
-        }
-
-        public TransactionLog<MapUpdate<String, byte[]>> transactionLog() {
-            return transactionLog;
-        }
-
-        @Override
-        public String toString() {
-            return toStringHelper(getClass())
-                    .add("transactionLog", transactionLog)
-                    .toString();
-        }
-    }
-
-    /**
-     * Map prepareAndCommit command.
-     */
-    @SuppressWarnings("serial")
-    public static class TransactionPrepareAndCommit extends TransactionPrepare {
-        public TransactionPrepareAndCommit() {
-        }
-
-        public TransactionPrepareAndCommit(TransactionLog<MapUpdate<String, byte[]>> transactionLog) {
-            super(transactionLog);
-        }
-    }
-
-    /**
-     * Map transaction commit command.
-     */
-    @SuppressWarnings("serial")
-    public static class TransactionCommit extends MapOperation {
-        private TransactionId transactionId;
-
-        public TransactionCommit() {
-        }
-
-        public TransactionCommit(TransactionId transactionId) {
-            this.transactionId = transactionId;
-        }
-
-        /**
-         * Returns the transaction identifier.
-         * @return transaction id
-         */
-        public TransactionId transactionId() {
-            return transactionId;
-        }
-
-        @Override
-        public String toString() {
-            return toStringHelper(getClass())
-                    .add("transactionId", transactionId)
-                    .toString();
-        }
-    }
-
-    /**
-     * Map transaction rollback command.
-     */
-    @SuppressWarnings("serial")
-    public static class TransactionRollback extends MapOperation {
-        private TransactionId transactionId;
-
-        public TransactionRollback() {
-        }
-
-        public TransactionRollback(TransactionId transactionId) {
-            this.transactionId = transactionId;
-        }
-
-        /**
-         * Returns the transaction identifier.
-         * @return transaction id
-         */
-        public TransactionId transactionId() {
-            return transactionId;
-        }
-
-        @Override
-        public String toString() {
-            return toStringHelper(getClass())
-                    .add("transactionId", transactionId)
-                    .toString();
-        }
-    }
-
-    /**
-     * Get query.
-     */
-    @SuppressWarnings("serial")
-    public static class Get extends KeyOperation {
-        public Get() {
-        }
-
-        public Get(String key) {
-            super(key);
-        }
-    }
-
-    /**
-     * Get or default query.
-     */
-    @SuppressWarnings("serial")
-    public static class GetOrDefault extends KeyOperation {
-        private byte[] defaultValue;
-
-        public GetOrDefault() {
-        }
-
-        public GetOrDefault(String key, byte[] defaultValue) {
-            super(key);
-            this.defaultValue = defaultValue;
-        }
-
-        /**
-         * Returns the default value.
-         *
-         * @return the default value
-         */
-        public byte[] defaultValue() {
-            return defaultValue;
-        }
-
-        @Override
-        public String toString() {
-            return toStringHelper(this)
-                    .add("key", key)
-                    .add("defaultValue", ArraySizeHashPrinter.of(defaultValue))
-                    .toString();
-        }
-    }
-
-    /**
-     * Iterator position.
-     */
-    public static class IteratorPosition {
-        private long iteratorId;
-        private int position;
-
-        private IteratorPosition() {
-        }
-
-        public IteratorPosition(long iteratorId, int position) {
-            this.iteratorId = iteratorId;
-            this.position = position;
-        }
-
-        public long iteratorId() {
-            return iteratorId;
-        }
-
-        public int position() {
-            return position;
-        }
-    }
-
-    /**
-     * Iterator batch.
-     */
-    public static class IteratorBatch implements Iterator<Map.Entry<String, Versioned<byte[]>>> {
-        private int position;
-        private Collection<Map.Entry<String, Versioned<byte[]>>> entries;
-        private transient volatile Iterator<Map.Entry<String, Versioned<byte[]>>> iterator;
-
-        private IteratorBatch() {
-        }
-
-        public IteratorBatch(int position, Collection<Map.Entry<String, Versioned<byte[]>>> entries) {
-            this.position = position;
-            this.entries = entries;
-        }
-
-        public int position() {
-            return position;
-        }
-
-        public Collection<Map.Entry<String, Versioned<byte[]>>> entries() {
-            return entries;
-        }
-
-        private Iterator<Map.Entry<String, Versioned<byte[]>>> iterator() {
-            Iterator<Map.Entry<String, Versioned<byte[]>>> iterator = this.iterator;
-            if (iterator == null) {
-                synchronized (entries) {
-                    iterator = this.iterator;
-                    if (iterator == null) {
-                        iterator = entries.iterator();
-                        this.iterator = iterator;
-                    }
-                }
-            }
-            return iterator;
-        }
-
-        @Override
-        public boolean hasNext() {
-            return iterator().hasNext();
-        }
-
-        @Override
-        public Map.Entry<String, Versioned<byte[]>> next() {
-            return iterator().next();
-        }
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapService.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapService.java
deleted file mode 100644
index 0fcf3b5..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapService.java
+++ /dev/null
@@ -1,1131 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.function.Predicate;
-import java.util.stream.Collectors;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import io.atomix.protocols.raft.service.AbstractRaftService;
-import io.atomix.protocols.raft.service.Commit;
-import io.atomix.protocols.raft.service.RaftServiceExecutor;
-import io.atomix.protocols.raft.session.RaftSession;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Match;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ContainsKey;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ContainsValue;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.Get;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.GetOrDefault;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.Put;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.Remove;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.RemoveValue;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.RemoveVersion;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.Replace;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ReplaceValue;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ReplaceVersion;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.TransactionBegin;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.TransactionCommit;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.TransactionPrepare;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.TransactionPrepareAndCommit;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.TransactionRollback;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.MapEvent;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Versioned;
-
-import static com.google.common.base.Preconditions.checkState;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapEvents.CHANGE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ADD_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.BEGIN;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.CLEAR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.CLOSE_ITERATOR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.COMMIT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.CONTAINS_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.CONTAINS_VALUE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ENTRY_SET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.GET_OR_DEFAULT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.IS_EMPTY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.IteratorBatch;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.IteratorPosition;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.KEY_SET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.NEXT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.OPEN_ITERATOR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PREPARE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PREPARE_AND_COMMIT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PUT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PUT_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.PUT_IF_ABSENT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REMOVE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REMOVE_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REMOVE_VALUE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REMOVE_VERSION;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REPLACE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REPLACE_VALUE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.REPLACE_VERSION;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.ROLLBACK;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.SIZE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.VALUES;
-
-/**
- * State Machine for {@link AtomixConsistentMap} resource.
- */
-public class AtomixConsistentMapService extends AbstractRaftService {
-
-    private static final int MAX_ITERATOR_BATCH_SIZE = 1024 * 32;
-
-    private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
-        .register(KryoNamespaces.BASIC)
-        .register(AtomixConsistentMapOperations.NAMESPACE)
-        .register(AtomixConsistentMapEvents.NAMESPACE)
-        .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 100)
-        .register(TransactionScope.class)
-        .register(TransactionLog.class)
-        .register(TransactionId.class)
-        .register(MapEntryValue.class)
-        .register(MapEntryValue.Type.class)
-        .register(new HashMap().keySet().getClass())
-        .build());
-
-    protected Map<Long, RaftSession> listeners = new LinkedHashMap<>();
-    private Map<String, MapEntryValue> map;
-    protected Set<String> preparedKeys = Sets.newHashSet();
-    private Map<Long, IteratorContext> iterators = Maps.newHashMap();
-    protected Map<TransactionId, TransactionScope> activeTransactions = Maps.newHashMap();
-    protected long currentVersion;
-
-    public AtomixConsistentMapService() {
-        map = createMap();
-    }
-
-    protected Map<String, MapEntryValue> createMap() {
-        return Maps.newConcurrentMap();
-    }
-
-    protected Map<String, MapEntryValue> entries() {
-        return map;
-    }
-
-    protected Serializer serializer() {
-        return SERIALIZER;
-    }
-
-    @Override
-    public void snapshot(SnapshotWriter writer) {
-        writer.writeObject(Sets.newHashSet(listeners.keySet()), serializer()::encode);
-        writer.writeObject(preparedKeys, serializer()::encode);
-        writer.writeObject(entries(), serializer()::encode);
-        writer.writeObject(activeTransactions, serializer()::encode);
-        writer.writeLong(currentVersion);
-
-        Map<Long, Long> iterators = Maps.newHashMap();
-        this.iterators.forEach((id, context) -> iterators.put(id, context.sessionId));
-        writer.writeObject(iterators, serializer()::encode);
-    }
-
-    @Override
-    public void install(SnapshotReader reader) {
-        listeners = new LinkedHashMap<>();
-        for (Long sessionId : reader.<Set<Long>>readObject(serializer()::decode)) {
-            listeners.put(sessionId, sessions().getSession(sessionId));
-        }
-        preparedKeys = reader.readObject(serializer()::decode);
-        map = reader.readObject(serializer()::decode);
-        activeTransactions = reader.readObject(serializer()::decode);
-        currentVersion = reader.readLong();
-
-        Map<Long, Long> iterators = reader.readObject(serializer()::decode);
-        this.iterators = Maps.newHashMap();
-        iterators.forEach((id, session) ->
-            this.iterators.put(id, new IteratorContext(session, entries().entrySet().iterator())));
-    }
-
-    @Override
-    protected void configure(RaftServiceExecutor executor) {
-        // Listeners
-        executor.register(ADD_LISTENER, (Commit<Void> c) -> listen(c.session()));
-        executor.register(REMOVE_LISTENER, (Commit<Void> c) -> unlisten(c.session()));
-        // Queries
-        executor.register(CONTAINS_KEY, serializer()::decode, this::containsKey, serializer()::encode);
-        executor.register(CONTAINS_VALUE, serializer()::decode, this::containsValue, serializer()::encode);
-        executor.register(ENTRY_SET, (Commit<Void> c) -> entrySet(), serializer()::encode);
-        executor.register(GET, serializer()::decode, this::get, serializer()::encode);
-        executor.register(GET_OR_DEFAULT, serializer()::decode, this::getOrDefault, serializer()::encode);
-        executor.register(IS_EMPTY, (Commit<Void> c) -> isEmpty(), serializer()::encode);
-        executor.register(KEY_SET, (Commit<Void> c) -> keySet(), serializer()::encode);
-        executor.register(SIZE, (Commit<Void> c) -> size(), serializer()::encode);
-        executor.register(VALUES, (Commit<Void> c) -> values(), serializer()::encode);
-        // Commands
-        executor.register(PUT, serializer()::decode, this::put, serializer()::encode);
-        executor.register(PUT_IF_ABSENT, serializer()::decode, this::putIfAbsent, serializer()::encode);
-        executor.register(PUT_AND_GET, serializer()::decode, this::putAndGet, serializer()::encode);
-        executor.register(REMOVE, serializer()::decode, this::remove, serializer()::encode);
-        executor.register(REMOVE_VALUE, serializer()::decode, this::removeValue, serializer()::encode);
-        executor.register(REMOVE_VERSION, serializer()::decode, this::removeVersion, serializer()::encode);
-        executor.register(REPLACE, serializer()::decode, this::replace, serializer()::encode);
-        executor.register(REPLACE_VALUE, serializer()::decode, this::replaceValue, serializer()::encode);
-        executor.register(REPLACE_VERSION, serializer()::decode, this::replaceVersion, serializer()::encode);
-        executor.register(CLEAR, (Commit<Void> c) -> clear(), serializer()::encode);
-        executor.register(BEGIN, serializer()::decode, this::begin, serializer()::encode);
-        executor.register(PREPARE, serializer()::decode, this::prepare, serializer()::encode);
-        executor.register(PREPARE_AND_COMMIT, serializer()::decode, this::prepareAndCommit, serializer()::encode);
-        executor.register(COMMIT, serializer()::decode, this::commit, serializer()::encode);
-        executor.register(ROLLBACK, serializer()::decode, this::rollback, serializer()::encode);
-        executor.register(OPEN_ITERATOR, this::openIterator, serializer()::encode);
-        executor.register(NEXT, serializer()::decode, this::next, serializer()::encode);
-        executor.register(CLOSE_ITERATOR, serializer()::decode, this::closeIterator);
-    }
-
-    /**
-     * Handles a contains key commit.
-     *
-     * @param commit containsKey commit
-     * @return {@code true} if map contains key
-     */
-    protected boolean containsKey(Commit<? extends ContainsKey> commit) {
-        MapEntryValue value = entries().get(commit.value().key());
-        return value != null && value.type() != MapEntryValue.Type.TOMBSTONE;
-    }
-
-    /**
-     * Handles a contains value commit.
-     *
-     * @param commit containsValue commit
-     * @return {@code true} if map contains value
-     */
-    protected boolean containsValue(Commit<? extends ContainsValue> commit) {
-        Match<byte[]> valueMatch = Match.ifValue(commit.value().value());
-        return entries().values().stream()
-            .filter(value -> value.type() != MapEntryValue.Type.TOMBSTONE)
-            .anyMatch(value -> valueMatch.matches(value.value()));
-    }
-
-    /**
-     * Handles a get commit.
-     *
-     * @param commit get commit
-     * @return value mapped to key
-     */
-    protected Versioned<byte[]> get(Commit<? extends Get> commit) {
-        return toVersioned(entries().get(commit.value().key()));
-    }
-
-    /**
-     * Handles a get or default commit.
-     *
-     * @param commit get or default commit
-     * @return value mapped to key
-     */
-    protected Versioned<byte[]> getOrDefault(Commit<? extends GetOrDefault> commit) {
-        MapEntryValue value = entries().get(commit.value().key());
-        if (value == null) {
-            return new Versioned<>(commit.value().defaultValue(), 0);
-        } else if (value.type() == MapEntryValue.Type.TOMBSTONE) {
-            return new Versioned<>(commit.value().defaultValue(), value.version);
-        } else {
-            return new Versioned<>(value.value(), value.version);
-        }
-    }
-
-    /**
-     * Handles a size commit.
-     *
-     * @return number of entries in map
-     */
-    protected int size() {
-        return (int) entries().values().stream()
-            .filter(value -> value.type() != MapEntryValue.Type.TOMBSTONE)
-            .count();
-    }
-
-    /**
-     * Handles an is empty commit.
-     *
-     * @return {@code true} if map is empty
-     */
-    protected boolean isEmpty() {
-        return entries().values().stream()
-            .noneMatch(value -> value.type() != MapEntryValue.Type.TOMBSTONE);
-    }
-
-    /**
-     * Handles a keySet commit.
-     *
-     * @return set of keys in map
-     */
-    protected Set<String> keySet() {
-        return entries().entrySet().stream()
-            .filter(entry -> entry.getValue().type() != MapEntryValue.Type.TOMBSTONE)
-            .map(Map.Entry::getKey)
-            .collect(Collectors.toSet());
-    }
-
-    /**
-     * Handles a values commit.
-     *
-     * @return collection of values in map
-     */
-    protected Collection<Versioned<byte[]>> values() {
-        return entries().entrySet().stream()
-            .filter(entry -> entry.getValue().type() != MapEntryValue.Type.TOMBSTONE)
-            .map(entry -> toVersioned(entry.getValue()))
-            .collect(Collectors.toList());
-    }
-
-    /**
-     * Handles a entry set commit.
-     *
-     * @return set of map entries
-     */
-    protected Set<Map.Entry<String, Versioned<byte[]>>> entrySet() {
-        return entries().entrySet().stream()
-            .filter(entry -> entry.getValue().type() != MapEntryValue.Type.TOMBSTONE)
-            .map(e -> Maps.immutableEntry(e.getKey(), toVersioned(e.getValue())))
-            .collect(Collectors.toSet());
-    }
-
-    /**
-     * Returns a boolean indicating whether the given MapEntryValues are equal.
-     *
-     * @param oldValue the first value to compare
-     * @param newValue the second value to compare
-     * @return indicates whether the two values are equal
-     */
-    protected boolean valuesEqual(MapEntryValue oldValue, MapEntryValue newValue) {
-        return (oldValue == null && newValue == null)
-            || (oldValue != null && newValue != null && valuesEqual(oldValue.value(), newValue.value()));
-    }
-
-    /**
-     * Returns a boolean indicating whether the given entry values are equal.
-     *
-     * @param oldValue the first value to compare
-     * @param newValue the second value to compare
-     * @return indicates whether the two values are equal
-     */
-    protected boolean valuesEqual(byte[] oldValue, byte[] newValue) {
-        return (oldValue == null && newValue == null)
-            || (oldValue != null && newValue != null && Arrays.equals(oldValue, newValue));
-    }
-
-    /**
-     * Returns a boolean indicating whether the given MapEntryValue is null or a tombstone.
-     *
-     * @param value the value to check
-     * @return indicates whether the given value is null or is a tombstone
-     */
-    protected boolean valueIsNull(MapEntryValue value) {
-        return value == null || value.type() == MapEntryValue.Type.TOMBSTONE;
-    }
-
-    /**
-     * Handles a put commit.
-     *
-     * @param commit put commit
-     * @return map entry update result
-     */
-    protected MapEntryUpdateResult<String, byte[]> put(Commit<? extends Put> commit) {
-        String key = commit.value().key();
-        MapEntryValue oldValue = entries().get(key);
-        MapEntryValue newValue = new MapEntryValue(MapEntryValue.Type.VALUE, commit.index(), commit.value().value());
-
-        // If the value is null or a tombstone, this is an insert.
-        // Otherwise, only update the value if it has changed to reduce the number of events.
-        if (valueIsNull(oldValue)) {
-            // If the key has been locked by a transaction, return a WRITE_LOCK error.
-            if (preparedKeys.contains(key)) {
-                return new MapEntryUpdateResult<>(
-                    MapEntryUpdateResult.Status.WRITE_LOCK,
-                    commit.index(),
-                    key,
-                    toVersioned(oldValue));
-            }
-            entries().put(commit.value().key(),
-                new MapEntryValue(MapEntryValue.Type.VALUE, newValue.version(), newValue.value()));
-            Versioned<byte[]> result = toVersioned(oldValue);
-            publish(new MapEvent<>(MapEvent.Type.INSERT, "", key, toVersioned(newValue), result));
-            return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.OK, commit.index(), key, result);
-        } else if (!valuesEqual(oldValue, newValue)) {
-            // If the key has been locked by a transaction, return a WRITE_LOCK error.
-            if (preparedKeys.contains(key)) {
-                return new MapEntryUpdateResult<>(
-                    MapEntryUpdateResult.Status.WRITE_LOCK,
-                    commit.index(),
-                    key,
-                    toVersioned(oldValue));
-            }
-            entries().put(commit.value().key(),
-                new MapEntryValue(MapEntryValue.Type.VALUE, newValue.version(), newValue.value()));
-            Versioned<byte[]> result = toVersioned(oldValue);
-            publish(new MapEvent<>(MapEvent.Type.UPDATE, "", key, toVersioned(newValue), result));
-            return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.OK, commit.index(), key, result);
-        }
-        // If the value hasn't changed, return a NOOP result.
-        return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.NOOP, commit.index(), key, toVersioned(oldValue));
-    }
-
-    /**
-     * Handles a putIfAbsent commit.
-     *
-     * @param commit putIfAbsent commit
-     * @return map entry update result
-     */
-    protected MapEntryUpdateResult<String, byte[]> putIfAbsent(Commit<? extends Put> commit) {
-        String key = commit.value().key();
-        MapEntryValue oldValue = entries().get(key);
-
-        // If the value is null, this is an INSERT.
-        if (valueIsNull(oldValue)) {
-            // If the key has been locked by a transaction, return a WRITE_LOCK error.
-            if (preparedKeys.contains(key)) {
-                return new MapEntryUpdateResult<>(
-                    MapEntryUpdateResult.Status.WRITE_LOCK,
-                    commit.index(),
-                    key,
-                    toVersioned(oldValue));
-            }
-            MapEntryValue newValue = new MapEntryValue(
-                MapEntryValue.Type.VALUE,
-                commit.index(),
-                commit.value().value());
-            entries().put(commit.value().key(), newValue);
-            Versioned<byte[]> result = toVersioned(newValue);
-            publish(new MapEvent<>(MapEvent.Type.INSERT, "", key, result, null));
-            return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.OK, commit.index(), key, null);
-        }
-        return new MapEntryUpdateResult<>(
-            MapEntryUpdateResult.Status.PRECONDITION_FAILED,
-            commit.index(),
-            key,
-            toVersioned(oldValue));
-    }
-
-    /**
-     * Handles a putAndGet commit.
-     *
-     * @param commit putAndGet commit
-     * @return map entry update result
-     */
-    protected MapEntryUpdateResult<String, byte[]> putAndGet(Commit<? extends Put> commit) {
-        String key = commit.value().key();
-        MapEntryValue oldValue = entries().get(key);
-        MapEntryValue newValue = new MapEntryValue(MapEntryValue.Type.VALUE, commit.index(), commit.value().value());
-
-        // If the value is null or a tombstone, this is an insert.
-        // Otherwise, only update the value if it has changed to reduce the number of events.
-        if (valueIsNull(oldValue)) {
-            // If the key has been locked by a transaction, return a WRITE_LOCK error.
-            if (preparedKeys.contains(key)) {
-                return new MapEntryUpdateResult<>(
-                    MapEntryUpdateResult.Status.WRITE_LOCK,
-                    commit.index(),
-                    key,
-                    toVersioned(oldValue));
-            }
-            entries().put(commit.value().key(), newValue);
-            Versioned<byte[]> result = toVersioned(newValue);
-            publish(new MapEvent<>(MapEvent.Type.INSERT, "", key, result, null));
-            return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.OK, commit.index(), key, result);
-        } else if (!valuesEqual(oldValue, newValue)) {
-            // If the key has been locked by a transaction, return a WRITE_LOCK error.
-            if (preparedKeys.contains(key)) {
-                return new MapEntryUpdateResult<>(
-                    MapEntryUpdateResult.Status.WRITE_LOCK,
-                    commit.index(),
-                    key,
-                    toVersioned(oldValue));
-            }
-            entries().put(commit.value().key(), newValue);
-            Versioned<byte[]> result = toVersioned(newValue);
-            publish(new MapEvent<>(MapEvent.Type.UPDATE, "", key, result, toVersioned(oldValue)));
-            return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.OK, commit.index(), key, result);
-        }
-        return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.NOOP, commit.index(), key, toVersioned(oldValue));
-    }
-
-    /**
-     * Handles a remove commit.
-     *
-     * @param index     the commit index
-     * @param key       the key to remove
-     * @param predicate predicate to determine whether to remove the entry
-     * @return map entry update result
-     */
-    private MapEntryUpdateResult<String, byte[]> removeIf(long index, String key, Predicate<MapEntryValue> predicate) {
-        MapEntryValue value = entries().get(key);
-
-        // If the value does not exist or doesn't match the predicate, return a PRECONDITION_FAILED error.
-        if (valueIsNull(value) || !predicate.test(value)) {
-            return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.PRECONDITION_FAILED, index, key, null);
-        }
-
-        // If the key has been locked by a transaction, return a WRITE_LOCK error.
-        if (preparedKeys.contains(key)) {
-            return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.WRITE_LOCK, index, key, null);
-        }
-
-        // If no transactions are active, remove the key. Otherwise, replace it with a tombstone.
-        if (activeTransactions.isEmpty()) {
-            entries().remove(key);
-        } else {
-            entries().put(key, new MapEntryValue(MapEntryValue.Type.TOMBSTONE, index, null));
-        }
-
-        Versioned<byte[]> result = toVersioned(value);
-        publish(new MapEvent<>(MapEvent.Type.REMOVE, "", key, null, result));
-        return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.OK, index, key, result);
-    }
-
-    /**
-     * Handles a remove commit.
-     *
-     * @param commit remove commit
-     * @return map entry update result
-     */
-    protected MapEntryUpdateResult<String, byte[]> remove(Commit<? extends Remove> commit) {
-        return removeIf(commit.index(), commit.value().key(), v -> true);
-    }
-
-    /**
-     * Handles a removeValue commit.
-     *
-     * @param commit removeValue commit
-     * @return map entry update result
-     */
-    protected MapEntryUpdateResult<String, byte[]> removeValue(Commit<? extends RemoveValue> commit) {
-        return removeIf(commit.index(), commit.value().key(), v ->
-            valuesEqual(v, new MapEntryValue(MapEntryValue.Type.VALUE, commit.index(), commit.value().value())));
-    }
-
-    /**
-     * Handles a removeVersion commit.
-     *
-     * @param commit removeVersion commit
-     * @return map entry update result
-     */
-    protected MapEntryUpdateResult<String, byte[]> removeVersion(Commit<? extends RemoveVersion> commit) {
-        return removeIf(commit.index(), commit.value().key(), v -> v.version() == commit.value().version());
-    }
-
-    /**
-     * Handles a replace commit.
-     *
-     * @param index     the commit index
-     * @param key       the key to replace
-     * @param newValue  the value with which to replace the key
-     * @param predicate a predicate to determine whether to replace the key
-     * @return map entry update result
-     */
-    private MapEntryUpdateResult<String, byte[]> replaceIf(
-        long index, String key, MapEntryValue newValue, Predicate<MapEntryValue> predicate) {
-        MapEntryValue oldValue = entries().get(key);
-
-        // If the key is not set or the current value doesn't match the predicate, return a PRECONDITION_FAILED error.
-        if (valueIsNull(oldValue) || !predicate.test(oldValue)) {
-            return new MapEntryUpdateResult<>(
-                MapEntryUpdateResult.Status.PRECONDITION_FAILED,
-                index,
-                key,
-                toVersioned(oldValue));
-        }
-
-        // If the key has been locked by a transaction, return a WRITE_LOCK error.
-        if (preparedKeys.contains(key)) {
-            return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.WRITE_LOCK, index, key, null);
-        }
-
-        entries().put(key, newValue);
-        Versioned<byte[]> result = toVersioned(oldValue);
-        publish(new MapEvent<>(MapEvent.Type.UPDATE, "", key, toVersioned(newValue), result));
-        return new MapEntryUpdateResult<>(MapEntryUpdateResult.Status.OK, index, key, result);
-    }
-
-    /**
-     * Handles a replace commit.
-     *
-     * @param commit replace commit
-     * @return map entry update result
-     */
-    protected MapEntryUpdateResult<String, byte[]> replace(Commit<? extends Replace> commit) {
-        MapEntryValue value = new MapEntryValue(MapEntryValue.Type.VALUE, commit.index(), commit.value().value());
-        return replaceIf(commit.index(), commit.value().key(), value, v -> true);
-    }
-
-    /**
-     * Handles a replaceValue commit.
-     *
-     * @param commit replaceValue commit
-     * @return map entry update result
-     */
-    protected MapEntryUpdateResult<String, byte[]> replaceValue(Commit<? extends ReplaceValue> commit) {
-        MapEntryValue value = new MapEntryValue(MapEntryValue.Type.VALUE, commit.index(), commit.value().newValue());
-        return replaceIf(commit.index(), commit.value().key(), value,
-            v -> valuesEqual(v.value(), commit.value().oldValue()));
-    }
-
-    /**
-     * Handles a replaceVersion commit.
-     *
-     * @param commit replaceVersion commit
-     * @return map entry update result
-     */
-    protected MapEntryUpdateResult<String, byte[]> replaceVersion(Commit<? extends ReplaceVersion> commit) {
-        MapEntryValue value = new MapEntryValue(MapEntryValue.Type.VALUE, commit.index(), commit.value().newValue());
-        return replaceIf(commit.index(), commit.value().key(), value,
-            v -> v.version() == commit.value().oldVersion());
-    }
-
-    /**
-     * Handles a clear commit.
-     *
-     * @return clear result
-     */
-    protected MapEntryUpdateResult.Status clear() {
-        Iterator<Map.Entry<String, MapEntryValue>> iterator = entries().entrySet().iterator();
-        Map<String, MapEntryValue> entriesToAdd = new HashMap<>();
-        while (iterator.hasNext()) {
-            Map.Entry<String, MapEntryValue> entry = iterator.next();
-            String key = entry.getKey();
-            MapEntryValue value = entry.getValue();
-            if (!valueIsNull(value)) {
-                Versioned<byte[]> removedValue = new Versioned<>(value.value(), value.version());
-                publish(new MapEvent<>(MapEvent.Type.REMOVE, "", key, null, removedValue));
-                if (activeTransactions.isEmpty()) {
-                    iterator.remove();
-                } else {
-                    entriesToAdd.put(key, new MapEntryValue(MapEntryValue.Type.TOMBSTONE, value.version, null));
-                }
-            }
-        }
-        entries().putAll(entriesToAdd);
-        return MapEntryUpdateResult.Status.OK;
-    }
-
-    /**
-     * Handles an open iterator commit.
-     *
-     * @param commit the open iterator commit
-     * @return iterator identifier
-     */
-    protected long openIterator(Commit<Void> commit) {
-        iterators.put(commit.index(), new IteratorContext(
-            commit.session().sessionId().id(),
-            entries().entrySet().iterator()));
-        return commit.index();
-    }
-
-    /**
-     * Handles an iterator next commit.
-     *
-     * @param commit the next commit
-     * @return a list of entries to iterate
-     */
-    protected IteratorBatch next(Commit<IteratorPosition> commit) {
-        final long iteratorId = commit.value().iteratorId();
-        final int position = commit.value().position();
-
-        IteratorContext context = iterators.get(iteratorId);
-        if (context == null) {
-            return null;
-        }
-
-        List<Map.Entry<String, Versioned<byte[]>>> entries = new ArrayList<>();
-        int size = 0;
-        while (context.iterator.hasNext()) {
-            context.position++;
-            if (context.position > position) {
-                Map.Entry<String, MapEntryValue> entry = context.iterator.next();
-                String key = entry.getKey();
-                Versioned<byte[]> value = toVersioned(entry.getValue());
-                size += key.length();
-                size += value.value() != null ? value.value().length : 0;
-                entries.add(Maps.immutableEntry(key, value));
-
-                if (size >= MAX_ITERATOR_BATCH_SIZE) {
-                    break;
-                }
-            }
-        }
-
-        if (entries.isEmpty()) {
-            return null;
-        }
-        return new IteratorBatch(context.position, entries);
-    }
-
-    /**
-     * Handles a close iterator commit.
-     *
-     * @param commit the close iterator commit
-     */
-    protected void closeIterator(Commit<Long> commit) {
-        iterators.remove(commit.value());
-    }
-
-    /**
-     * Handles a listen commit.
-     *
-     * @param session listen session
-     */
-    protected void listen(RaftSession session) {
-        listeners.put(session.sessionId().id(), session);
-    }
-
-    /**
-     * Handles an unlisten commit.
-     *
-     * @param session unlisten session
-     */
-    protected void unlisten(RaftSession session) {
-        listeners.remove(session.sessionId().id());
-    }
-
-    /**
-     * Handles a begin commit.
-     *
-     * @param commit transaction begin commit
-     * @return transaction state version
-     */
-    protected long begin(Commit<? extends TransactionBegin> commit) {
-        long version = commit.index();
-        activeTransactions.put(commit.value().transactionId(), new TransactionScope(version));
-        return version;
-    }
-
-    /**
-     * Handles an prepare and commit commit.
-     *
-     * @param commit transaction prepare and commit commit
-     * @return prepare result
-     */
-    protected PrepareResult prepareAndCommit(Commit<? extends TransactionPrepareAndCommit> commit) {
-        TransactionId transactionId = commit.value().transactionLog().transactionId();
-        PrepareResult prepareResult = prepare(commit);
-        TransactionScope transactionScope = activeTransactions.remove(transactionId);
-        if (prepareResult == PrepareResult.OK) {
-            this.currentVersion = commit.index();
-            transactionScope = transactionScope.prepared(commit);
-            commitTransaction(transactionScope);
-        }
-        discardTombstones();
-        return prepareResult;
-    }
-
-    /**
-     * Handles an prepare commit.
-     *
-     * @param commit transaction prepare commit
-     * @return prepare result
-     */
-    protected PrepareResult prepare(Commit<? extends TransactionPrepare> commit) {
-        try {
-            TransactionLog<MapUpdate<String, byte[]>> transactionLog = commit.value().transactionLog();
-
-            // Iterate through records in the transaction log and perform isolation checks.
-            for (MapUpdate<String, byte[]> record : transactionLog.records()) {
-                String key = record.key();
-
-                // If the record is a VERSION_MATCH then check that the record's version matches the current
-                // version of the state machine.
-                if (record.type() == MapUpdate.Type.VERSION_MATCH && key == null) {
-                    if (record.version() > currentVersion) {
-                        return PrepareResult.OPTIMISTIC_LOCK_FAILURE;
-                    } else {
-                        continue;
-                    }
-                }
-
-                // If the prepared keys already contains the key contained within the record, that indicates a
-                // conflict with a concurrent transaction.
-                if (preparedKeys.contains(key)) {
-                    return PrepareResult.CONCURRENT_TRANSACTION;
-                }
-
-                // Read the existing value from the map.
-                MapEntryValue existingValue = entries().get(key);
-
-                // Note: if the existing value is null, that means the key has not changed during the transaction,
-                // otherwise a tombstone would have been retained.
-                if (existingValue == null) {
-                    // If the value is null, ensure the version is equal to the transaction version.
-                    if (record.version() != transactionLog.version()) {
-                        return PrepareResult.OPTIMISTIC_LOCK_FAILURE;
-                    }
-                } else {
-                    // If the value is non-null, compare the current version with the record version.
-                    if (existingValue.version() > record.version()) {
-                        return PrepareResult.OPTIMISTIC_LOCK_FAILURE;
-                    }
-                }
-            }
-
-            // No violations detected. Mark modified keys locked for transactions.
-            transactionLog.records().forEach(record -> {
-                if (record.type() != MapUpdate.Type.VERSION_MATCH) {
-                    preparedKeys.add(record.key());
-                }
-            });
-
-            // Update the transaction scope. If the transaction scope is not set on this node, that indicates the
-            // coordinator is communicating with another node. Transactions assume that the client is communicating
-            // with a single leader in order to limit the overhead of retaining tombstones.
-            TransactionScope transactionScope = activeTransactions.get(transactionLog.transactionId());
-            if (transactionScope == null) {
-                activeTransactions.put(
-                    transactionLog.transactionId(),
-                    new TransactionScope(transactionLog.version(), commit.value().transactionLog()));
-                return PrepareResult.PARTIAL_FAILURE;
-            } else {
-                activeTransactions.put(
-                    transactionLog.transactionId(),
-                    transactionScope.prepared(commit));
-                return PrepareResult.OK;
-            }
-        } catch (Exception e) {
-            logger().warn("Failure applying {}", commit, e);
-            throw new IllegalStateException(e);
-        }
-    }
-
-    /**
-     * Handles an commit commit (ha!).
-     *
-     * @param commit transaction commit commit
-     * @return commit result
-     */
-    protected CommitResult commit(Commit<? extends TransactionCommit> commit) {
-        TransactionId transactionId = commit.value().transactionId();
-        TransactionScope transactionScope = activeTransactions.remove(transactionId);
-        if (transactionScope == null) {
-            return CommitResult.UNKNOWN_TRANSACTION_ID;
-        }
-
-        try {
-            this.currentVersion = commit.index();
-            return commitTransaction(transactionScope);
-        } catch (Exception e) {
-            logger().warn("Failure applying {}", commit, e);
-            throw new IllegalStateException(e);
-        } finally {
-            discardTombstones();
-        }
-    }
-
-    /**
-     * Applies committed operations to the state machine.
-     */
-    private CommitResult commitTransaction(TransactionScope transactionScope) {
-        TransactionLog<MapUpdate<String, byte[]>> transactionLog = transactionScope.transactionLog();
-        boolean retainTombstones = !activeTransactions.isEmpty();
-
-        List<MapEvent<String, byte[]>> eventsToPublish = Lists.newArrayList();
-        for (MapUpdate<String, byte[]> record : transactionLog.records()) {
-            if (record.type() == MapUpdate.Type.VERSION_MATCH) {
-                continue;
-            }
-
-            String key = record.key();
-            checkState(preparedKeys.remove(key), "key is not prepared");
-
-            if (record.type() == MapUpdate.Type.LOCK) {
-                continue;
-            }
-
-            MapEntryValue previousValue = entries().remove(key);
-            MapEntryValue newValue = null;
-
-            // If the record is not a delete, create a transactional commit.
-            if (record.type() != MapUpdate.Type.REMOVE_IF_VERSION_MATCH) {
-                newValue = new MapEntryValue(MapEntryValue.Type.VALUE, currentVersion, record.value());
-            } else if (retainTombstones) {
-                // For deletes, if tombstones need to be retained then create and store a tombstone commit.
-                newValue = new MapEntryValue(MapEntryValue.Type.TOMBSTONE, currentVersion, null);
-            }
-
-            MapEvent<String, byte[]> event;
-            if (newValue != null) {
-                entries().put(key, newValue);
-                if (!valueIsNull(newValue)) {
-                    if (!valueIsNull(previousValue)) {
-                        event = new MapEvent<>(
-                            MapEvent.Type.UPDATE,
-                            "",
-                            key,
-                            toVersioned(newValue),
-                            toVersioned(previousValue));
-                    } else {
-                        event = new MapEvent<>(
-                            MapEvent.Type.INSERT,
-                            "",
-                            key,
-                            toVersioned(newValue),
-                            null);
-                    }
-                } else {
-                    event = new MapEvent<>(
-                        MapEvent.Type.REMOVE,
-                        "",
-                        key,
-                        null,
-                        toVersioned(previousValue));
-                }
-            } else {
-                event = new MapEvent<>(
-                    MapEvent.Type.REMOVE,
-                    "",
-                    key,
-                    null,
-                    toVersioned(previousValue));
-            }
-            eventsToPublish.add(event);
-        }
-        publish(eventsToPublish);
-        return CommitResult.OK;
-    }
-
-    /**
-     * Handles an rollback commit (ha!).
-     *
-     * @param commit transaction rollback commit
-     * @return rollback result
-     */
-    protected RollbackResult rollback(Commit<? extends TransactionRollback> commit) {
-        TransactionId transactionId = commit.value().transactionId();
-        TransactionScope transactionScope = activeTransactions.remove(transactionId);
-        if (transactionScope == null) {
-            return RollbackResult.UNKNOWN_TRANSACTION_ID;
-        } else if (!transactionScope.isPrepared()) {
-            discardTombstones();
-            return RollbackResult.OK;
-        } else {
-            try {
-                transactionScope.transactionLog().records()
-                    .forEach(record -> {
-                        if (record.type() != MapUpdate.Type.VERSION_MATCH) {
-                            preparedKeys.remove(record.key());
-                        }
-                    });
-                return RollbackResult.OK;
-            } finally {
-                discardTombstones();
-            }
-        }
-
-    }
-
-    /**
-     * Discards tombstones no longer needed by active transactions.
-     */
-    private void discardTombstones() {
-        if (activeTransactions.isEmpty()) {
-            Iterator<Map.Entry<String, MapEntryValue>> iterator = entries().entrySet().iterator();
-            while (iterator.hasNext()) {
-                MapEntryValue value = iterator.next().getValue();
-                if (value.type() == MapEntryValue.Type.TOMBSTONE) {
-                    iterator.remove();
-                }
-            }
-        } else {
-            long lowWaterMark = activeTransactions.values().stream()
-                .mapToLong(TransactionScope::version)
-                .min().getAsLong();
-            Iterator<Map.Entry<String, MapEntryValue>> iterator = entries().entrySet().iterator();
-            while (iterator.hasNext()) {
-                MapEntryValue value = iterator.next().getValue();
-                if (value.type() == MapEntryValue.Type.TOMBSTONE && value.version < lowWaterMark) {
-                    iterator.remove();
-                }
-            }
-        }
-    }
-
-    /**
-     * Utility for turning a {@code MapEntryValue} to {@code Versioned}.
-     *
-     * @param value map entry value
-     * @return versioned instance
-     */
-    protected Versioned<byte[]> toVersioned(MapEntryValue value) {
-        return value != null && value.type() != MapEntryValue.Type.TOMBSTONE
-            ? new Versioned<>(value.value(), value.version()) : null;
-    }
-
-    /**
-     * Publishes an event to listeners.
-     *
-     * @param event event to publish
-     */
-    private void publish(MapEvent<String, byte[]> event) {
-        publish(Lists.newArrayList(event));
-    }
-
-    /**
-     * Publishes events to listeners.
-     *
-     * @param events list of map event to publish
-     */
-    private void publish(List<MapEvent<String, byte[]>> events) {
-        listeners.values().forEach(session -> {
-            session.publish(CHANGE, serializer()::encode, events);
-        });
-    }
-
-    @Override
-    public void onExpire(RaftSession session) {
-        closeListener(session.sessionId().id());
-    }
-
-    @Override
-    public void onClose(RaftSession session) {
-        closeListener(session.sessionId().id());
-    }
-
-    private void closeListener(Long sessionId) {
-        listeners.remove(sessionId);
-    }
-
-    /**
-     * Interface implemented by map values.
-     */
-    protected static class MapEntryValue {
-        protected final Type type;
-        protected final long version;
-        protected final byte[] value;
-
-        MapEntryValue(Type type, long version, byte[] value) {
-            this.type = type;
-            this.version = version;
-            this.value = value;
-        }
-
-        /**
-         * Returns the value type.
-         *
-         * @return the value type
-         */
-        Type type() {
-            return type;
-        }
-
-        /**
-         * Returns the version of the value.
-         *
-         * @return version
-         */
-        long version() {
-            return version;
-        }
-
-        /**
-         * Returns the raw {@code byte[]}.
-         *
-         * @return raw value
-         */
-        byte[] value() {
-            return value;
-        }
-
-        /**
-         * Value type.
-         */
-        enum Type {
-            VALUE,
-            TOMBSTONE,
-        }
-    }
-
-    /**
-     * Map transaction scope.
-     */
-    protected static final class TransactionScope {
-        private final long version;
-        private final TransactionLog<MapUpdate<String, byte[]>> transactionLog;
-
-        private TransactionScope(long version) {
-            this(version, null);
-        }
-
-        private TransactionScope(long version, TransactionLog<MapUpdate<String, byte[]>> transactionLog) {
-            this.version = version;
-            this.transactionLog = transactionLog;
-        }
-
-        /**
-         * Returns the transaction version.
-         *
-         * @return the transaction version
-         */
-        long version() {
-            return version;
-        }
-
-        /**
-         * Returns whether this is a prepared transaction scope.
-         *
-         * @return whether this is a prepared transaction scope
-         */
-        boolean isPrepared() {
-            return transactionLog != null;
-        }
-
-        /**
-         * Returns the transaction commit log.
-         *
-         * @return the transaction commit log
-         */
-        TransactionLog<MapUpdate<String, byte[]>> transactionLog() {
-            checkState(isPrepared());
-            return transactionLog;
-        }
-
-        /**
-         * Returns a new transaction scope with a prepare commit.
-         *
-         * @param commit the prepare commit
-         * @return new transaction scope updated with the prepare commit
-         */
-        TransactionScope prepared(Commit<? extends TransactionPrepare> commit) {
-            return new TransactionScope(version, commit.value().transactionLog());
-        }
-    }
-
-    private static class IteratorContext {
-        private final long sessionId;
-        private final Iterator<Map.Entry<String, MapEntryValue>> iterator;
-        private int position = 0;
-
-        IteratorContext(long sessionId, Iterator<Map.Entry<String, MapEntryValue>> iterator) {
-            this.sessionId = sessionId;
-            this.iterator = iterator;
-        }
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimap.java
deleted file mode 100644
index b04c9dd..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimap.java
+++ /dev/null
@@ -1,354 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.Collection;
-import java.util.Collections;
-import java.util.ConcurrentModificationException;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.NoSuchElementException;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executor;
-import java.util.function.Function;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Multiset;
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Tools;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncConsistentMultimap;
-import org.onosproject.store.service.AsyncIterator;
-import org.onosproject.store.service.MultimapEvent;
-import org.onosproject.store.service.MultimapEventListener;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.Versioned;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapEvents.CHANGE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.ADD_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.CLEAR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.CLOSE_ITERATOR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.CONTAINS_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.CONTAINS_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.CONTAINS_VALUE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.ContainsEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.ContainsKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.ContainsValue;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.ENTRIES;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.Get;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.IS_EMPTY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.IteratorBatch;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.IteratorPosition;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.KEYS;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.KEY_SET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.MultiRemove;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.NEXT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.OPEN_ITERATOR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.PUT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.PUT_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.Put;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.REMOVE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.REMOVE_ALL;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.REMOVE_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.REMOVE_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.REPLACE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.RemoveAll;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.Replace;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.SIZE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.VALUES;
-
-
-/**
- * Set based implementation of the {@link AsyncConsistentMultimap}.
- * <p>
- * Note: this implementation does not allow null entries or duplicate entries.
- */
-public class AtomixConsistentSetMultimap
-    extends AbstractRaftPrimitive
-    implements AsyncConsistentMultimap<String, byte[]> {
-
-    private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
-        .register(KryoNamespaces.BASIC)
-        .register(AtomixConsistentSetMultimapOperations.NAMESPACE)
-        .register(AtomixConsistentSetMultimapEvents.NAMESPACE)
-        .build());
-
-    private final Map<MultimapEventListener<String, byte[]>, Executor> mapEventListeners = new ConcurrentHashMap<>();
-
-    public AtomixConsistentSetMultimap(RaftProxy proxy) {
-        super(proxy);
-        proxy.addEventListener(CHANGE, SERIALIZER::decode, this::handleChange);
-        proxy.addStateChangeListener(state -> {
-            if (state == RaftProxy.State.CONNECTED && isListening()) {
-                proxy.invoke(ADD_LISTENER);
-            }
-        });
-    }
-
-    private void handleChange(List<MultimapEvent<String, byte[]>> events) {
-        events.forEach(event ->
-            mapEventListeners.forEach((listener, executor) -> executor.execute(() -> listener.event(event))));
-    }
-
-    @Override
-    public CompletableFuture<Integer> size() {
-        return proxy.invoke(SIZE, SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> isEmpty() {
-        return proxy.invoke(IS_EMPTY, SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsKey(String key) {
-        return proxy.invoke(CONTAINS_KEY, SERIALIZER::encode, new ContainsKey(key), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsValue(byte[] value) {
-        return proxy.invoke(CONTAINS_VALUE, SERIALIZER::encode, new ContainsValue(value), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> containsEntry(String key, byte[] value) {
-        return proxy.invoke(CONTAINS_ENTRY, SERIALIZER::encode, new ContainsEntry(key, value), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> put(String key, byte[] value) {
-        return proxy.invoke(
-            PUT,
-            SERIALIZER::encode,
-            new Put(key, Lists.newArrayList(value), null),
-            SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends byte[]>>> putAndGet(String key, byte[] value) {
-        return proxy.invoke(
-            PUT_AND_GET,
-            SERIALIZER::encode,
-            new Put(key, Lists.newArrayList(value), null),
-            SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> remove(String key, byte[] value) {
-        return proxy.invoke(REMOVE, SERIALIZER::encode, new MultiRemove(key,
-            Lists.newArrayList(value),
-            null), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends byte[]>>> removeAndGet(String key, byte[] value) {
-        return proxy.invoke(REMOVE_AND_GET, SERIALIZER::encode, new MultiRemove(key,
-            Lists.newArrayList(value),
-            null), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> removeAll(String key, Collection<? extends byte[]> values) {
-        return proxy.invoke(
-            REMOVE,
-            SERIALIZER::encode,
-            new MultiRemove(key, (Collection<byte[]>) values, null),
-            SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends byte[]>>> removeAll(String key) {
-        return proxy.invoke(REMOVE_ALL, SERIALIZER::encode, new RemoveAll(key, null), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> putAll(
-        String key, Collection<? extends byte[]> values) {
-        return proxy.invoke(PUT, SERIALIZER::encode, new Put(key, values, null), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends byte[]>>> replaceValues(
-        String key, Collection<byte[]> values) {
-        return proxy.invoke(
-            REPLACE,
-            SERIALIZER::encode,
-            new Replace(key, values, null),
-            SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Void> clear() {
-        return proxy.invoke(CLEAR);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<Collection<? extends byte[]>>> get(String key) {
-        return proxy.invoke(GET, SERIALIZER::encode, new Get(key), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Set<String>> keySet() {
-        return proxy.invoke(KEY_SET, SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Multiset<String>> keys() {
-        return proxy.invoke(KEYS, SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Multiset<byte[]>> values() {
-        return proxy.invoke(VALUES, SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Collection<Map.Entry<String, byte[]>>> entries() {
-        return proxy.invoke(ENTRIES, SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<AsyncIterator<Map.Entry<String, byte[]>>> iterator() {
-        return proxy.<Long>invoke(OPEN_ITERATOR, SERIALIZER::decode).thenApply(ConsistentMultimapIterator::new);
-    }
-
-    /**
-     * Consistent multimap iterator.
-     */
-    private class ConsistentMultimapIterator implements AsyncIterator<Map.Entry<String, byte[]>> {
-        private final long id;
-        private volatile CompletableFuture<IteratorBatch> batch;
-        private volatile CompletableFuture<Void> closeFuture;
-
-        ConsistentMultimapIterator(long id) {
-            this.id = id;
-            this.batch = CompletableFuture.completedFuture(
-                new IteratorBatch(0, Collections.emptyList()));
-        }
-
-        /**
-         * Returns the current batch iterator or lazily fetches the next batch from the cluster.
-         *
-         * @return the next batch iterator
-         */
-        private CompletableFuture<Iterator<Map.Entry<String, byte[]>>> batch() {
-            return batch.thenCompose(iterator -> {
-                if (iterator != null && !iterator.hasNext()) {
-                    batch = fetch(iterator.position());
-                    return batch.thenApply(Function.identity());
-                }
-                return CompletableFuture.completedFuture(iterator);
-            });
-        }
-
-        /**
-         * Fetches the next batch of entries from the cluster.
-         *
-         * @param position the position from which to fetch the next batch
-         * @return the next batch of entries from the cluster
-         */
-        private CompletableFuture<IteratorBatch> fetch(int position) {
-            return proxy.<IteratorPosition, IteratorBatch>invoke(
-                NEXT,
-                SERIALIZER::encode,
-                new IteratorPosition(id, position),
-                SERIALIZER::decode)
-                .thenCompose(batch -> {
-                    if (batch == null) {
-                        return close().thenApply(v -> null);
-                    }
-                    return CompletableFuture.completedFuture(batch);
-                });
-        }
-
-        /**
-         * Closes the iterator.
-         *
-         * @return future to be completed once the iterator has been closed
-         */
-        private CompletableFuture<Void> close() {
-            if (closeFuture == null) {
-                synchronized (this) {
-                    if (closeFuture == null) {
-                        closeFuture = proxy.invoke(CLOSE_ITERATOR, SERIALIZER::encode, id);
-                    }
-                }
-            }
-            return closeFuture;
-        }
-
-        @Override
-        public CompletableFuture<Boolean> hasNext() {
-            return batch().thenApply(iterator -> iterator != null && iterator.hasNext());
-        }
-
-        @Override
-        public CompletableFuture<Map.Entry<String, byte[]>> next() {
-            return batch().thenCompose(iterator -> {
-                if (iterator == null) {
-                    return Tools.exceptionalFuture(new NoSuchElementException());
-                }
-                return CompletableFuture.completedFuture(iterator.next());
-            });
-        }
-    }
-
-    @Override
-    public CompletableFuture<Void> addListener(MultimapEventListener<String, byte[]> listener, Executor executor) {
-        if (mapEventListeners.isEmpty()) {
-            return proxy.invoke(ADD_LISTENER).thenRun(() -> mapEventListeners.put(listener, executor));
-        } else {
-            mapEventListeners.put(listener, executor);
-            return CompletableFuture.completedFuture(null);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(MultimapEventListener<String, byte[]> listener) {
-        if (mapEventListeners.remove(listener) != null && mapEventListeners.isEmpty()) {
-            return proxy.invoke(REMOVE_LISTENER).thenApply(v -> null);
-        }
-        return CompletableFuture.completedFuture(null);
-    }
-
-    @Override
-    public CompletableFuture<Map<String, Collection<byte[]>>> asMap() {
-        throw new UnsupportedOperationException("Expensive operation.");
-    }
-
-    /**
-     * Helper to check if there was a lock based issue.
-     *
-     * @param status the status of an update result
-     */
-    private void throwIfLocked(MapEntryUpdateResult.Status status) {
-        if (status == MapEntryUpdateResult.Status.WRITE_LOCK) {
-            throw new ConcurrentModificationException("Cannot update map: " +
-                "Another transaction " +
-                "in progress");
-        }
-    }
-
-    private boolean isListening() {
-        return !mapEventListeners.isEmpty();
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapEvents.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapEvents.java
deleted file mode 100644
index 83908ae..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapEvents.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import io.atomix.protocols.raft.event.EventType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.MultimapEvent;
-
-/**
- * Atomix consistent set multimap events.
- */
-public enum AtomixConsistentSetMultimapEvents implements EventType {
-    CHANGE,
-    ENTRY;
-
-    @Override
-    public String id() {
-        return name();
-    }
-
-    public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
-            .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 50)
-            .register(MultimapEvent.class)
-            .register(MultimapEvent.Type.class)
-            .register(byte[].class)
-            .build("AtomixConsistentSetMultimapEvents");
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapOperations.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapOperations.java
deleted file mode 100644
index 4d38d71..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapOperations.java
+++ /dev/null
@@ -1,464 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.Map;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.Maps;
-import io.atomix.protocols.raft.operation.OperationId;
-import io.atomix.protocols.raft.operation.OperationType;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Match;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.Versioned;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * AsyncConsistentMultimap state machine commands.
- */
-public enum AtomixConsistentSetMultimapOperations implements OperationId {
-    GET(OperationType.QUERY),
-    SIZE(OperationType.QUERY),
-    IS_EMPTY(OperationType.QUERY),
-    CONTAINS_KEY(OperationType.QUERY),
-    CONTAINS_VALUE(OperationType.QUERY),
-    CONTAINS_ENTRY(OperationType.QUERY),
-    KEY_SET(OperationType.QUERY),
-    KEYS(OperationType.QUERY),
-    VALUES(OperationType.QUERY),
-    ENTRIES(OperationType.QUERY),
-    PUT(OperationType.COMMAND),
-    PUT_AND_GET(OperationType.COMMAND),
-    REMOVE(OperationType.COMMAND),
-    REMOVE_AND_GET(OperationType.COMMAND),
-    REMOVE_ALL(OperationType.COMMAND),
-    REPLACE(OperationType.COMMAND),
-    CLEAR(OperationType.COMMAND),
-    ADD_LISTENER(OperationType.COMMAND),
-    REMOVE_LISTENER(OperationType.COMMAND),
-    OPEN_ITERATOR(OperationType.COMMAND),
-    NEXT(OperationType.QUERY),
-    CLOSE_ITERATOR(OperationType.COMMAND);
-
-    private final OperationType type;
-
-    AtomixConsistentSetMultimapOperations(OperationType type) {
-        this.type = type;
-    }
-
-    @Override
-    public String id() {
-        return name();
-    }
-
-    @Override
-    public OperationType type() {
-        return type;
-    }
-
-    public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
-            .register(KryoNamespaces.BASIC)
-            .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
-            .register(ContainsEntry.class)
-            .register(ContainsKey.class)
-            .register(ContainsValue.class)
-            .register(Get.class)
-            .register(MultiRemove.class)
-            .register(Put.class)
-            .register(RemoveAll.class)
-            .register(Replace.class)
-            .register(Match.class)
-            .register(Versioned.class)
-            .register(ArrayList.class)
-            .register(Maps.immutableEntry("", "").getClass())
-            .register(IteratorBatch.class)
-            .register(IteratorPosition.class)
-            .build("AtomixConsistentSetMultimapOperations");
-
-    /**
-     * Abstract multimap command.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class MultimapOperation {
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .toString();
-        }
-    }
-
-    /**
-     * Abstract key-based multimap query.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class KeyOperation extends MultimapOperation {
-        protected String key;
-
-        public KeyOperation() {
-        }
-
-        public KeyOperation(String key) {
-            this.key = checkNotNull(key);
-        }
-
-        public String key() {
-            return key;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("key", key)
-                    .toString();
-        }
-    }
-
-    /**
-     * Abstract value-based query.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class ValueOperation extends MultimapOperation {
-        protected byte[] value;
-
-        public ValueOperation() {
-        }
-
-        public ValueOperation(byte[] value) {
-            this.value = checkNotNull(value);
-        }
-
-        /**
-         * Returns the value.
-         *
-         * @return value.
-         */
-        public byte[] value() {
-            return value;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("value", value)
-                    .toString();
-        }
-    }
-
-    /**
-     * Contains key query.
-     */
-    @SuppressWarnings("serial")
-    public static class ContainsKey extends KeyOperation {
-        public ContainsKey() {
-        }
-
-        public ContainsKey(String key) {
-            super(key);
-        }
-    }
-
-    /**
-     * Contains value query.
-     */
-    @SuppressWarnings("serial")
-    public static class ContainsValue extends ValueOperation {
-        public ContainsValue() {
-        }
-
-        public ContainsValue(byte[] value) {
-            super(value);
-        }
-    }
-
-    /**
-     * Contains entry query.
-     */
-    @SuppressWarnings("serial")
-    public static class ContainsEntry extends MultimapOperation {
-        protected String key;
-        protected byte[] value;
-
-        public ContainsEntry() {
-        }
-
-        public ContainsEntry(String key, byte[] value) {
-            this.key = checkNotNull(key);
-            this.value = checkNotNull(value);
-        }
-
-        public String key() {
-            return key;
-        }
-
-        public byte[] value() {
-            return value;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("key", key)
-                    .add("value", value)
-                    .toString();
-        }
-    }
-
-    /**
-     * Remove command, backs remove and removeAll's that return booleans.
-     */
-    @SuppressWarnings("serial")
-    public static class RemoveAll extends MultimapOperation {
-        private String key;
-        private Match<Long> versionMatch;
-
-        public RemoveAll() {
-        }
-
-        public RemoveAll(String key, Match<Long> versionMatch) {
-            this.key = checkNotNull(key);
-            this.versionMatch = versionMatch;
-        }
-
-        public String key() {
-            return this.key;
-        }
-
-        public Match<Long> versionMatch() {
-            return versionMatch;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("key", key)
-                    .add("versionMatch", versionMatch)
-                    .toString();
-        }
-    }
-
-    /**
-     * Remove command, backs remove and removeAll's that return booleans.
-     */
-    @SuppressWarnings("serial")
-    public static class MultiRemove extends MultimapOperation {
-        private String key;
-        private Collection<byte[]> values;
-        private Match<Long> versionMatch;
-
-        public MultiRemove() {
-        }
-
-        public MultiRemove(String key, Collection<byte[]> valueMatches,
-                           Match<Long> versionMatch) {
-            this.key = checkNotNull(key);
-            this.values = valueMatches;
-            this.versionMatch = versionMatch;
-        }
-
-        public String key() {
-            return this.key;
-        }
-
-        public Collection<byte[]> values() {
-            return values;
-        }
-
-        public Match<Long> versionMatch() {
-            return versionMatch;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("key", key)
-                    .add("values", values)
-                    .add("versionMatch", versionMatch)
-                    .toString();
-        }
-    }
-
-    /**
-     * Command to back the put and putAll methods.
-     */
-    @SuppressWarnings("serial")
-    public static class  Put extends MultimapOperation {
-        private String key;
-        private Collection<? extends byte[]> values;
-        private Match<Long> versionMatch;
-
-        public Put() {
-        }
-
-        public Put(String key, Collection<? extends byte[]> values, Match<Long> versionMatch) {
-            this.key = checkNotNull(key);
-            this.values = values;
-            this.versionMatch = versionMatch;
-        }
-
-        public String key() {
-            return key;
-        }
-
-        public Collection<? extends byte[]> values() {
-            return values;
-        }
-
-        public Match<Long> versionMatch() {
-            return versionMatch;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("key", key)
-                    .add("values", values)
-                    .add("versionMatch", versionMatch)
-                    .toString();
-        }
-    }
-
-    /**
-     * Replace command, returns the collection that was replaced.
-     */
-    @SuppressWarnings("serial")
-    public static class Replace extends MultimapOperation {
-        private String key;
-        private Collection<byte[]> values;
-        private Match<Long> versionMatch;
-
-        public Replace() {
-        }
-
-        public Replace(String key, Collection<byte[]> values,
-                       Match<Long> versionMatch) {
-            this.key = checkNotNull(key);
-            this.values = values;
-            this.versionMatch = versionMatch;
-        }
-
-        public String key() {
-            return this.key;
-        }
-
-        public Match<Long> versionMatch() {
-            return versionMatch;
-        }
-
-        public Collection<byte[]> values() {
-            return values;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("key", key)
-                    .add("values", values)
-                    .add("versionMatch", versionMatch)
-                    .toString();
-        }
-    }
-
-    /**
-     * Get value query.
-     */
-    public static class Get extends KeyOperation {
-        public Get() {
-        }
-
-        public Get(String key) {
-            super(key);
-        }
-    }
-
-    /**
-     * Iterator position.
-     */
-    public static class IteratorPosition {
-        private long iteratorId;
-        private int position;
-
-        private IteratorPosition() {
-        }
-
-        public IteratorPosition(long iteratorId, int position) {
-            this.iteratorId = iteratorId;
-            this.position = position;
-        }
-
-        public long iteratorId() {
-            return iteratorId;
-        }
-
-        public int position() {
-            return position;
-        }
-    }
-
-    /**
-     * Iterator batch.
-     */
-    public static class IteratorBatch implements Iterator<Map.Entry<String, byte[]>> {
-        private int position;
-        private Collection<Map.Entry<String, byte[]>> entries;
-        private transient volatile Iterator<Map.Entry<String, byte[]>> iterator;
-
-        private IteratorBatch() {
-        }
-
-        public IteratorBatch(int position, Collection<Map.Entry<String, byte[]>> entries) {
-            this.position = position;
-            this.entries = entries;
-        }
-
-        public int position() {
-            return position;
-        }
-
-        public Collection<Map.Entry<String, byte[]>> entries() {
-            return entries;
-        }
-
-        private Iterator<Map.Entry<String, byte[]>> iterator() {
-            Iterator<Map.Entry<String, byte[]>> iterator = this.iterator;
-            if (iterator == null) {
-                synchronized (entries) {
-                    iterator = this.iterator;
-                    if (iterator == null) {
-                        iterator = entries.iterator();
-                        this.iterator = iterator;
-                    }
-                }
-            }
-            return iterator;
-        }
-
-        @Override
-        public boolean hasNext() {
-            return iterator().hasNext();
-        }
-
-        @Override
-        public Map.Entry<String, byte[]> next() {
-            return iterator().next();
-        }
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapService.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapService.java
deleted file mode 100644
index fd2e676..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapService.java
+++ /dev/null
@@ -1,884 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Comparator;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.function.BiConsumer;
-import java.util.function.BinaryOperator;
-import java.util.function.Function;
-import java.util.function.Supplier;
-import java.util.stream.Collector;
-import java.util.stream.Collectors;
-
-import com.esotericsoftware.kryo.Kryo;
-import com.esotericsoftware.kryo.io.Input;
-import com.esotericsoftware.kryo.io.Output;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.HashMultiset;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Multiset;
-import com.google.common.collect.Sets;
-import io.atomix.protocols.raft.service.AbstractRaftService;
-import io.atomix.protocols.raft.service.Commit;
-import io.atomix.protocols.raft.service.RaftServiceExecutor;
-import io.atomix.protocols.raft.session.RaftSession;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Match;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.MultimapEvent;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.Versioned;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapEvents.CHANGE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.ADD_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.CLEAR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.CLOSE_ITERATOR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.CONTAINS_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.CONTAINS_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.CONTAINS_VALUE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.ContainsEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.ContainsKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.ContainsValue;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.ENTRIES;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.Get;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.IS_EMPTY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.IteratorBatch;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.IteratorPosition;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.KEYS;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.KEY_SET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.MultiRemove;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.MultimapOperation;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.NEXT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.OPEN_ITERATOR;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.PUT;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.PUT_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.Put;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.REMOVE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.REMOVE_ALL;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.REMOVE_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.REMOVE_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.REPLACE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.RemoveAll;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.Replace;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.SIZE;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentSetMultimapOperations.VALUES;
-
-/**
- * State Machine for {@link AtomixConsistentSetMultimap} resource.
- */
-public class AtomixConsistentSetMultimapService extends AbstractRaftService {
-    private static final int MAX_ITERATOR_BATCH_SIZE = 1024 * 32;
-
-    private final Serializer serializer = Serializer.using(KryoNamespace.newBuilder()
-            .register(KryoNamespaces.BASIC)
-            .register(AtomixConsistentSetMultimapOperations.NAMESPACE)
-            .register(AtomixConsistentSetMultimapEvents.NAMESPACE)
-            .register(ByteArrayComparator.class)
-            .register(new HashMap().keySet().getClass())
-            .register(TreeSet.class)
-            .register(new com.esotericsoftware.kryo.Serializer<NonTransactionalCommit>() {
-                @Override
-                public void write(Kryo kryo, Output output, NonTransactionalCommit object) {
-                    kryo.writeClassAndObject(output, object.valueSet);
-                }
-
-                @Override
-                @SuppressWarnings("unchecked")
-                public NonTransactionalCommit read(Kryo kryo, Input input, Class<NonTransactionalCommit> type) {
-                    NonTransactionalCommit commit = new NonTransactionalCommit();
-                    commit.valueSet.addAll((Collection<byte[]>) kryo.readClassAndObject(input));
-                    return commit;
-                }
-            }, NonTransactionalCommit.class)
-            .build());
-
-    private AtomicLong globalVersion = new AtomicLong(1);
-    private Map<Long, RaftSession> listeners = new LinkedHashMap<>();
-    private Map<String, MapEntryValue> backingMap = Maps.newConcurrentMap();
-    private Map<Long, IteratorContext> iterators = Maps.newHashMap();
-
-    @Override
-    public void snapshot(SnapshotWriter writer) {
-        writer.writeLong(globalVersion.get());
-        writer.writeObject(Sets.newHashSet(listeners.keySet()), serializer::encode);
-        writer.writeObject(backingMap, serializer::encode);
-
-        Map<Long, Long> iterators = Maps.newHashMap();
-        this.iterators.forEach((id, context) -> iterators.put(id, context.sessionId));
-        writer.writeObject(iterators, serializer::encode);
-    }
-
-    @Override
-    public void install(SnapshotReader reader) {
-        globalVersion = new AtomicLong(reader.readLong());
-
-        listeners = new LinkedHashMap<>();
-        for (Long sessionId : reader.<Set<Long>>readObject(serializer::decode)) {
-            listeners.put(sessionId, sessions().getSession(sessionId));
-        }
-
-        backingMap = reader.readObject(serializer::decode);
-
-        Map<Long, Long> iterators = reader.readObject(serializer::decode);
-        this.iterators = Maps.newHashMap();
-        iterators.forEach((id, session) ->
-            this.iterators.put(id, new IteratorContext(session, backingMap.entrySet().iterator())));
-    }
-
-    @Override
-    protected void configure(RaftServiceExecutor executor) {
-        executor.register(SIZE, this::size, serializer::encode);
-        executor.register(IS_EMPTY, this::isEmpty, serializer::encode);
-        executor.register(CONTAINS_KEY, serializer::decode, this::containsKey, serializer::encode);
-        executor.register(CONTAINS_VALUE, serializer::decode, this::containsValue, serializer::encode);
-        executor.register(CONTAINS_ENTRY, serializer::decode, this::containsEntry, serializer::encode);
-        executor.register(CLEAR, this::clear);
-        executor.register(KEY_SET, this::keySet, serializer::encode);
-        executor.register(KEYS, this::keys, serializer::encode);
-        executor.register(VALUES, this::values, serializer::encode);
-        executor.register(ENTRIES, this::entries, serializer::encode);
-        executor.register(GET, serializer::decode, this::get, serializer::encode);
-        executor.register(REMOVE_ALL, serializer::decode, this::removeAll, serializer::encode);
-        executor.register(REMOVE, serializer::decode, this::multiRemove, serializer::encode);
-        executor.register(REMOVE_AND_GET, serializer::decode, this::removeAndGet, serializer::encode);
-        executor.register(PUT, serializer::decode, this::put, serializer::encode);
-        executor.register(PUT_AND_GET, serializer::decode, this::putAndGet, serializer::encode);
-        executor.register(REPLACE, serializer::decode, this::replace, serializer::encode);
-        executor.register(ADD_LISTENER, this::listen);
-        executor.register(REMOVE_LISTENER, this::unlisten);
-        executor.register(OPEN_ITERATOR, this::openIterator, serializer::encode);
-        executor.register(NEXT, serializer::decode, this::next, serializer::encode);
-        executor.register(CLOSE_ITERATOR, serializer::decode, this::closeIterator);
-    }
-
-    @Override
-    public void onExpire(RaftSession session) {
-        listeners.remove(session.sessionId().id());
-        iterators.entrySet().removeIf(entry -> entry.getValue().sessionId == session.sessionId().id());
-    }
-
-    @Override
-    public void onClose(RaftSession session) {
-        listeners.remove(session.sessionId().id());
-        iterators.entrySet().removeIf(entry -> entry.getValue().sessionId == session.sessionId().id());
-    }
-
-    /**
-     * Handles a Size commit.
-     *
-     * @param commit Size commit
-     * @return number of unique key value pairs in the multimap
-     */
-    protected int size(Commit<Void> commit) {
-        return backingMap.values()
-                .stream()
-                .map(valueCollection -> valueCollection.values().size())
-                .collect(Collectors.summingInt(size -> size));
-    }
-
-    /**
-     * Handles an IsEmpty commit.
-     *
-     * @param commit IsEmpty commit
-     * @return true if the multimap contains no key-value pairs, else false
-     */
-    protected boolean isEmpty(Commit<Void> commit) {
-        return backingMap.isEmpty();
-    }
-
-    /**
-     * Handles a contains key commit.
-     *
-     * @param commit ContainsKey commit
-     * @return returns true if the key is in the multimap, else false
-     */
-    protected boolean containsKey(Commit<? extends ContainsKey> commit) {
-        return backingMap.containsKey(commit.value().key());
-    }
-
-    /**
-     * Handles a ContainsValue commit.
-     *
-     * @param commit ContainsValue commit
-     * @return true if the value is in the multimap, else false
-     */
-    protected boolean containsValue(Commit<? extends ContainsValue> commit) {
-        if (backingMap.values().isEmpty()) {
-            return false;
-        }
-        Match<byte[]> match = Match.ifValue(commit.value().value());
-        return backingMap
-                .values()
-                .stream()
-                .anyMatch(valueList ->
-                        valueList
-                                .values()
-                                .stream()
-                                .anyMatch(byteValue ->
-                                        match.matches(byteValue)));
-    }
-
-    /**
-     * Handles a ContainsEntry commit.
-     *
-     * @param commit ContainsEntry commit
-     * @return true if the key-value pair exists, else false
-     */
-    protected boolean containsEntry(Commit<? extends ContainsEntry> commit) {
-        MapEntryValue entryValue =
-                backingMap.get(commit.value().key());
-        if (entryValue == null) {
-            return false;
-        } else {
-            Match valueMatch = Match.ifValue(commit.value().value());
-            return entryValue
-                    .values()
-                    .stream()
-                    .anyMatch(byteValue -> valueMatch.matches(byteValue));
-        }
-    }
-
-    /**
-     * Handles a Clear commit.
-     *
-     * @param commit Clear commit
-     */
-    protected void clear(Commit<Void> commit) {
-        backingMap.clear();
-    }
-
-    /**
-     * Handles a KeySet commit.
-     *
-     * @param commit KeySet commit
-     * @return a set of all keys in the multimap
-     */
-    protected Set<String> keySet(Commit<Void> commit) {
-        return ImmutableSet.copyOf(backingMap.keySet());
-    }
-
-    /**
-     * Handles a Keys commit.
-     *
-     * @param commit Keys commit
-     * @return a multiset of keys with each key included an equal number of
-     * times to the total key-value pairs in which that key participates
-     */
-    protected Multiset<String> keys(Commit<Void> commit) {
-        Multiset keys = HashMultiset.create();
-        backingMap.forEach((key, mapEntryValue) -> {
-            keys.add(key, mapEntryValue.values().size());
-        });
-        return keys;
-    }
-
-    /**
-     * Handles a Values commit.
-     *
-     * @param commit Values commit
-     * @return the set of values in the multimap with duplicates included
-     */
-    protected Multiset<byte[]> values(Commit<Void> commit) {
-        return backingMap
-                .values()
-                .stream()
-                .collect(new HashMultisetValueCollector());
-    }
-
-    /**
-     * Handles an Entries commit.
-     *
-     * @param commit Entries commit
-     * @return a set of all key-value pairs in the multimap
-     */
-    protected Collection<Map.Entry<String, byte[]>> entries(Commit<Void> commit) {
-        return backingMap
-                .entrySet()
-                .stream()
-                .collect(new EntrySetCollector());
-    }
-
-    /**
-     * Handles a Get commit.
-     *
-     * @param commit Get commit
-     * @return the collection of values associated with the key or an empty
-     * list if none exist
-     */
-    protected Versioned<Collection<? extends byte[]>> get(Commit<? extends Get> commit) {
-        return toVersioned(backingMap.get(commit.value().key()));
-    }
-
-    /**
-     * Handles a removeAll commit, and returns the previous mapping.
-     *
-     * @param commit removeAll commit
-     * @return collection of removed values
-     */
-    protected Versioned<Collection<? extends byte[]>> removeAll(Commit<? extends RemoveAll> commit) {
-        String key = commit.value().key();
-
-        if (!backingMap.containsKey(key)) {
-            return new Versioned<>(Sets.newHashSet(), -1);
-        }
-
-        Versioned<Collection<? extends byte[]>> removedValues =
-                backingMap.get(key).addCommit(commit);
-        publish(removedValues.value().stream()
-                .map(value -> new MultimapEvent<String, byte[]>(
-                        "", key, null, value))
-                .collect(Collectors.toList()));
-        return removedValues;
-    }
-
-    /**
-     * Handles a multiRemove commit, returns true if the remove results in any
-     * change.
-     * @param commit multiRemove commit
-     * @return true if any change results, else false
-     */
-    protected boolean multiRemove(Commit<? extends MultiRemove> commit) {
-        String key = commit.value().key();
-
-        if (!backingMap.containsKey(key)) {
-            return false;
-        }
-
-        Versioned<Collection<? extends byte[]>> removedValues = backingMap
-            .get(key)
-            .addCommit(commit);
-
-        if (removedValues != null) {
-            if (removedValues.value().isEmpty()) {
-                backingMap.remove(key);
-            }
-
-            publish(removedValues.value().stream()
-                .map(value -> new MultimapEvent<String, byte[]>(
-                    "", key, null, value))
-                .collect(Collectors.toList()));
-            return true;
-        }
-
-        return false;
-    }
-
-    /**
-     * Handles a removeAndGet commit.
-     *
-     * @param commit multiRemove commit
-     * @return the updated values or null if the values are empty
-     */
-    protected Versioned<Collection<? extends byte[]>> removeAndGet(Commit<? extends MultiRemove> commit) {
-        String key = commit.value().key();
-
-        if (!backingMap.containsKey(key)) {
-            return null;
-        }
-
-        Versioned<Collection<? extends byte[]>> removedValues = backingMap
-            .get(key)
-            .addCommit(commit);
-
-        if (removedValues != null) {
-            if (removedValues.value().isEmpty()) {
-                backingMap.remove(key);
-            }
-
-            publish(removedValues.value().stream()
-                .map(value -> new MultimapEvent<String, byte[]>(
-                    "", key, null, value))
-                .collect(Collectors.toList()));
-        }
-
-        return toVersioned(backingMap.get(key));
-    }
-
-    /**
-     * Handles a put commit, returns true if any change results from this
-     * commit.
-     * @param commit a put commit
-     * @return true if this commit results in a change, else false
-     */
-    protected boolean put(Commit<? extends Put> commit) {
-        String key = commit.value().key();
-        if (commit.value().values().isEmpty()) {
-            return false;
-        }
-        if (!backingMap.containsKey(key)) {
-            backingMap.put(key, new NonTransactionalCommit());
-        }
-
-        Versioned<Collection<? extends byte[]>> addedValues = backingMap
-            .get(key)
-            .addCommit(commit);
-
-        if (addedValues != null) {
-            publish(addedValues.value().stream()
-                .map(value -> new MultimapEvent<String, byte[]>(
-                    "", key, value, null))
-                .collect(Collectors.toList()));
-            return true;
-        }
-
-        return false;
-    }
-
-    /**
-     * Handles a putAndGet commit.
-     *
-     * @param commit a put commit
-     * @return the updated values
-     */
-    protected Versioned<Collection<? extends byte[]>> putAndGet(Commit<? extends Put> commit) {
-        String key = commit.value().key();
-        if (commit.value().values().isEmpty()) {
-            return null;
-        }
-        if (!backingMap.containsKey(key)) {
-            backingMap.put(key, new NonTransactionalCommit());
-        }
-
-        Versioned<Collection<? extends byte[]>> addedValues = backingMap
-            .get(key)
-            .addCommit(commit);
-
-        if (addedValues != null) {
-            publish(addedValues.value().stream()
-                .map(value -> new MultimapEvent<String, byte[]>(
-                    "", key, value, null))
-                .collect(Collectors.toList()));
-        }
-
-        return toVersioned(backingMap.get(key));
-    }
-
-    protected Versioned<Collection<? extends byte[]>> replace(
-            Commit<? extends Replace> commit) {
-        String key = commit.value().key();
-        if (!backingMap.containsKey(key)) {
-            backingMap.put(key, new NonTransactionalCommit());
-        }
-
-        Versioned<Collection<? extends byte[]>> values = backingMap.get(commit.value().key()).addCommit(commit);
-        if (values != null) {
-            Set<byte[]> addedValues = Sets.newTreeSet(new ByteArrayComparator());
-            addedValues.addAll(commit.value().values());
-
-            Set<byte[]> removedValues = Sets.newTreeSet(new ByteArrayComparator());
-            removedValues.addAll(values.value());
-
-            List<MultimapEvent<String, byte[]>> events = Lists.newArrayList();
-            Sets.difference(removedValues, addedValues)
-                .forEach(value -> events.add(new MultimapEvent<>("", key, null, value)));
-            Sets.difference(addedValues, removedValues)
-                .forEach(value -> events.add(new MultimapEvent<>("", key, value, null)));
-
-            publish(events);
-        }
-        return values;
-    }
-
-    /**
-     * Handles a listen commit.
-     *
-     * @param commit listen commit
-     */
-    protected void listen(Commit<Void> commit) {
-        listeners.put(commit.session().sessionId().id(), commit.session());
-    }
-
-    /**
-     * Handles an unlisten commit.
-     *
-     * @param commit unlisten commit
-     */
-    protected void unlisten(Commit<Void> commit) {
-        listeners.remove(commit.session().sessionId().id());
-    }
-
-    /**
-     * Handles an open iterator commit.
-     *
-     * @param commit the open iterator commit
-     * @return iterator identifier
-     */
-    protected long openIterator(Commit<Void> commit) {
-        iterators.put(commit.index(), new IteratorContext(
-            commit.session().sessionId().id(),
-            backingMap.entrySet().iterator()));
-        return commit.index();
-    }
-
-    /**
-     * Handles an iterator next commit.
-     *
-     * @param commit the next commit
-     * @return a list of entries to iterate
-     */
-    protected IteratorBatch next(Commit<IteratorPosition> commit) {
-        final long iteratorId = commit.value().iteratorId();
-        final int position = commit.value().position();
-
-        IteratorContext context = iterators.get(iteratorId);
-        if (context == null) {
-            return null;
-        }
-
-        List<Map.Entry<String, byte[]>> entries = new ArrayList<>();
-        int size = 0;
-        while (context.iterator.hasNext()) {
-            context.position++;
-            if (context.position > position) {
-                Map.Entry<String, MapEntryValue> entry = context.iterator.next();
-                String key = entry.getKey();
-                int keySize = key.length();
-                for (byte[] value : entry.getValue().values()) {
-                    entries.add(Maps.immutableEntry(key, value));
-                    size += keySize;
-                    size += value.length;
-                }
-
-                if (size >= MAX_ITERATOR_BATCH_SIZE) {
-                    break;
-                }
-            }
-        }
-
-        if (entries.isEmpty()) {
-            return null;
-        }
-        return new IteratorBatch(context.position, entries);
-    }
-
-    /**
-     * Handles a close iterator commit.
-     *
-     * @param commit the close iterator commit
-     */
-    protected void closeIterator(Commit<Long> commit) {
-        iterators.remove(commit.value());
-    }
-
-    /**
-     * Publishes events to listeners.
-     *
-     * @param events list of map event to publish
-     */
-    private void publish(List<MultimapEvent<String, byte[]>> events) {
-        listeners.values().forEach(session -> session.publish(CHANGE, serializer::encode, events));
-    }
-
-    private interface MapEntryValue {
-
-        /**
-         * Returns the list of raw {@code byte[]'s}.
-         *
-         * @return list of raw values
-         */
-        Collection<? extends byte[]> values();
-
-        /**
-         * Returns the version of the value.
-         *
-         * @return version
-         */
-        long version();
-
-        /**
-         * Add a new commit and modifies the set of values accordingly.
-         * In the case of a replace or removeAll it returns the set of removed
-         * values. In the case of put or multiRemove it returns null for no
-         * change and a set of the added or removed values respectively if a
-         * change resulted.
-         *
-         * @param commit the commit to be added
-         */
-        Versioned<Collection<? extends byte[]>> addCommit(
-                Commit<? extends MultimapOperation> commit);
-    }
-
-    private class NonTransactionalCommit implements MapEntryValue {
-        private long version;
-        private final TreeSet<byte[]> valueSet = Sets.newTreeSet(new ByteArrayComparator());
-
-        public NonTransactionalCommit() {
-            //Set the version to current it will only be updated once this is
-            // populated
-            this.version = globalVersion.get();
-        }
-
-        @Override
-        public Collection<? extends byte[]> values() {
-            return ImmutableSet.copyOf(valueSet);
-        }
-
-        @Override
-        public long version() {
-            return version;
-        }
-
-        @Override
-        public Versioned<Collection<? extends byte[]>> addCommit(
-                Commit<? extends MultimapOperation> commit) {
-            Preconditions.checkNotNull(commit);
-            Preconditions.checkNotNull(commit.value());
-            Versioned<Collection<? extends byte[]>> retVersion;
-
-            if (commit.value() instanceof Put) {
-                //Using a treeset here sanitizes the input, removing duplicates
-                Set<byte[]> valuesToAdd =
-                        Sets.newTreeSet(new ByteArrayComparator());
-                ((Put) commit.value()).values().forEach(value -> {
-                    if (!valueSet.contains(value)) {
-                        valuesToAdd.add(value);
-                    }
-                });
-                if (valuesToAdd.isEmpty()) {
-                    //Do not increment or add the commit if no change resulted
-                    return null;
-                }
-                retVersion = new Versioned<>(valuesToAdd, version);
-                valuesToAdd.forEach(value -> valueSet.add(value));
-                version++;
-                return retVersion;
-
-            } else if (commit.value() instanceof Replace) {
-                //Will this work??  Need to check before check-in!
-                Set<byte[]> removedValues = Sets.newHashSet();
-                removedValues.addAll(valueSet);
-                retVersion = new Versioned<>(removedValues, version);
-                valueSet.clear();
-                Set<byte[]> valuesToAdd =
-                        Sets.newTreeSet(new ByteArrayComparator());
-                ((Replace) commit.value()).values().forEach(value -> {
-                    valuesToAdd.add(value);
-                });
-                if (valuesToAdd.isEmpty()) {
-                    version = globalVersion.incrementAndGet();
-                    backingMap.remove(((Replace) commit.value()).key());
-                    return retVersion;
-                }
-                valuesToAdd.forEach(value -> valueSet.add(value));
-                version = globalVersion.incrementAndGet();
-                return retVersion;
-
-            } else if (commit.value() instanceof RemoveAll) {
-                Set<byte[]> removed = Sets.newHashSet();
-                //We can assume here that values only appear once and so we
-                //do not need to sanitize the return for duplicates.
-                removed.addAll(valueSet);
-                retVersion = new Versioned<>(removed, version);
-                valueSet.clear();
-                //In the case of a removeAll all commits will be removed and
-                //unlike the multiRemove case we do not need to consider
-                //dependencies among additive and removal commits.
-
-                //Save the key for use after the commit is closed
-                String key = ((RemoveAll) commit.value()).key();
-                version = globalVersion.incrementAndGet();
-                backingMap.remove(key);
-                return retVersion;
-
-            } else if (commit.value() instanceof MultiRemove) {
-                //Must first calculate how many commits the removal depends on.
-                //At this time we also sanitize the removal set by adding to a
-                //set with proper handling of byte[] equality.
-                Set<byte[]> removed = Sets.newHashSet();
-                ((MultiRemove) commit.value()).values().forEach(value -> {
-                    if (valueSet.contains(value)) {
-                        removed.add(value);
-                    }
-                });
-                //If there is nothing to be removed no action should be taken.
-                if (removed.isEmpty()) {
-                    return null;
-                }
-                //Save key in case countdown results in closing the commit.
-                String removedKey = ((MultiRemove) commit.value()).key();
-                removed.forEach(removedValue -> {
-                    valueSet.remove(removedValue);
-                });
-                //The version is updated locally as well as globally even if
-                //this object will be removed from the map in case any other
-                //party still holds a reference to this object.
-                retVersion = new Versioned<>(removed, version);
-                version = globalVersion.incrementAndGet();
-                if (valueSet.isEmpty()) {
-                    backingMap.remove(removedKey);
-                }
-                return retVersion;
-
-            } else {
-                throw new IllegalArgumentException();
-            }
-        }
-    }
-
-    /**
-     * A collector that creates MapEntryValues and creates a multiset of all
-     * values in the map an equal number of times to the number of sets in
-     * which they participate.
-     */
-    private class HashMultisetValueCollector implements
-            Collector<MapEntryValue,
-                    HashMultiset<byte[]>,
-                    HashMultiset<byte[]>> {
-
-        @Override
-        public Supplier<HashMultiset<byte[]>> supplier() {
-            return HashMultiset::create;
-        }
-
-        @Override
-        public BiConsumer<HashMultiset<byte[]>, MapEntryValue> accumulator() {
-            return (multiset, mapEntryValue) ->
-                    multiset.addAll(mapEntryValue.values());
-        }
-
-        @Override
-        public BinaryOperator<HashMultiset<byte[]>> combiner() {
-            return (setOne, setTwo) -> {
-                setOne.addAll(setTwo);
-                return setOne;
-            };
-        }
-
-        @Override
-        public Function<HashMultiset<byte[]>,
-                HashMultiset<byte[]>> finisher() {
-            return Function.identity();
-        }
-
-        @Override
-        public Set<Characteristics> characteristics() {
-            return EnumSet.of(Characteristics.UNORDERED);
-        }
-    }
-
-    /**
-     * A collector that creates Entries of {@code <String, MapEntryValue>} and
-     * creates a set of entries all key value pairs in the map.
-     */
-    private class EntrySetCollector implements
-            Collector<Map.Entry<String, MapEntryValue>,
-                    Set<Map.Entry<String, byte[]>>,
-                    Set<Map.Entry<String, byte[]>>> {
-        private Set<Map.Entry<String, byte[]>> set = null;
-
-        @Override
-        public Supplier<Set<Map.Entry<String, byte[]>>> supplier() {
-            return () -> {
-                if (set == null) {
-                    set = Sets.newHashSet();
-                }
-                return set;
-            };
-        }
-
-        @Override
-        public BiConsumer<Set<Map.Entry<String, byte[]>>,
-                Map.Entry<String, MapEntryValue>> accumulator() {
-            return (set, entry) -> {
-                entry
-                        .getValue()
-                        .values()
-                        .forEach(byteValue ->
-                                set.add(Maps.immutableEntry(entry.getKey(),
-                                        byteValue)));
-            };
-        }
-
-        @Override
-        public BinaryOperator<Set<Map.Entry<String, byte[]>>> combiner() {
-            return (setOne, setTwo) -> {
-                setOne.addAll(setTwo);
-                return setOne;
-            };
-        }
-
-        @Override
-        public Function<Set<Map.Entry<String, byte[]>>,
-                Set<Map.Entry<String, byte[]>>> finisher() {
-            return (unused) -> set;
-        }
-
-        @Override
-        public Set<Characteristics> characteristics() {
-            return EnumSet.of(Characteristics.UNORDERED);
-        }
-    }
-    /**
-     * Utility for turning a {@code MapEntryValue} to {@code Versioned}.
-     * @param value map entry value
-     * @return versioned instance or an empty list versioned -1 if argument is
-     * null
-     */
-    private Versioned<Collection<? extends byte[]>> toVersioned(
-            MapEntryValue value) {
-        return value == null ? new Versioned<>(Lists.newArrayList(), -1) :
-                new Versioned<>(value.values(),
-                        value.version());
-    }
-
-    private static class ByteArrayComparator implements Comparator<byte[]> {
-
-        @Override
-        public int compare(byte[] o1, byte[] o2) {
-            if (Arrays.equals(o1, o2)) {
-                return 0;
-            } else {
-                for (int i = 0; i < o1.length && i < o2.length; i++) {
-                    if (o1[i] < o2[i]) {
-                        return -1;
-                    } else if (o1[i] > o2[i]) {
-                        return 1;
-                    }
-                }
-                return o1.length > o2.length ? 1 : -1;
-            }
-        }
-    }
-
-    private static class IteratorContext {
-        private final long sessionId;
-        private final Iterator<Map.Entry<String, MapEntryValue>> iterator;
-        private int position = 0;
-
-        IteratorContext(long sessionId, Iterator<Map.Entry<String, MapEntryValue>> iterator) {
-            this.sessionId = sessionId;
-            this.iterator = iterator;
-        }
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMap.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMap.java
deleted file mode 100644
index 25c4d84..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMap.java
+++ /dev/null
@@ -1,169 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.NavigableSet;
-import java.util.TreeMap;
-import java.util.concurrent.CompletableFuture;
-
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FloorEntry;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.HigherEntry;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LowerEntry;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncConsistentTreeMap;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Versioned;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.CEILING_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.CEILING_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.CeilingEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.CeilingKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FIRST_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FIRST_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FLOOR_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FLOOR_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FloorKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.HIGHER_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.HIGHER_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.HigherKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LAST_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LAST_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LOWER_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LOWER_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LowerKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.POLL_FIRST_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.POLL_LAST_ENTRY;
-
-/**
- * Implementation of {@link AsyncConsistentTreeMap}.
- */
-public class AtomixConsistentTreeMap extends AtomixConsistentMap implements AsyncConsistentTreeMap<byte[]> {
-    private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
-            .register(KryoNamespaces.BASIC)
-            .register(AtomixConsistentMapOperations.NAMESPACE)
-            .register(AtomixConsistentTreeMapOperations.NAMESPACE)
-            .register(AtomixConsistentMapEvents.NAMESPACE)
-            .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 150)
-            .register(AtomixConsistentMapService.TransactionScope.class)
-            .register(TransactionLog.class)
-            .register(TransactionId.class)
-            .register(AtomixConsistentMapService.MapEntryValue.class)
-            .register(AtomixConsistentMapService.MapEntryValue.Type.class)
-            .register(new HashMap().keySet().getClass())
-            .register(TreeMap.class)
-            .build());
-
-    public AtomixConsistentTreeMap(RaftProxy proxy) {
-        super(proxy);
-    }
-
-    @Override
-    protected Serializer serializer() {
-        return SERIALIZER;
-    }
-
-    @Override
-    public CompletableFuture<String> firstKey() {
-        return proxy.invoke(FIRST_KEY, serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<String> lastKey() {
-        return proxy.invoke(LAST_KEY, serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> ceilingEntry(String key) {
-        return proxy.invoke(CEILING_ENTRY, serializer()::encode, new CeilingEntry(key), serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> floorEntry(String key) {
-        return proxy.invoke(FLOOR_ENTRY, serializer()::encode, new FloorEntry(key), serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> higherEntry(
-            String key) {
-        return proxy.invoke(HIGHER_ENTRY, serializer()::encode, new HigherEntry(key), serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> lowerEntry(
-            String key) {
-        return proxy.invoke(LOWER_ENTRY, serializer()::encode, new LowerEntry(key), serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> firstEntry() {
-        return proxy.invoke(FIRST_ENTRY, serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> lastEntry() {
-        return proxy.invoke(LAST_ENTRY, serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> pollFirstEntry() {
-        return proxy.invoke(POLL_FIRST_ENTRY, serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> pollLastEntry() {
-        return proxy.invoke(POLL_LAST_ENTRY, serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<String> lowerKey(String key) {
-        return proxy.invoke(LOWER_KEY, serializer()::encode, new LowerKey(key), serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<String> floorKey(String key) {
-        return proxy.invoke(FLOOR_KEY, serializer()::encode, new FloorKey(key), serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<String> ceilingKey(String key) {
-        return proxy.invoke(CEILING_KEY, serializer()::encode, new CeilingKey(key), serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<String> higherKey(String key) {
-        return proxy.invoke(HIGHER_KEY, serializer()::encode, new HigherKey(key), serializer()::decode);
-    }
-
-    @Override
-    public CompletableFuture<NavigableSet<String>> navigableKeySet() {
-        throw new UnsupportedOperationException("This operation is not yet supported.");
-    }
-
-    @Override
-    public CompletableFuture<NavigableMap<String, byte[]>> subMap(
-            String upperKey, String lowerKey, boolean inclusiveUpper,
-            boolean inclusiveLower) {
-        throw new UnsupportedOperationException("This operation is not yet supported.");
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapOperations.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapOperations.java
deleted file mode 100644
index 74ae7eb..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapOperations.java
+++ /dev/null
@@ -1,415 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.AbstractMap;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.Maps;
-import io.atomix.protocols.raft.operation.OperationId;
-import io.atomix.protocols.raft.operation.OperationType;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Match;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.Versioned;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * {@link org.onosproject.store.service.AsyncConsistentTreeMap} Resource
- * state machine operations.
- */
-public enum AtomixConsistentTreeMapOperations implements OperationId {
-    SUB_MAP(OperationType.QUERY),
-    FIRST_KEY(OperationType.QUERY),
-    LAST_KEY(OperationType.QUERY),
-    FIRST_ENTRY(OperationType.QUERY),
-    LAST_ENTRY(OperationType.QUERY),
-    POLL_FIRST_ENTRY(OperationType.QUERY),
-    POLL_LAST_ENTRY(OperationType.QUERY),
-    LOWER_ENTRY(OperationType.QUERY),
-    LOWER_KEY(OperationType.QUERY),
-    FLOOR_ENTRY(OperationType.QUERY),
-    FLOOR_KEY(OperationType.QUERY),
-    CEILING_ENTRY(OperationType.QUERY),
-    CEILING_KEY(OperationType.QUERY),
-    HIGHER_ENTRY(OperationType.QUERY),
-    HIGHER_KEY(OperationType.QUERY);
-
-    private final OperationType type;
-
-    AtomixConsistentTreeMapOperations(OperationType type) {
-        this.type = type;
-    }
-
-    @Override
-    public String id() {
-        return name();
-    }
-
-    @Override
-    public OperationType type() {
-        return type;
-    }
-
-    public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
-            .register(KryoNamespaces.BASIC)
-            .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 100)
-            .register(LowerKey.class)
-            .register(LowerEntry.class)
-            .register(HigherKey.class)
-            .register(HigherEntry.class)
-            .register(FloorKey.class)
-            .register(FloorEntry.class)
-            .register(CeilingKey.class)
-            .register(CeilingEntry.class)
-            .register(Versioned.class)
-            .register(AbstractMap.SimpleImmutableEntry.class)
-            .register(Maps.immutableEntry("", "").getClass())
-            .build("AtomixConsistentTreeMapOperations");
-
-    /**
-     * Abstract treeMap command.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class TreeOperation {
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .toString();
-        }
-    }
-
-    /**
-     * Abstract key-based query.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class KeyOperation extends TreeOperation {
-        protected String key;
-
-        public KeyOperation(String key) {
-            this.key = checkNotNull(key);
-        }
-
-        public KeyOperation() {
-        }
-
-        public String key() {
-            return key;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("key", key)
-                    .toString();
-        }
-    }
-
-    /**
-     * Abstract value-based query.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class ValueOperation extends TreeOperation {
-        protected byte[] value;
-
-        public ValueOperation() {}
-
-        public ValueOperation(byte[] value) {
-            this.value = checkNotNull(value);
-        }
-
-        public byte[] value() {
-            return value;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("value", value)
-                    .toString();
-        }
-    }
-
-    /**
-     * Contains key command.
-     */
-    @SuppressWarnings("serial")
-    public static class ContainsKey extends KeyOperation {
-
-        public ContainsKey(String key) {
-            super(key);
-        }
-
-        public ContainsKey() {
-        }
-    }
-    /**
-     * Contains value command.
-     */
-    @SuppressWarnings("serial")
-    public static class ContainsValue extends ValueOperation {
-        public ContainsValue() {
-        }
-
-        public ContainsValue(byte[] value) {
-            super(value);
-        }
-
-    }
-
-    /**
-     * AsyncConsistentTreeMap update command.
-     */
-    @SuppressWarnings("serial")
-    public static class UpdateAndGet extends TreeOperation {
-        private String key;
-        private byte[] value;
-        private Match<byte[]> valueMatch;
-        private Match<Long> versionMatch;
-        public UpdateAndGet() {
-        }
-
-        public UpdateAndGet(String key,
-                            byte[] value,
-                            Match<byte[]> valueMatch,
-                            Match<Long> versionMatch) {
-            this.key = key;
-            this.value = value;
-            this.valueMatch = valueMatch;
-            this.versionMatch = versionMatch;
-        }
-
-        public String key() {
-            return this.key;
-        }
-
-        public byte[] value() {
-            return this.value;
-        }
-
-        public Match<byte[]> valueMatch() {
-            return this.valueMatch;
-        }
-
-        public Match<Long> versionMatch() {
-            return this.versionMatch;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("key", key)
-                    .add("value", value)
-                    .add("valueMatch", valueMatch)
-                    .add("versionMatch", versionMatch)
-                    .toString();
-        }
-    }
-
-    /**
-     * Get query.
-     */
-    @SuppressWarnings("serial")
-    public static class Get extends KeyOperation {
-        public Get() {
-        }
-
-        public Get(String key) {
-            super(key);
-        }
-    }
-
-    /**
-     * Get or default query.
-     */
-    @SuppressWarnings("serial")
-    public static class GetOrDefault extends KeyOperation {
-        private byte[] defaultValue;
-
-        public GetOrDefault() {
-        }
-
-        public GetOrDefault(String key, byte[] defaultValue) {
-            super(key);
-            this.defaultValue = defaultValue;
-        }
-
-        /**
-         * Returns the default value.
-         *
-         * @return the default value
-         */
-        public byte[] defaultValue() {
-            return defaultValue;
-        }
-    }
-
-    /**
-     * Query returns the entry associated with the largest key less than the
-     * passed in key.
-     */
-    @SuppressWarnings("serial")
-    public static class LowerEntry extends KeyOperation {
-        public LowerEntry() {
-        }
-
-        public LowerEntry(String key) {
-            super(key);
-        }
-    }
-
-    /**
-     * Query returns the largest key less than the specified key.
-     */
-    @SuppressWarnings("serial")
-    public static class LowerKey extends KeyOperation {
-        public LowerKey() {
-        }
-
-        public LowerKey(String key) {
-            super(key);
-        }
-    }
-
-    /**
-     * Query returns the entry associated with the largest key smaller than or
-     * equal to the specified key.
-     */
-    @SuppressWarnings("serial")
-    public static class FloorEntry extends KeyOperation {
-        public FloorEntry() {
-        }
-
-        public FloorEntry(String key) {
-            super(key);
-        }
-    }
-
-    /**
-     * Query returns the largest key smaller than or equal to the passed in
-     * key.
-     */
-    @SuppressWarnings("serial")
-    public static class FloorKey extends KeyOperation {
-        public FloorKey() {
-        }
-
-        public FloorKey(String key) {
-            super(key);
-        }
-    }
-
-    /**
-     * Returns the entry associated with the smallest key larger than or equal
-     * to the specified key.
-     */
-    @SuppressWarnings("serial")
-    public static class CeilingEntry extends KeyOperation {
-        public CeilingEntry() {
-        }
-
-        public CeilingEntry(String key) {
-            super(key);
-        }
-    }
-
-    /**
-     * Returns the smallest key larger than or equal to the specified key.
-     */
-    @SuppressWarnings("serial")
-    public static class CeilingKey extends KeyOperation {
-        public CeilingKey() {
-        }
-
-        public CeilingKey(String key) {
-            super(key);
-        }
-    }
-
-    /**
-     * Returns the entry associated with the smallest key larger than the
-     * specified key.
-     */
-    @SuppressWarnings("serial")
-    public static class HigherEntry extends KeyOperation {
-        public HigherEntry() {
-        }
-
-        public HigherEntry(String key) {
-            super(key);
-        }
-    }
-
-    /**
-     * Returns the smallest key larger than the specified key.
-     */
-    @SuppressWarnings("serial")
-    public static class HigherKey extends KeyOperation {
-        public HigherKey() {
-        }
-
-        public HigherKey(String key) {
-            super(key);
-        }
-    }
-
-    @SuppressWarnings("serial")
-    public static class SubMap<K, V> extends TreeOperation {
-        private K fromKey;
-        private K toKey;
-        private boolean inclusiveFrom;
-        private boolean inclusiveTo;
-
-        public SubMap() {
-        }
-
-        public SubMap(K fromKey, K toKey, boolean inclusiveFrom,
-                      boolean inclusiveTo) {
-            this.fromKey = fromKey;
-            this.toKey = toKey;
-            this.inclusiveFrom = inclusiveFrom;
-            this.inclusiveTo = inclusiveTo;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("getFromKey", fromKey)
-                    .add("getToKey", toKey)
-                    .add("inclusiveFromBound", inclusiveFrom)
-                    .add("inclusiveToBound", inclusiveTo)
-                    .toString();
-        }
-
-        public K fromKey() {
-            return fromKey;
-        }
-
-        public K toKey() {
-            return toKey;
-        }
-
-        public boolean isInclusiveFrom() {
-            return inclusiveFrom;
-        }
-
-        public boolean isInclusiveTo() {
-            return inclusiveTo;
-        }
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapService.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapService.java
deleted file mode 100644
index 89c8da6..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapService.java
+++ /dev/null
@@ -1,201 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.TreeMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-
-import com.google.common.collect.Maps;
-import io.atomix.protocols.raft.service.Commit;
-import io.atomix.protocols.raft.service.RaftServiceExecutor;
-import io.atomix.protocols.raft.session.RaftSession;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Versioned;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.CEILING_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.CEILING_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.CeilingEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.CeilingKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FIRST_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FIRST_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FLOOR_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FLOOR_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FloorEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.FloorKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.HIGHER_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.HIGHER_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.HigherEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.HigherKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LAST_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LAST_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LOWER_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LOWER_KEY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LowerEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.LowerKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.POLL_FIRST_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.POLL_LAST_ENTRY;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.SUB_MAP;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.SubMap;
-
-/**
- * State machine corresponding to {@link AtomixConsistentTreeMap} backed by a
- * {@link TreeMap}.
- */
-public class AtomixConsistentTreeMapService extends AtomixConsistentMapService {
-
-    private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
-            .register(KryoNamespaces.BASIC)
-            .register(AtomixConsistentMapOperations.NAMESPACE)
-            .register(AtomixConsistentTreeMapOperations.NAMESPACE)
-            .register(AtomixConsistentMapEvents.NAMESPACE)
-            .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 150)
-            .register(TransactionScope.class)
-            .register(TransactionLog.class)
-            .register(TransactionId.class)
-            .register(MapEntryValue.class)
-            .register(MapEntryValue.Type.class)
-            .register(new HashMap().keySet().getClass())
-            .register(ConcurrentSkipListMap.class)
-            .build());
-
-    @Override
-    protected NavigableMap<String, MapEntryValue> createMap() {
-        return new ConcurrentSkipListMap<>();
-    }
-
-    @Override
-    protected NavigableMap<String, MapEntryValue> entries() {
-        return (NavigableMap<String, MapEntryValue>) super.entries();
-    }
-
-    @Override
-    protected Serializer serializer() {
-        return SERIALIZER;
-    }
-
-    @Override
-    public void configure(RaftServiceExecutor executor) {
-        super.configure(executor);
-        executor.register(SUB_MAP, serializer()::decode, this::subMap, serializer()::encode);
-        executor.register(FIRST_KEY, (Commit<Void> c) -> firstKey(), serializer()::encode);
-        executor.register(LAST_KEY, (Commit<Void> c) -> lastKey(), serializer()::encode);
-        executor.register(FIRST_ENTRY, (Commit<Void> c) -> firstEntry(), serializer()::encode);
-        executor.register(LAST_ENTRY, (Commit<Void> c) -> lastEntry(), serializer()::encode);
-        executor.register(POLL_FIRST_ENTRY, (Commit<Void> c) -> pollFirstEntry(), serializer()::encode);
-        executor.register(POLL_LAST_ENTRY, (Commit<Void> c) -> pollLastEntry(), serializer()::encode);
-        executor.register(LOWER_ENTRY, serializer()::decode, this::lowerEntry, serializer()::encode);
-        executor.register(LOWER_KEY, serializer()::decode, this::lowerKey, serializer()::encode);
-        executor.register(FLOOR_ENTRY, serializer()::decode, this::floorEntry, serializer()::encode);
-        executor.register(FLOOR_KEY, serializer()::decode, this::floorKey, serializer()::encode);
-        executor.register(CEILING_ENTRY, serializer()::decode, this::ceilingEntry, serializer()::encode);
-        executor.register(CEILING_KEY, serializer()::decode, this::ceilingKey, serializer()::encode);
-        executor.register(HIGHER_ENTRY, serializer()::decode, this::higherEntry, serializer()::encode);
-        executor.register(HIGHER_KEY, serializer()::decode, this::higherKey, serializer()::encode);
-    }
-
-    protected NavigableMap<String, MapEntryValue> subMap(
-            Commit<? extends SubMap> commit) {
-        // Do not support this until lazy communication is possible.  At present
-        // it transmits up to the entire map.
-        SubMap<String, MapEntryValue> subMap = commit.value();
-        return entries().subMap(subMap.fromKey(), subMap.isInclusiveFrom(),
-                subMap.toKey(), subMap.isInclusiveTo());
-    }
-
-    protected String firstKey() {
-        return isEmpty() ? null : entries().firstKey();
-    }
-
-    protected String lastKey() {
-        return isEmpty() ? null : entries().lastKey();
-    }
-
-    protected Map.Entry<String, Versioned<byte[]>> higherEntry(Commit<? extends HigherEntry> commit) {
-        return isEmpty() ? null : toVersionedEntry(entries().higherEntry(commit.value().key()));
-    }
-
-    protected Map.Entry<String, Versioned<byte[]>> firstEntry() {
-        return isEmpty() ? null : toVersionedEntry(entries().firstEntry());
-    }
-
-    protected Map.Entry<String, Versioned<byte[]>> lastEntry() {
-        return isEmpty() ? null : toVersionedEntry(entries().lastEntry());
-    }
-
-    protected Map.Entry<String, Versioned<byte[]>> pollFirstEntry() {
-        return toVersionedEntry(entries().pollFirstEntry());
-    }
-
-    protected Map.Entry<String, Versioned<byte[]>> pollLastEntry() {
-        return toVersionedEntry(entries().pollLastEntry());
-    }
-
-    protected Map.Entry<String, Versioned<byte[]>> lowerEntry(Commit<? extends LowerEntry> commit) {
-        return toVersionedEntry(entries().lowerEntry(commit.value().key()));
-    }
-
-    protected String lowerKey(Commit<? extends LowerKey> commit) {
-        return entries().lowerKey(commit.value().key());
-    }
-
-    protected Map.Entry<String, Versioned<byte[]>> floorEntry(Commit<? extends FloorEntry> commit) {
-        return toVersionedEntry(entries().floorEntry(commit.value().key()));
-    }
-
-    protected String floorKey(Commit<? extends FloorKey> commit) {
-        return entries().floorKey(commit.value().key());
-    }
-
-    protected Map.Entry<String, Versioned<byte[]>> ceilingEntry(Commit<CeilingEntry> commit) {
-        return toVersionedEntry(entries().ceilingEntry(commit.value().key()));
-    }
-
-    protected String ceilingKey(Commit<CeilingKey> commit) {
-        return entries().ceilingKey(commit.value().key());
-    }
-
-    protected String higherKey(Commit<HigherKey> commit) {
-        return entries().higherKey(commit.value().key());
-    }
-
-    private Map.Entry<String, Versioned<byte[]>> toVersionedEntry(
-            Map.Entry<String, MapEntryValue> entry) {
-        return entry == null || valueIsNull(entry.getValue())
-                ? null : Maps.immutableEntry(entry.getKey(), toVersioned(entry.getValue()));
-    }
-
-    @Override
-    public void onExpire(RaftSession session) {
-        closeListener(session.sessionId().id());
-    }
-
-    @Override
-    public void onClose(RaftSession session) {
-        closeListener(session.sessionId().id());
-    }
-
-    private void closeListener(Long sessionId) {
-        listeners.remove(sessionId);
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixCounter.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixCounter.java
deleted file mode 100644
index 9c579d6..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixCounter.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.concurrent.CompletableFuture;
-
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncAtomicCounter;
-import org.onosproject.store.service.Serializer;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.ADD_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.AddAndGet;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.COMPARE_AND_SET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.CompareAndSet;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.GET_AND_ADD;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.GET_AND_INCREMENT;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.GetAndAdd;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.INCREMENT_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.SET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.Set;
-
-/**
- * Atomix counter implementation.
- */
-public class AtomixCounter extends AbstractRaftPrimitive implements AsyncAtomicCounter {
-    private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
-            .register(KryoNamespaces.BASIC)
-            .register(AtomixCounterOperations.NAMESPACE)
-            .build());
-
-    public AtomixCounter(RaftProxy proxy) {
-        super(proxy);
-    }
-
-    private long nullOrZero(Long value) {
-        return value != null ? value : 0;
-    }
-
-    @Override
-    public CompletableFuture<Long> get() {
-        return proxy.<Long>invoke(GET, SERIALIZER::decode).thenApply(this::nullOrZero);
-    }
-
-    @Override
-    public CompletableFuture<Void> set(long value) {
-        return proxy.invoke(SET, SERIALIZER::encode, new Set(value));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> compareAndSet(long expectedValue, long updateValue) {
-        return proxy.invoke(COMPARE_AND_SET, SERIALIZER::encode,
-                new CompareAndSet(expectedValue, updateValue), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Long> addAndGet(long delta) {
-        return proxy.invoke(ADD_AND_GET, SERIALIZER::encode, new AddAndGet(delta), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Long> getAndAdd(long delta) {
-        return proxy.invoke(GET_AND_ADD, SERIALIZER::encode, new GetAndAdd(delta), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Long> incrementAndGet() {
-        return proxy.invoke(INCREMENT_AND_GET, SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Long> getAndIncrement() {
-        return proxy.invoke(GET_AND_INCREMENT, SERIALIZER::decode);
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixCounterOperations.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixCounterOperations.java
deleted file mode 100644
index 67ff936..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixCounterOperations.java
+++ /dev/null
@@ -1,186 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import io.atomix.protocols.raft.operation.OperationId;
-import io.atomix.protocols.raft.operation.OperationType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-
-/**
- * Counter commands.
- */
-public enum AtomixCounterOperations implements OperationId {
-    SET(OperationType.COMMAND),
-    COMPARE_AND_SET(OperationType.COMMAND),
-    INCREMENT_AND_GET(OperationType.COMMAND),
-    GET_AND_INCREMENT(OperationType.COMMAND),
-    ADD_AND_GET(OperationType.COMMAND),
-    GET_AND_ADD(OperationType.COMMAND),
-    GET(OperationType.QUERY);
-
-    private final OperationType type;
-
-    AtomixCounterOperations(OperationType type) {
-        this.type = type;
-    }
-
-    @Override
-    public String id() {
-        return name();
-    }
-
-    @Override
-    public OperationType type() {
-        return type;
-    }
-
-    public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
-            .register(KryoNamespaces.BASIC)
-            .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
-            .register(Get.class)
-            .register(Set.class)
-            .register(CompareAndSet.class)
-            .register(AddAndGet.class)
-            .register(GetAndAdd.class)
-            .build("AtomixCounterOperations");
-
-    /**
-     * Abstract value command.
-     */
-    public abstract static class ValueOperation {
-    }
-
-    /**
-     * Get query.
-     */
-    public static class Get extends ValueOperation {
-    }
-
-    /**
-     * Set command.
-     */
-    public static class Set extends ValueOperation {
-        private Long value;
-
-        public Set() {
-        }
-
-        public Set(Long value) {
-            this.value = value;
-        }
-
-        /**
-         * Returns the command value.
-         *
-         * @return The command value.
-         */
-        public Long value() {
-            return value;
-        }
-
-        @Override
-        public String toString() {
-            return String.format("%s[value=%s]", getClass().getSimpleName(), value);
-        }
-    }
-
-    /**
-     * Compare and set command.
-     */
-    public static class CompareAndSet extends ValueOperation {
-        private Long expect;
-        private Long update;
-
-        public CompareAndSet() {
-        }
-
-        public CompareAndSet(Long expect, Long update) {
-            this.expect = expect;
-            this.update = update;
-        }
-
-        /**
-         * Returns the expected value.
-         *
-         * @return The expected value.
-         */
-        public Long expect() {
-            return expect;
-        }
-
-        /**
-         * Returns the updated value.
-         *
-         * @return The updated value.
-         */
-        public Long update() {
-            return update;
-        }
-
-        @Override
-        public String toString() {
-            return String.format("%s[expect=%s, update=%s]", getClass().getSimpleName(), expect, update);
-        }
-    }
-
-    /**
-     * Delta command.
-     */
-    public abstract static class DeltaOperation extends ValueOperation {
-        private long delta;
-
-        public DeltaOperation() {
-        }
-
-        public DeltaOperation(long delta) {
-            this.delta = delta;
-        }
-
-        /**
-         * Returns the delta.
-         *
-         * @return The delta.
-         */
-        public long delta() {
-            return delta;
-        }
-    }
-
-    /**
-     * Get and add command.
-     */
-    public static class GetAndAdd extends DeltaOperation {
-        public GetAndAdd() {
-        }
-
-        public GetAndAdd(long delta) {
-            super(delta);
-        }
-    }
-
-    /**
-     * Add and get command.
-     */
-    public static class AddAndGet extends DeltaOperation {
-        public AddAndGet() {
-        }
-
-        public AddAndGet(long delta) {
-            super(delta);
-        }
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixCounterService.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixCounterService.java
deleted file mode 100644
index a56478b..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixCounterService.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.Objects;
-
-import io.atomix.protocols.raft.service.AbstractRaftService;
-import io.atomix.protocols.raft.service.Commit;
-import io.atomix.protocols.raft.service.RaftServiceExecutor;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.Serializer;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.ADD_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.AddAndGet;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.COMPARE_AND_SET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.CompareAndSet;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.GET_AND_ADD;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.GET_AND_INCREMENT;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.GetAndAdd;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.INCREMENT_AND_GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.SET;
-import static org.onosproject.store.primitives.resources.impl.AtomixCounterOperations.Set;
-
-/**
- * Atomix long state.
- */
-public class AtomixCounterService extends AbstractRaftService {
-    private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
-            .register(KryoNamespaces.BASIC)
-            .register(AtomixCounterOperations.NAMESPACE)
-            .build());
-
-    private Long value = 0L;
-
-    @Override
-    protected void configure(RaftServiceExecutor executor) {
-        executor.register(SET, SERIALIZER::decode, this::set);
-        executor.register(GET, this::get, SERIALIZER::encode);
-        executor.register(COMPARE_AND_SET, SERIALIZER::decode, this::compareAndSet, SERIALIZER::encode);
-        executor.register(INCREMENT_AND_GET, this::incrementAndGet, SERIALIZER::encode);
-        executor.register(GET_AND_INCREMENT, this::getAndIncrement, SERIALIZER::encode);
-        executor.register(ADD_AND_GET, SERIALIZER::decode, this::addAndGet, SERIALIZER::encode);
-        executor.register(GET_AND_ADD, SERIALIZER::decode, this::getAndAdd, SERIALIZER::encode);
-    }
-
-    @Override
-    public void snapshot(SnapshotWriter writer) {
-        writer.writeLong(value);
-    }
-
-    @Override
-    public void install(SnapshotReader reader) {
-        value = reader.readLong();
-    }
-
-    /**
-     * Handles a set commit.
-     *
-     * @param commit the commit to handle
-     */
-    protected void set(Commit<Set> commit) {
-        value = commit.value().value();
-    }
-
-    /**
-     * Handles a get commit.
-     *
-     * @param commit the commit to handle
-     * @return counter value
-     */
-    protected Long get(Commit<Void> commit) {
-        return value;
-    }
-
-    /**
-     * Handles a compare and set commit.
-     *
-     * @param commit the commit to handle
-     * @return counter value
-     */
-    protected boolean compareAndSet(Commit<CompareAndSet> commit) {
-        if (Objects.equals(value, commit.value().expect())) {
-            value = commit.value().update();
-            return true;
-        }
-        return false;
-    }
-
-    /**
-     * Handles an increment and get commit.
-     *
-     * @param commit the commit to handle
-     * @return counter value
-     */
-    protected long incrementAndGet(Commit<Void> commit) {
-        Long oldValue = value;
-        value = oldValue + 1;
-        return value;
-    }
-
-    /**
-     * Handles a get and increment commit.
-     *
-     * @param commit the commit to handle
-     * @return counter value
-     */
-    protected long getAndIncrement(Commit<Void> commit) {
-        Long oldValue = value;
-        value = oldValue + 1;
-        return oldValue;
-    }
-
-    /**
-     * Handles an add and get commit.
-     *
-     * @param commit the commit to handle
-     * @return counter value
-     */
-    protected long addAndGet(Commit<AddAndGet> commit) {
-        Long oldValue = value;
-        value = oldValue + commit.value().delta();
-        return value;
-    }
-
-    /**
-     * Handles a get and add commit.
-     *
-     * @param commit the commit to handle
-     * @return counter value
-     */
-    protected long getAndAdd(Commit<GetAndAdd> commit) {
-        Long oldValue = value;
-        value = oldValue + commit.value().delta();
-        return oldValue;
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLock.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLock.java
deleted file mode 100644
index c234be3..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLock.java
+++ /dev/null
@@ -1,247 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.time.Duration;
-import java.util.Map;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.function.Consumer;
-
-import com.google.common.collect.Maps;
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.OrderedExecutor;
-import org.onlab.util.SharedScheduledExecutors;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncDistributedLock;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.Version;
-
-import static org.onlab.util.Tools.orderedFuture;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockEvents.FAILED;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockEvents.LOCKED;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockOperations.LOCK;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockOperations.Lock;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockOperations.UNLOCK;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockOperations.Unlock;
-
-/**
- * Atomix lock implementation.
- * <p>
- * This {@link org.onosproject.store.service.DistributedLock} implementation uses a {@link RaftProxy} to interact
- * with a {@link AtomixDistributedLockService} replicated state machine.
- */
-public class AtomixDistributedLock extends AbstractRaftPrimitive implements AsyncDistributedLock {
-    private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
-        .register(KryoNamespaces.BASIC)
-        .register(AtomixDistributedLockOperations.NAMESPACE)
-        .register(AtomixDistributedLockEvents.NAMESPACE)
-        .build());
-
-    private final ScheduledExecutorService scheduledExecutor;
-    private final Executor orderedExecutor;
-    private final Map<Integer, LockAttempt> attempts = Maps.newConcurrentMap();
-    private final AtomicInteger id = new AtomicInteger();
-    private final AtomicInteger lock = new AtomicInteger();
-
-    public AtomixDistributedLock(RaftProxy proxy) {
-        super(proxy);
-        this.scheduledExecutor = SharedScheduledExecutors.getPoolThreadExecutor();
-        this.orderedExecutor = new OrderedExecutor(scheduledExecutor);
-        proxy.addEventListener(LOCKED, SERIALIZER::decode, this::handleLocked);
-        proxy.addEventListener(FAILED, SERIALIZER::decode, this::handleFailed);
-    }
-
-    /**
-     * Handles a {@code LOCKED} event.
-     *
-     * @param event the event to handle
-     */
-    private void handleLocked(LockEvent event) {
-        // Remove the LockAttempt from the attempts map and complete it with the lock version if it exists.
-        // If the attempt no longer exists, it likely was expired by a client-side timer.
-        LockAttempt attempt = attempts.remove(event.id());
-        if (attempt != null) {
-            attempt.complete(new Version(event.version()));
-        }
-    }
-
-    /**
-     * Handles a {@code FAILED} event.
-     *
-     * @param event the event to handle
-     */
-    private void handleFailed(LockEvent event) {
-        // Remove the LockAttempt from the attempts map and complete it with a null value if it exists.
-        // If the attempt no longer exists, it likely was expired by a client-side timer.
-        LockAttempt attempt = attempts.remove(event.id());
-        if (attempt != null) {
-            attempt.complete(null);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Version> lock() {
-        // Create and register a new attempt and invoke the LOCK operation on the replicated state machine.
-        LockAttempt attempt = new LockAttempt();
-        proxy.invoke(LOCK, SERIALIZER::encode, new Lock(attempt.id(), -1)).whenComplete((result, error) -> {
-            if (error != null) {
-                attempt.completeExceptionally(error);
-            }
-        });
-
-        // Return an ordered future that can safely be blocked inside the executor thread.
-        return orderedFuture(attempt, orderedExecutor, scheduledExecutor);
-    }
-
-    @Override
-    public CompletableFuture<Optional<Version>> tryLock() {
-        // If the proxy is currently disconnected from the cluster, we can just fail the lock attempt here.
-        RaftProxy.State state = proxy.getState();
-        if (state != RaftProxy.State.CONNECTED) {
-            return CompletableFuture.completedFuture(Optional.empty());
-        }
-
-        // Create and register a new attempt and invoke the LOCK operation on teh replicated state machine with
-        // a 0 timeout. The timeout will cause the state machine to immediately reject the request if the lock is
-        // already owned by another process.
-        LockAttempt attempt = new LockAttempt();
-        proxy.invoke(LOCK, SERIALIZER::encode, new Lock(attempt.id(), 0)).whenComplete((result, error) -> {
-            if (error != null) {
-                attempt.completeExceptionally(error);
-            }
-        });
-
-        // Return an ordered future that can safely be blocked inside the executor thread.
-        return orderedFuture(attempt, orderedExecutor, scheduledExecutor)
-            .thenApply(Optional::ofNullable);
-    }
-
-    @Override
-    public CompletableFuture<Optional<Version>> tryLock(Duration timeout) {
-        // Create a lock attempt with a client-side timeout and fail the lock if the timer expires.
-        // Because time does not progress at the same rate on different nodes, we can't guarantee that
-        // the lock won't be granted to this process after it's expired here. Thus, if this timer expires and
-        // we fail the lock on the client, we also still need to send an UNLOCK command to the cluster in case it's
-        // later granted by the cluster. Note that the semantics of the Raft client will guarantee this operation
-        // occurs after any prior LOCK attempt, and the Raft client will retry the UNLOCK request until successful.
-        // Additionally, sending the unique lock ID with the command ensures we won't accidentally unlock a different
-        // lock call also granted to this process.
-        LockAttempt attempt = new LockAttempt(timeout, a -> {
-            a.complete(null);
-            proxy.invoke(UNLOCK, SERIALIZER::encode, new Unlock(a.id()));
-        });
-
-        // Invoke the LOCK operation on the replicated state machine with the given timeout. If the lock is currently
-        // held by another process, the state machine will add the attempt to a queue and publish a FAILED event if
-        // the timer expires before this process can be granted the lock. If the client cannot reach the Raft cluster,
-        // the client-side timer will expire the attempt.
-        proxy.invoke(LOCK, SERIALIZER::encode, new Lock(attempt.id(), timeout.toMillis()))
-            .whenComplete((result, error) -> {
-                if (error != null) {
-                    attempt.completeExceptionally(error);
-                }
-            });
-
-        // Return an ordered future that can safely be blocked inside the executor thread.
-        return orderedFuture(attempt, orderedExecutor, scheduledExecutor)
-            .thenApply(Optional::ofNullable);
-    }
-
-    @Override
-    public CompletableFuture<Void> unlock() {
-        // Use the current lock ID to ensure we only unlock the lock currently held by this process.
-        int lock = this.lock.getAndSet(0);
-        if (lock != 0) {
-            return orderedFuture(
-                proxy.invoke(UNLOCK, SERIALIZER::encode, new Unlock(lock)),
-                orderedExecutor,
-                scheduledExecutor);
-        }
-        return CompletableFuture.completedFuture(null);
-    }
-
-    /**
-     * Closes the lock.
-     *
-     * @return a future to be completed once the lock has been closed
-     */
-    public CompletableFuture<Void> close() {
-        return proxy.close();
-    }
-
-    /**
-     * Lock attempt.
-     */
-    private class LockAttempt extends CompletableFuture<Version> {
-        private final int id;
-        private final ScheduledFuture<?> scheduledFuture;
-
-        LockAttempt() {
-            this(null, null);
-        }
-
-        LockAttempt(Duration duration, Consumer<LockAttempt> callback) {
-            this.id = AtomixDistributedLock.this.id.incrementAndGet();
-            this.scheduledFuture = duration != null && callback != null
-                ? scheduledExecutor.schedule(() -> callback.accept(this), duration.toMillis(), TimeUnit.MILLISECONDS)
-                : null;
-            attempts.put(id, this);
-        }
-
-        /**
-         * Returns the lock attempt ID.
-         *
-         * @return the lock attempt ID
-         */
-        int id() {
-            return id;
-        }
-
-        @Override
-        public boolean complete(Version version) {
-            if (isDone()) {
-                return super.complete(null);
-            }
-            cancel();
-            if (version != null) {
-                lock.set(id);
-                return super.complete(version);
-            } else {
-                return super.complete(null);
-            }
-        }
-
-        @Override
-        public boolean completeExceptionally(Throwable ex) {
-            cancel();
-            return super.completeExceptionally(ex);
-        }
-
-        private void cancel() {
-            if (scheduledFuture != null) {
-                scheduledFuture.cancel(false);
-            }
-            attempts.remove(id);
-        }
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockEvents.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockEvents.java
deleted file mode 100644
index e6d2b2a..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockEvents.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import io.atomix.protocols.raft.event.EventType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-
-/**
- * Raft value events.
- */
-public enum AtomixDistributedLockEvents implements EventType {
-    LOCKED("lock"),
-    FAILED("fail");
-
-    private final String id;
-
-    AtomixDistributedLockEvents(String id) {
-        this.id = id;
-    }
-
-    @Override
-    public String id() {
-        return id;
-    }
-
-    public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
-        .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 50)
-        .register(LockEvent.class)
-        .register(byte[].class)
-        .build(AtomixDistributedLockEvents.class.getSimpleName());
-
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockOperations.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockOperations.java
deleted file mode 100644
index 5cfbc84..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockOperations.java
+++ /dev/null
@@ -1,137 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import io.atomix.protocols.raft.operation.OperationId;
-import io.atomix.protocols.raft.operation.OperationType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-
-import static com.google.common.base.MoreObjects.toStringHelper;
-
-/**
- * {@link org.onosproject.store.service.DistributedLock} operations.
- * <p>
- * WARNING: Do not refactor enum values. Only add to them.
- * Changing values risk breaking the ability to backup/restore/upgrade clusters.
- */
-public enum AtomixDistributedLockOperations implements OperationId {
-    LOCK(OperationType.COMMAND),
-    UNLOCK(OperationType.COMMAND);
-
-    private final OperationType type;
-
-    AtomixDistributedLockOperations(OperationType type) {
-        this.type = type;
-    }
-
-    @Override
-    public String id() {
-        return name();
-    }
-
-    @Override
-    public OperationType type() {
-        return type;
-    }
-
-    public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
-        .register(KryoNamespaces.BASIC)
-        .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
-        .register(Lock.class)
-        .register(Unlock.class)
-        .build(AtomixDistributedLockOperations.class.getSimpleName());
-
-    /**
-     * Abstract lock operation.
-     */
-    public abstract static class LockOperation {
-        @Override
-        public String toString() {
-            return toStringHelper(this).toString();
-        }
-    }
-
-    /**
-     * Lock command.
-     */
-    public static class Lock extends LockOperation {
-        private final int id;
-        private final long timeout;
-
-        public Lock() {
-            this(0, 0);
-        }
-
-        public Lock(int id, long timeout) {
-            this.id = id;
-            this.timeout = timeout;
-        }
-
-        /**
-         * Returns the lock identifier.
-         *
-         * @return the lock identifier
-         */
-        public int id() {
-            return id;
-        }
-
-        /**
-         * Returns the lock attempt timeout.
-         *
-         * @return the lock attempt timeout
-         */
-        public long timeout() {
-            return timeout;
-        }
-
-        @Override
-        public String toString() {
-            return toStringHelper(this)
-                .add("id", id)
-                .add("timeout", timeout)
-                .toString();
-        }
-    }
-
-    /**
-     * Unlock command.
-     */
-    public static class Unlock extends LockOperation {
-        private final int id;
-
-        public Unlock(int id) {
-            this.id = id;
-        }
-
-        /**
-         * Returns the lock identifier.
-         *
-         * @return the lock identifier
-         */
-        public int id() {
-            return id;
-        }
-
-        @Override
-        public String toString() {
-            return toStringHelper(this)
-                .add("id", id)
-                .toString();
-        }
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockService.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockService.java
deleted file mode 100644
index e730ffd..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDistributedLockService.java
+++ /dev/null
@@ -1,268 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.time.Duration;
-import java.util.ArrayDeque;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Queue;
-
-import io.atomix.protocols.raft.service.AbstractRaftService;
-import io.atomix.protocols.raft.service.Commit;
-import io.atomix.protocols.raft.service.RaftServiceExecutor;
-import io.atomix.protocols.raft.session.RaftSession;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import io.atomix.utils.concurrent.Scheduled;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.Serializer;
-
-import static com.google.common.base.MoreObjects.toStringHelper;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockEvents.FAILED;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockEvents.LOCKED;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockOperations.LOCK;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockOperations.Lock;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockOperations.UNLOCK;
-import static org.onosproject.store.primitives.resources.impl.AtomixDistributedLockOperations.Unlock;
-
-/**
- * Raft atomic value service.
- */
-public class AtomixDistributedLockService extends AbstractRaftService {
-    private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
-        .register(KryoNamespaces.BASIC)
-        .register(AtomixDistributedLockOperations.NAMESPACE)
-        .register(AtomixDistributedLockEvents.NAMESPACE)
-        .register(LockHolder.class)
-        .register(ArrayDeque.class)
-        .build());
-
-    private LockHolder lock;
-    private Queue<LockHolder> queue = new ArrayDeque<>();
-    private final Map<Long, Scheduled> timers = new HashMap<>();
-
-    @Override
-    protected void configure(RaftServiceExecutor executor) {
-        executor.register(LOCK, SERIALIZER::decode, this::lock);
-        executor.register(UNLOCK, SERIALIZER::decode, this::unlock);
-    }
-
-    @Override
-    public void snapshot(SnapshotWriter writer) {
-        writer.writeObject(lock, SERIALIZER::encode);
-        writer.writeObject(queue, SERIALIZER::encode);
-    }
-
-    @Override
-    public void install(SnapshotReader reader) {
-        lock = reader.readObject(SERIALIZER::decode);
-        queue = reader.readObject(SERIALIZER::decode);
-
-        // After the snapshot is installed, we need to cancel any existing timers and schedule new ones based on the
-        // state provided by the snapshot.
-        timers.values().forEach(Scheduled::cancel);
-        timers.clear();
-        for (LockHolder holder : queue) {
-            if (holder.expire > 0) {
-                timers.put(holder.index,
-                    scheduler().schedule(Duration.ofMillis(holder.expire - wallClock().getTime().unixTimestamp()),
-                        () -> {
-                            timers.remove(holder.index);
-                            queue.remove(holder);
-                            RaftSession session = sessions().getSession(holder.session);
-                            if (session != null && session.getState().active()) {
-                                session.publish(FAILED, SERIALIZER::encode, new LockEvent(holder.id, holder.index));
-                            }
-                        }));
-            }
-        }
-    }
-
-    @Override
-    public void onExpire(RaftSession session) {
-        releaseSession(session);
-    }
-
-    @Override
-    public void onClose(RaftSession session) {
-        releaseSession(session);
-    }
-
-    /**
-     * Applies a lock commit.
-     *
-     * @param commit the lock commit
-     */
-    protected void lock(Commit<Lock> commit) {
-        // If the lock is not already owned, immediately grant the lock to the requester.
-        // Note that we still have to publish an event to the session. The event is guaranteed to be received
-        // by the client-side primitive after the LOCK response.
-        if (lock == null) {
-            lock = new LockHolder(
-                commit.value().id(),
-                commit.index(),
-                commit.session().sessionId().id(),
-                0);
-            commit.session().publish(
-                LOCKED,
-                SERIALIZER::encode,
-                new LockEvent(commit.value().id(), commit.index()));
-        // If the timeout is 0, that indicates this is a tryLock request. Immediately fail the request.
-        } else if (commit.value().timeout() == 0) {
-            commit.session().publish(FAILED, SERIALIZER::encode, new LockEvent(commit.value().id(), commit.index()));
-        // If a timeout exists, add the request to the queue and set a timer. Note that the lock request expiration
-        // time is based on the *state machine* time - not the system time - to ensure consistency across servers.
-        } else if (commit.value().timeout() > 0) {
-            LockHolder holder = new LockHolder(
-                commit.value().id(),
-                commit.index(),
-                commit.session().sessionId().id(),
-                wallClock().getTime().unixTimestamp() + commit.value().timeout());
-            queue.add(holder);
-            timers.put(commit.index(), scheduler().schedule(Duration.ofMillis(commit.value().timeout()), () -> {
-                // When the lock request timer expires, remove the request from the queue and publish a FAILED
-                // event to the session. Note that this timer is guaranteed to be executed in the same thread as the
-                // state machine commands, so there's no need to use a lock here.
-                timers.remove(commit.index());
-                queue.remove(holder);
-                if (commit.session().getState().active()) {
-                    commit.session().publish(
-                        FAILED,
-                        SERIALIZER::encode,
-                        new LockEvent(commit.value().id(), commit.index()));
-                }
-            }));
-        // If the lock is -1, just add the request to the queue with no expiration.
-        } else {
-            LockHolder holder = new LockHolder(
-                commit.value().id(),
-                commit.index(),
-                commit.session().sessionId().id(),
-                0);
-            queue.add(holder);
-        }
-    }
-
-    /**
-     * Applies an unlock commit.
-     *
-     * @param commit the unlock commit
-     */
-    protected void unlock(Commit<Unlock> commit) {
-        if (lock != null) {
-            // If the commit's session does not match the current lock holder, ignore the request.
-            if (lock.session != commit.session().sessionId().id()) {
-                return;
-            }
-
-            // If the current lock ID does not match the requested lock ID, ignore the request. This ensures that
-            // internal releases of locks that were never acquired by the client-side primitive do not cause
-            // legitimate locks to be unlocked.
-            if (lock.id != commit.value().id()) {
-                return;
-            }
-
-            // The lock has been released. Populate the lock from the queue.
-            lock = queue.poll();
-            while (lock != null) {
-                // If the waiter has a lock timer, cancel the timer.
-                Scheduled timer = timers.remove(lock.index);
-                if (timer != null) {
-                    timer.cancel();
-                }
-
-                // If the lock session is for some reason inactive, continue on to the next waiter. Otherwise,
-                // publish a LOCKED event to the new lock holder's session.
-                RaftSession session = sessions().getSession(lock.session);
-                if (session == null || !session.getState().active()) {
-                    lock = queue.poll();
-                } else {
-                    session.publish(
-                        LOCKED,
-                        SERIALIZER::encode,
-                        new LockEvent(lock.id, commit.index()));
-                    break;
-                }
-            }
-        }
-    }
-
-    /**
-     * Handles a session that has been closed by a client or expired by the cluster.
-     * <p>
-     * When a session is removed, if the session is the current lock holder then the lock is released and the next
-     * session waiting in the queue is granted the lock. Additionally, all pending lock requests for the session
-     * are removed from the lock queue.
-     *
-     * @param session the closed session
-     */
-    private void releaseSession(RaftSession session) {
-        // Remove all instances of the session from the lock queue.
-        queue.removeIf(lock -> lock.session == session.sessionId().id());
-
-        // If the removed session is the current holder of the lock, nullify the lock and attempt to grant it
-        // to the next waiter in the queue.
-        if (lock != null && lock.session == session.sessionId().id()) {
-            lock = queue.poll();
-            while (lock != null) {
-                // If the waiter has a lock timer, cancel the timer.
-                Scheduled timer = timers.remove(lock.index);
-                if (timer != null) {
-                    timer.cancel();
-                }
-
-                // If the lock session is inactive, continue on to the next waiter. Otherwise,
-                // publish a LOCKED event to the new lock holder's session.
-                RaftSession lockSession = sessions().getSession(lock.session);
-                if (lockSession == null || !lockSession.getState().active()) {
-                    lock = queue.poll();
-                } else {
-                    lockSession.publish(
-                        LOCKED,
-                        SERIALIZER::encode,
-                        new LockEvent(lock.id, lock.index));
-                    break;
-                }
-            }
-        }
-    }
-
-    private class LockHolder {
-        private final int id;
-        private final long index;
-        private final long session;
-        private final long expire;
-
-        public LockHolder(int id, long index, long session, long expire) {
-            this.id = id;
-            this.index = index;
-            this.session = session;
-            this.expire = expire;
-        }
-
-        @Override
-        public String toString() {
-            return toStringHelper(this)
-                .add("id", id)
-                .add("index", index)
-                .add("session", session)
-                .add("expire", expire)
-                .toString();
-        }
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTree.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTree.java
deleted file mode 100644
index 877d3fe..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTree.java
+++ /dev/null
@@ -1,332 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.Executor;
-
-import com.google.common.util.concurrent.MoreExecutors;
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Match;
-import org.onlab.util.Tools;
-import org.onosproject.store.primitives.NodeUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.Get;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.GetChildren;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.Listen;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.Unlisten;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.Update;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.TransactionBegin;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.TransactionPrepare;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.TransactionPrepareAndCommit;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.TransactionCommit;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.TransactionRollback;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncDocumentTree;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTreeEvent;
-import org.onosproject.store.service.DocumentTreeListener;
-import org.onosproject.store.service.IllegalDocumentModificationException;
-import org.onosproject.store.service.NoSuchDocumentPathException;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Version;
-import org.onosproject.store.service.Versioned;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeEvents.CHANGE;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.BEGIN;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.PREPARE;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.PREPARE_AND_COMMIT;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.COMMIT;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.ROLLBACK;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.CLEAR;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.GET_CHILDREN;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.UPDATE;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.ADD_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.REMOVE_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.DocumentTreeResult.Status.ILLEGAL_MODIFICATION;
-import static org.onosproject.store.primitives.resources.impl.DocumentTreeResult.Status.INVALID_PATH;
-import static org.onosproject.store.primitives.resources.impl.DocumentTreeResult.Status.OK;
-
-/**
- * Distributed resource providing the {@link AsyncDocumentTree} primitive.
- */
-public class AtomixDocumentTree extends AbstractRaftPrimitive implements AsyncDocumentTree<byte[]> {
-    private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
-            .register(KryoNamespaces.BASIC)
-            .register(AtomixDocumentTreeOperations.NAMESPACE)
-            .register(AtomixDocumentTreeEvents.NAMESPACE)
-            .build());
-
-    private final Map<DocumentTreeListener<byte[]>, InternalListener> eventListeners = new ConcurrentHashMap<>();
-
-    public AtomixDocumentTree(RaftProxy proxy) {
-        super(proxy);
-        proxy.addStateChangeListener(state -> {
-            if (state == RaftProxy.State.CONNECTED && isListening()) {
-                proxy.invoke(ADD_LISTENER, SERIALIZER::encode, new Listen());
-            }
-        });
-        proxy.addEventListener(CHANGE, SERIALIZER::decode, this::processTreeUpdates);
-    }
-
-    @Override
-    public Type primitiveType() {
-        return Type.DOCUMENT_TREE;
-    }
-
-    @Override
-    public CompletableFuture<Void> destroy() {
-        return proxy.invoke(CLEAR);
-    }
-
-    @Override
-    public DocumentPath root() {
-        return DocumentPath.ROOT;
-    }
-
-    @Override
-    public CompletableFuture<Map<String, Versioned<byte[]>>> getChildren(DocumentPath path) {
-        return proxy.<GetChildren, DocumentTreeResult<Map<String, Versioned<byte[]>>>>invoke(
-                GET_CHILDREN,
-                SERIALIZER::encode,
-                new GetChildren(checkNotNull(path)),
-                SERIALIZER::decode)
-                .thenCompose(result -> {
-                    if (result.status() == INVALID_PATH) {
-                        return Tools.exceptionalFuture(new NoSuchDocumentPathException());
-                    } else if (result.status() == ILLEGAL_MODIFICATION) {
-                        return Tools.exceptionalFuture(new IllegalDocumentModificationException());
-                    } else {
-                        return CompletableFuture.completedFuture(result);
-                    }
-                }).thenApply(result -> result.result());
-    }
-
-    @Override
-    public CompletableFuture<Versioned<byte[]>> get(DocumentPath path) {
-        return proxy.invoke(GET, SERIALIZER::encode, new Get(checkNotNull(path)), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Versioned<byte[]>> set(DocumentPath path, byte[] value) {
-        return proxy.<Update, DocumentTreeResult<Versioned<byte[]>>>invoke(UPDATE,
-                SERIALIZER::encode,
-                new Update(checkNotNull(path), Optional.ofNullable(value), Match.any(), Match.any()),
-                SERIALIZER::decode)
-                .thenCompose(result -> {
-                    if (result.status() == INVALID_PATH) {
-                        return Tools.exceptionalFuture(new NoSuchDocumentPathException());
-                    } else if (result.status() == ILLEGAL_MODIFICATION) {
-                        return Tools.exceptionalFuture(new IllegalDocumentModificationException());
-                    } else {
-                        return CompletableFuture.completedFuture(result);
-                    }
-                }).thenApply(result -> result.result());
-    }
-
-    @Override
-    public CompletableFuture<Boolean> create(DocumentPath path, byte[] value) {
-        return createInternal(path, value)
-                .thenCompose(status -> {
-                    if (status == ILLEGAL_MODIFICATION) {
-                        return Tools.exceptionalFuture(new IllegalDocumentModificationException());
-                    }
-                    return CompletableFuture.completedFuture(true);
-                });
-    }
-
-    @Override
-    public CompletableFuture<Boolean> createRecursive(DocumentPath path, byte[] value) {
-        return createInternal(path, value)
-                .thenCompose(status -> {
-                    if (status == ILLEGAL_MODIFICATION) {
-                        return createRecursive(path.parent(), null)
-                                .thenCompose(r -> createInternal(path, value).thenApply(v -> true));
-                    }
-                    return CompletableFuture.completedFuture(status == OK);
-                });
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(DocumentPath path, byte[] newValue, long version) {
-        return proxy.<Update, DocumentTreeResult<byte[]>>invoke(UPDATE,
-                SERIALIZER::encode,
-                new Update(checkNotNull(path),
-                        Optional.ofNullable(newValue),
-                        Match.any(),
-                        Match.ifValue(version)), SERIALIZER::decode)
-                .thenApply(result -> result.updated());
-    }
-
-    @Override
-    public CompletableFuture<Boolean> replace(DocumentPath path, byte[] newValue, byte[] currentValue) {
-        return proxy.<Update, DocumentTreeResult<byte[]>>invoke(UPDATE,
-                SERIALIZER::encode,
-                new Update(checkNotNull(path),
-                        Optional.ofNullable(newValue),
-                        Match.ifValue(currentValue),
-                        Match.any()),
-                SERIALIZER::decode)
-                .thenCompose(result -> {
-                    if (result.status() == INVALID_PATH) {
-                        return Tools.exceptionalFuture(new NoSuchDocumentPathException());
-                    } else if (result.status() == ILLEGAL_MODIFICATION) {
-                        return Tools.exceptionalFuture(new IllegalDocumentModificationException());
-                    } else {
-                        return CompletableFuture.completedFuture(result);
-                    }
-                }).thenApply(result -> result.updated());
-    }
-
-    @Override
-    public CompletableFuture<Versioned<byte[]>> removeNode(DocumentPath path) {
-        if (path.equals(DocumentPath.from("root"))) {
-            return Tools.exceptionalFuture(new IllegalDocumentModificationException());
-        }
-        return proxy.<Update, DocumentTreeResult<Versioned<byte[]>>>invoke(UPDATE,
-                SERIALIZER::encode,
-                new Update(checkNotNull(path), null, Match.any(), Match.ifNotNull()),
-                SERIALIZER::decode)
-                .thenCompose(result -> {
-                    if (result.status() == INVALID_PATH) {
-                        return Tools.exceptionalFuture(new NoSuchDocumentPathException());
-                    } else if (result.status() == ILLEGAL_MODIFICATION) {
-                        return Tools.exceptionalFuture(new IllegalDocumentModificationException());
-                    } else {
-                        return CompletableFuture.completedFuture(result);
-                    }
-                }).thenApply(result -> result.result());
-    }
-
-    @Override
-    public CompletableFuture<Void> addListener(DocumentPath path, DocumentTreeListener<byte[]> listener) {
-        checkNotNull(path);
-        checkNotNull(listener);
-        InternalListener internalListener = new InternalListener(path, listener, MoreExecutors.directExecutor());
-        // TODO: Support API that takes an executor
-        if (!eventListeners.containsKey(listener)) {
-            return proxy.invoke(ADD_LISTENER, SERIALIZER::encode, new Listen(path))
-                    .thenRun(() -> eventListeners.put(listener, internalListener));
-        }
-        return CompletableFuture.completedFuture(null);
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(DocumentTreeListener<byte[]> listener) {
-        checkNotNull(listener);
-        InternalListener internalListener = eventListeners.remove(listener);
-        if  (internalListener != null && eventListeners.isEmpty()) {
-            return proxy.invoke(REMOVE_LISTENER, SERIALIZER::encode, new Unlisten(internalListener.path))
-                    .thenApply(v -> null);
-        }
-        return CompletableFuture.completedFuture(null);
-    }
-
-    private CompletableFuture<DocumentTreeResult.Status> createInternal(DocumentPath path, byte[] value) {
-        return proxy.<Update, DocumentTreeResult<byte[]>>invoke(UPDATE,
-                SERIALIZER::encode,
-                new Update(checkNotNull(path), Optional.ofNullable(value), Match.any(), Match.ifNull()),
-                SERIALIZER::decode)
-                .thenApply(result -> result.status());
-    }
-
-    private boolean isListening() {
-        return !eventListeners.isEmpty();
-    }
-
-    private void processTreeUpdates(List<DocumentTreeEvent<byte[]>> events) {
-        events.forEach(event -> eventListeners.values().forEach(listener -> listener.event(event)));
-    }
-
-    @Override
-    public CompletableFuture<Version> begin(TransactionId transactionId) {
-        return proxy.<TransactionBegin, Long>invoke(
-                BEGIN,
-                SERIALIZER::encode,
-                new TransactionBegin(transactionId),
-                SERIALIZER::decode)
-                .thenApply(Version::new);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepare(TransactionLog<NodeUpdate<byte[]>> transactionLog) {
-        return proxy.<TransactionPrepare, PrepareResult>invoke(
-                PREPARE,
-                SERIALIZER::encode,
-                new TransactionPrepare(transactionLog),
-                SERIALIZER::decode)
-                .thenApply(v -> v == PrepareResult.OK);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> prepareAndCommit(TransactionLog<NodeUpdate<byte[]>> transactionLog) {
-        return proxy.<TransactionPrepareAndCommit, PrepareResult>invoke(
-                PREPARE_AND_COMMIT,
-                SERIALIZER::encode,
-                new TransactionPrepareAndCommit(transactionLog),
-                SERIALIZER::decode)
-                .thenApply(v -> v == PrepareResult.OK);
-    }
-
-    @Override
-    public CompletableFuture<Void> commit(TransactionId transactionId) {
-        return proxy.<TransactionCommit, CommitResult>invoke(
-                COMMIT,
-                SERIALIZER::encode,
-                new TransactionCommit(transactionId),
-                SERIALIZER::decode)
-                .thenApply(v -> null);
-    }
-
-    @Override
-    public CompletableFuture<Void> rollback(TransactionId transactionId) {
-        return proxy.invoke(
-                ROLLBACK,
-                SERIALIZER::encode,
-                new TransactionRollback(transactionId),
-                SERIALIZER::decode)
-                .thenApply(v -> null);
-    }
-
-    private class InternalListener implements DocumentTreeListener<byte[]> {
-
-        private final DocumentPath path;
-        private final DocumentTreeListener<byte[]> listener;
-        private final Executor executor;
-
-        public InternalListener(DocumentPath path, DocumentTreeListener<byte[]> listener, Executor executor) {
-            this.path = path;
-            this.listener = listener;
-            this.executor = executor;
-        }
-
-        @Override
-        public void event(DocumentTreeEvent<byte[]> event) {
-            if (event.path().isDescendentOf(path)) {
-                executor.execute(() -> listener.event(event));
-            }
-        }
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeEvents.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeEvents.java
deleted file mode 100644
index 47dcd31..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeEvents.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import io.atomix.protocols.raft.event.EventType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.DocumentTreeEvent;
-
-/**
- * Atomix document tree events.
- */
-public enum AtomixDocumentTreeEvents implements EventType {
-    CHANGE;
-
-    @Override
-    public String id() {
-        return name();
-    }
-
-    public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
-            .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 50)
-            .register(DocumentTreeEvent.class)
-            .register(DocumentTreeEvent.Type.class)
-            .build("AtomixDocumentTreeEvents");
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeOperations.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeOperations.java
deleted file mode 100644
index 6ec6091..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeOperations.java
+++ /dev/null
@@ -1,370 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.LinkedHashMap;
-import java.util.Optional;
-
-import com.google.common.base.MoreObjects;
-import io.atomix.protocols.raft.operation.OperationId;
-import io.atomix.protocols.raft.operation.OperationType;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Match;
-import org.onosproject.store.primitives.NodeUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Versioned;
-
-/**
- * {@link AtomixDocumentTree} resource state machine operations.
- */
-public enum AtomixDocumentTreeOperations implements OperationId {
-    ADD_LISTENER(OperationType.COMMAND),
-    REMOVE_LISTENER(OperationType.COMMAND),
-    GET(OperationType.QUERY),
-    GET_CHILDREN(OperationType.QUERY),
-    UPDATE(OperationType.COMMAND),
-    CLEAR(OperationType.COMMAND),
-    BEGIN(OperationType.COMMAND),
-    PREPARE(OperationType.COMMAND),
-    PREPARE_AND_COMMIT(OperationType.COMMAND),
-    COMMIT(OperationType.COMMAND),
-    ROLLBACK(OperationType.COMMAND);
-
-    private final OperationType type;
-
-    AtomixDocumentTreeOperations(OperationType type) {
-        this.type = type;
-    }
-
-    @Override
-    public String id() {
-        return name();
-    }
-
-    @Override
-    public OperationType type() {
-        return type;
-    }
-
-    public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
-            .register(KryoNamespaces.BASIC)
-            .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
-            .register(LinkedHashMap.class)
-            .register(Listen.class)
-            .register(Unlisten.class)
-            .register(Get.class)
-            .register(GetChildren.class)
-            .register(Update.class)
-            .register(TransactionBegin.class)
-            .register(TransactionPrepare.class)
-            .register(TransactionPrepareAndCommit.class)
-            .register(TransactionCommit.class)
-            .register(TransactionRollback.class)
-            .register(TransactionId.class)
-            .register(TransactionLog.class)
-            .register(PrepareResult.class)
-            .register(CommitResult.class)
-            .register(RollbackResult.class)
-            .register(NodeUpdate.class)
-            .register(NodeUpdate.Type.class)
-            .register(DocumentPath.class)
-            .register(Match.class)
-            .register(Versioned.class)
-            .register(DocumentTreeResult.class)
-            .register(DocumentTreeResult.Status.class)
-            .build("AtomixDocumentTreeOperations");
-
-    /**
-     * Base class for document tree operations.
-     */
-    public abstract static class DocumentTreeOperation {
-    }
-
-    /**
-     * Base class for document tree operations that serialize a {@link DocumentPath}.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class PathOperation extends DocumentTreeOperation {
-        private DocumentPath path;
-
-        PathOperation(DocumentPath path) {
-            this.path = path;
-        }
-
-        public DocumentPath path() {
-            return path;
-        }
-    }
-
-    /**
-     * DocumentTree#get query.
-     */
-    @SuppressWarnings("serial")
-    public static class Get extends PathOperation {
-        public Get() {
-            super(null);
-        }
-
-        public Get(DocumentPath path) {
-            super(path);
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("path", path())
-                    .toString();
-        }
-    }
-
-    /**
-     * DocumentTree#getChildren query.
-     */
-    @SuppressWarnings("serial")
-    public static class GetChildren extends PathOperation {
-        public GetChildren() {
-            super(null);
-        }
-
-        public GetChildren(DocumentPath path) {
-            super(path);
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("path", path())
-                    .toString();
-        }
-    }
-
-    /**
-     * DocumentTree update command.
-     */
-    @SuppressWarnings("serial")
-    public static class Update extends PathOperation {
-        private Optional<byte[]> value;
-        private Match<byte[]> valueMatch;
-        private Match<Long> versionMatch;
-
-        public Update() {
-            super(null);
-            this.value = null;
-            this.valueMatch = null;
-            this.versionMatch = null;
-        }
-
-        public Update(DocumentPath path, Optional<byte[]> value, Match<byte[]> valueMatch, Match<Long> versionMatch) {
-            super(path);
-            this.value = value;
-            this.valueMatch = valueMatch;
-            this.versionMatch = versionMatch;
-        }
-
-        public Optional<byte[]> value() {
-            return value;
-        }
-
-        public Match<byte[]> valueMatch() {
-            return valueMatch;
-        }
-
-        public Match<Long> versionMatch() {
-            return versionMatch;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("path", path())
-                    .add("value", value)
-                    .add("valueMatch", valueMatch)
-                    .add("versionMatch", versionMatch)
-                    .toString();
-        }
-    }
-
-    /**
-     * Change listen.
-     */
-    @SuppressWarnings("serial")
-    public static class Listen extends PathOperation {
-        public Listen() {
-            this(DocumentPath.from("root"));
-        }
-
-        public Listen(DocumentPath path) {
-            super(path);
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("path", path())
-                    .toString();
-        }
-    }
-
-    /**
-     * Change unlisten.
-     */
-    @SuppressWarnings("serial")
-    public static class Unlisten extends PathOperation {
-        public Unlisten() {
-            this(DocumentPath.from("root"));
-        }
-
-        public Unlisten(DocumentPath path) {
-            super(path);
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("path", path())
-                    .toString();
-        }
-    }
-
-    /**
-     * Transaction begin command.
-     */
-    public static class TransactionBegin extends PathOperation {
-        private TransactionId transactionId;
-
-        public TransactionBegin() {
-            super(null);
-        }
-
-        public TransactionBegin(TransactionId transactionId) {
-            super(DocumentPath.from(transactionId.toString()));
-            this.transactionId = transactionId;
-        }
-
-        public TransactionId transactionId() {
-            return transactionId;
-        }
-    }
-
-    /**
-     * Transaction prepare command.
-     */
-    @SuppressWarnings("serial")
-    public static class TransactionPrepare extends PathOperation {
-        private TransactionLog<NodeUpdate<byte[]>> transactionLog;
-
-        public TransactionPrepare() {
-            super(null);
-        }
-
-        public TransactionPrepare(TransactionLog<NodeUpdate<byte[]>> transactionLog) {
-            super(DocumentPath.from(transactionLog.transactionId().toString()));
-            this.transactionLog = transactionLog;
-        }
-
-        public TransactionLog<NodeUpdate<byte[]>> transactionLog() {
-            return transactionLog;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("transactionLog", transactionLog)
-                    .toString();
-        }
-    }
-
-    /**
-     * Transaction prepareAndCommit command.
-     */
-    @SuppressWarnings("serial")
-    public static class TransactionPrepareAndCommit extends TransactionPrepare {
-        public TransactionPrepareAndCommit() {
-        }
-
-        public TransactionPrepareAndCommit(TransactionLog<NodeUpdate<byte[]>> transactionLog) {
-            super(transactionLog);
-        }
-    }
-
-    /**
-     * Transaction commit command.
-     */
-    @SuppressWarnings("serial")
-    public static class TransactionCommit extends PathOperation {
-        private TransactionId transactionId;
-
-        public TransactionCommit() {
-            super(null);
-        }
-
-        public TransactionCommit(TransactionId transactionId) {
-            super(DocumentPath.from(transactionId.toString()));
-            this.transactionId = transactionId;
-        }
-
-        /**
-         * Returns the transaction identifier.
-         * @return transaction id
-         */
-        public TransactionId transactionId() {
-            return transactionId;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("transactionId", transactionId)
-                    .toString();
-        }
-    }
-
-    /**
-     * Transaction rollback command.
-     */
-    @SuppressWarnings("serial")
-    public static class TransactionRollback extends PathOperation {
-        private TransactionId transactionId;
-
-        public TransactionRollback() {
-            super(null);
-        }
-
-        public TransactionRollback(TransactionId transactionId) {
-            super(DocumentPath.from(transactionId.toString()));
-            this.transactionId = transactionId;
-        }
-
-        /**
-         * Returns the transaction identifier.
-         * @return transaction id
-         */
-        public TransactionId transactionId() {
-            return transactionId;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("transactionId", transactionId)
-                    .toString();
-        }
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeService.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeService.java
deleted file mode 100644
index 41a1a9f..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeService.java
+++ /dev/null
@@ -1,622 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Queue;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.stream.Collectors;
-
-import com.esotericsoftware.kryo.Kryo;
-import com.esotericsoftware.kryo.io.Input;
-import com.esotericsoftware.kryo.io.Output;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Queues;
-import com.google.common.collect.Sets;
-import io.atomix.protocols.raft.event.EventType;
-import io.atomix.protocols.raft.service.AbstractRaftService;
-import io.atomix.protocols.raft.service.Commit;
-import io.atomix.protocols.raft.service.RaftServiceExecutor;
-import io.atomix.protocols.raft.session.RaftSession;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import org.onlab.util.KryoNamespace;
-import org.onlab.util.Match;
-import org.onosproject.store.primitives.NodeUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.Get;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.GetChildren;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.Listen;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.Unlisten;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.Update;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.TransactionBegin;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.TransactionPrepare;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.TransactionPrepareAndCommit;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.TransactionCommit;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.TransactionRollback;
-
-import org.onosproject.store.primitives.resources.impl.DocumentTreeResult.Status;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTree;
-import org.onosproject.store.service.DocumentTreeEvent;
-import org.onosproject.store.service.DocumentTreeEvent.Type;
-import org.onosproject.store.service.IllegalDocumentModificationException;
-import org.onosproject.store.service.NoSuchDocumentPathException;
-import org.onosproject.store.service.Ordering;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Versioned;
-
-import static com.google.common.base.Preconditions.checkState;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeEvents.CHANGE;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.BEGIN;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.PREPARE;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.PREPARE_AND_COMMIT;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.COMMIT;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.ROLLBACK;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.CLEAR;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.GET;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.GET_CHILDREN;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.UPDATE;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.ADD_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.REMOVE_LISTENER;
-
-/**
- * State Machine for {@link AtomixDocumentTree} resource.
- */
-public class AtomixDocumentTreeService extends AbstractRaftService {
-    private final Serializer serializer = Serializer.using(KryoNamespace.newBuilder()
-            .register(KryoNamespaces.BASIC)
-            .register(AtomixDocumentTreeOperations.NAMESPACE)
-            .register(AtomixDocumentTreeEvents.NAMESPACE)
-            .register(new com.esotericsoftware.kryo.Serializer<Listener>() {
-                @Override
-                public void write(Kryo kryo, Output output, Listener listener) {
-                    output.writeLong(listener.session.sessionId().id());
-                    kryo.writeObject(output, listener.path);
-                }
-
-                @Override
-                public Listener read(Kryo kryo, Input input, Class<Listener> type) {
-                    return new Listener(sessions().getSession(input.readLong()),
-                            kryo.readObjectOrNull(input, DocumentPath.class));
-                }
-            }, Listener.class)
-            .register(Versioned.class)
-            .register(DocumentPath.class)
-            .register(new LinkedHashMap().keySet().getClass())
-            .register(TreeMap.class)
-            .register(Ordering.class)
-            .register(TransactionScope.class)
-            .register(TransactionLog.class)
-            .register(TransactionId.class)
-            .register(SessionListenCommits.class)
-            .register(new com.esotericsoftware.kryo.Serializer<DefaultDocumentTree>() {
-                @Override
-                public void write(Kryo kryo, Output output, DefaultDocumentTree object) {
-                    kryo.writeObject(output, object.root);
-                }
-
-                @Override
-                @SuppressWarnings("unchecked")
-                public DefaultDocumentTree read(Kryo kryo, Input input, Class<DefaultDocumentTree> type) {
-                    return new DefaultDocumentTree(versionCounter::incrementAndGet,
-                            kryo.readObject(input, DefaultDocumentTreeNode.class));
-                }
-            }, DefaultDocumentTree.class)
-            .register(DefaultDocumentTreeNode.class)
-            .build());
-
-    private Map<Long, SessionListenCommits> listeners = new HashMap<>();
-    private AtomicLong versionCounter = new AtomicLong(0);
-    private DocumentTree<byte[]> docTree;
-    private Map<TransactionId, TransactionScope> activeTransactions = Maps.newHashMap();
-    private Set<DocumentPath> preparedKeys = Sets.newHashSet();
-
-    public AtomixDocumentTreeService(Ordering ordering) {
-        this.docTree = new DefaultDocumentTree<>(versionCounter::incrementAndGet, ordering);
-    }
-
-    @Override
-    public void snapshot(SnapshotWriter writer) {
-        writer.writeLong(versionCounter.get());
-        writer.writeObject(listeners, serializer::encode);
-        writer.writeObject(docTree, serializer::encode);
-        writer.writeObject(preparedKeys, serializer::encode);
-        writer.writeObject(activeTransactions, serializer::encode);
-    }
-
-    @Override
-    public void install(SnapshotReader reader) {
-        versionCounter = new AtomicLong(reader.readLong());
-        listeners = reader.readObject(serializer::decode);
-        docTree = reader.readObject(serializer::decode);
-        preparedKeys = reader.readObject(serializer::decode);
-        activeTransactions = reader.readObject(serializer::decode);
-    }
-
-    @Override
-    protected void configure(RaftServiceExecutor executor) {
-        // Listeners
-        executor.register(ADD_LISTENER, serializer::decode, this::listen);
-        executor.register(REMOVE_LISTENER, serializer::decode, this::unlisten);
-        // queries
-        executor.register(GET, serializer::decode, this::get, serializer::encode);
-        executor.register(GET_CHILDREN, serializer::decode, this::getChildren, serializer::encode);
-        // commands
-        executor.register(UPDATE, serializer::decode, this::update, serializer::encode);
-        executor.register(CLEAR, this::clear);
-        executor.register(BEGIN, serializer::decode, this::begin, serializer::encode);
-        executor.register(PREPARE, serializer::decode, this::prepare, serializer::encode);
-        executor.register(PREPARE_AND_COMMIT, serializer::decode, this::prepareAndCommit, serializer::encode);
-        executor.register(COMMIT, serializer::decode, this::commit, serializer::encode);
-        executor.register(ROLLBACK, serializer::decode, this::rollback, serializer::encode);
-    }
-
-    /**
-     * Returns a boolean indicating whether the given path is currently locked by a transaction.
-     *
-     * @param path the path to check
-     * @return whether the given path is locked by a running transaction
-     */
-    private boolean isLocked(DocumentPath path) {
-        return preparedKeys.contains(path);
-    }
-
-    protected void listen(Commit<? extends Listen> commit) {
-        Long sessionId = commit.session().sessionId().id();
-        listeners.computeIfAbsent(sessionId, k -> new SessionListenCommits())
-                .add(new Listener(commit.session(), commit.value().path()));
-    }
-
-    protected void unlisten(Commit<? extends Unlisten> commit) {
-        Long sessionId = commit.session().sessionId().id();
-        SessionListenCommits listenCommits = listeners.get(sessionId);
-        if (listenCommits != null) {
-            listenCommits.remove(commit);
-        }
-    }
-
-    protected Versioned<byte[]> get(Commit<? extends Get> commit) {
-        try {
-            Versioned<byte[]> value = docTree.get(commit.value().path());
-            return value == null ? null : value.map(node -> node);
-        } catch (IllegalStateException e) {
-            return null;
-        }
-    }
-
-    protected DocumentTreeResult<Map<String, Versioned<byte[]>>> getChildren(Commit<? extends GetChildren> commit) {
-        try {
-            return DocumentTreeResult.ok(docTree.getChildren(commit.value().path()));
-        } catch (NoSuchDocumentPathException e) {
-            return DocumentTreeResult.invalidPath();
-        }
-    }
-
-    protected DocumentTreeResult<Versioned<byte[]>> update(Commit<? extends Update> commit) {
-        DocumentTreeResult<Versioned<byte[]>> result = null;
-        DocumentPath path = commit.value().path();
-
-        // If the path is locked by a transaction, return a WRITE_LOCK error.
-        if (isLocked(path)) {
-            return DocumentTreeResult.writeLock();
-        }
-
-        Versioned<byte[]> currentValue = docTree.get(path);
-        try {
-            Match<Long> versionMatch = commit.value().versionMatch();
-            Match<byte[]> valueMatch = commit.value().valueMatch();
-
-            if (versionMatch.matches(currentValue == null ? null : currentValue.version())
-                    && valueMatch.matches(currentValue == null ? null : currentValue.value())) {
-                if (commit.value().value() == null) {
-                    Versioned<byte[]> oldValue = docTree.removeNode(path);
-                    result = new DocumentTreeResult<>(Status.OK, oldValue);
-                    if (oldValue != null) {
-                        notifyListeners(new DocumentTreeEvent<>(
-                                path,
-                                Type.DELETED,
-                                Optional.empty(),
-                                Optional.of(oldValue)));
-                    }
-                } else {
-                    Versioned<byte[]> oldValue = docTree.set(path, commit.value().value().orElse(null));
-                    Versioned<byte[]> newValue = docTree.get(path);
-                    result = new DocumentTreeResult<>(Status.OK, newValue);
-                    if (oldValue == null) {
-                        notifyListeners(new DocumentTreeEvent<>(
-                                path,
-                                Type.CREATED,
-                                Optional.of(newValue),
-                                Optional.empty()));
-                    } else {
-                        notifyListeners(new DocumentTreeEvent<>(
-                                path,
-                                Type.UPDATED,
-                                Optional.of(newValue),
-                                Optional.of(oldValue)));
-                    }
-                }
-            } else {
-                result = new DocumentTreeResult<>(
-                        commit.value().value() == null ? Status.INVALID_PATH : Status.NOOP,
-                        currentValue);
-            }
-        } catch (IllegalDocumentModificationException e) {
-            result = DocumentTreeResult.illegalModification();
-        } catch (NoSuchDocumentPathException e) {
-            result = DocumentTreeResult.invalidPath();
-        } catch (Exception e) {
-            logger().error("Failed to apply {} to state machine", commit.value(), e);
-            throw new IllegalStateException(e);
-        }
-        return result;
-    }
-
-    protected void clear(Commit<Void> commit) {
-        Queue<DocumentPath> toClearQueue = Queues.newArrayDeque();
-        Map<String, Versioned<byte[]>> topLevelChildren = docTree.getChildren(DocumentPath.from("root"));
-        toClearQueue.addAll(topLevelChildren.keySet()
-                .stream()
-                .map(name -> new DocumentPath(name, DocumentPath.from("root")))
-                .collect(Collectors.toList()));
-        while (!toClearQueue.isEmpty()) {
-            DocumentPath path = toClearQueue.remove();
-            Map<String, Versioned<byte[]>> children = docTree.getChildren(path);
-            if (children.size() == 0) {
-                docTree.removeNode(path);
-            } else {
-                children.keySet().forEach(name -> toClearQueue.add(new DocumentPath(name, path)));
-                toClearQueue.add(path);
-            }
-        }
-    }
-
-    /**
-     * Handles a begin commit.
-     *
-     * @param commit transaction begin commit
-     * @return transaction state version
-     */
-    protected long begin(Commit<? extends TransactionBegin> commit) {
-        long version = commit.index();
-        activeTransactions.put(commit.value().transactionId(), new TransactionScope(version));
-        return version;
-    }
-
-    /**
-     * Handles an prepare commit.
-     *
-     * @param commit transaction prepare commit
-     * @return prepare result
-     */
-    protected PrepareResult prepare(Commit<? extends TransactionPrepare> commit) {
-        try {
-            TransactionLog<NodeUpdate<byte[]>> transactionLog = commit.value().transactionLog();
-            // Iterate through records in the transaction log and perform isolation checks.
-            for (NodeUpdate<byte[]> record : transactionLog.records()) {
-                DocumentPath path = record.path();
-
-                // If the prepared keys already contains the key contained within the record, that indicates a
-                // conflict with a concurrent transaction.
-                if (preparedKeys.contains(path)) {
-                    return PrepareResult.CONCURRENT_TRANSACTION;
-                }
-
-                // Read the existing value from the map.
-                Versioned<byte[]> existingValue = docTree.get(path);
-
-                // If the update is an UPDATE_NODE or DELETE_NODE, verify that versions match.
-                switch (record.type()) {
-                    case UPDATE_NODE:
-                    case DELETE_NODE:
-                        if (existingValue == null || existingValue.version() != record.version()) {
-                            return PrepareResult.OPTIMISTIC_LOCK_FAILURE;
-                        }
-                    default:
-                        break;
-                }
-            }
-
-            // No violations detected. Mark modified keys locked for transactions.
-            transactionLog.records().forEach(record -> {
-                preparedKeys.add(record.path());
-            });
-
-            // Update the transaction scope. If the transaction scope is not set on this node, that indicates the
-            // coordinator is communicating with another node. Transactions assume that the client is communicating
-            // with a single leader in order to limit the overhead of retaining tombstones.
-            TransactionScope transactionScope = activeTransactions.get(transactionLog.transactionId());
-            if (transactionScope == null) {
-                activeTransactions.put(
-                        transactionLog.transactionId(),
-                        new TransactionScope(transactionLog.version(), commit.value().transactionLog()));
-                return PrepareResult.PARTIAL_FAILURE;
-            } else {
-                activeTransactions.put(
-                        transactionLog.transactionId(),
-                        transactionScope.prepared(commit));
-                return PrepareResult.OK;
-            }
-        } catch (Exception e) {
-            logger().warn("Failure applying {}", commit, e);
-            throw new IllegalStateException(e);
-        }
-    }
-
-    /**
-     * Handles an prepare and commit commit.
-     *
-     * @param commit transaction prepare and commit commit
-     * @return prepare result
-     */
-    protected PrepareResult prepareAndCommit(Commit<? extends TransactionPrepareAndCommit> commit) {
-        TransactionId transactionId = commit.value().transactionLog().transactionId();
-        PrepareResult prepareResult = prepare(commit);
-        TransactionScope transactionScope = activeTransactions.remove(transactionId);
-        if (prepareResult == PrepareResult.OK) {
-            transactionScope = transactionScope.prepared(commit);
-            commitTransaction(transactionScope);
-        }
-        return prepareResult;
-    }
-
-    /**
-     * Applies committed operations to the state machine.
-     */
-    private CommitResult commitTransaction(TransactionScope transactionScope) {
-        TransactionLog<NodeUpdate<byte[]>> transactionLog = transactionScope.transactionLog();
-
-        List<DocumentTreeEvent<byte[]>> eventsToPublish = Lists.newArrayList();
-        DocumentTreeEvent<byte[]> start = new DocumentTreeEvent<>(
-                DocumentPath.from(transactionScope.transactionLog().transactionId().toString()),
-                Type.TRANSACTION_START,
-                Optional.empty(),
-                Optional.empty());
-        eventsToPublish.add(start);
-
-        for (NodeUpdate<byte[]> record : transactionLog.records()) {
-            DocumentPath path = record.path();
-            checkState(preparedKeys.remove(path), "path is not prepared");
-
-            // FIXME revisit this block, it never respects NodeUpdate type
-
-            Versioned<byte[]> previousValue = null;
-            try {
-                previousValue = docTree.removeNode(path);
-            } catch (NoSuchDocumentPathException e) {
-                logger().info("Value is being inserted first time");
-            }
-
-            if (record.value() != null) {
-                if (docTree.create(path, record.value())) {
-                    Versioned<byte[]> newValue = docTree.get(path);
-                    eventsToPublish.add(new DocumentTreeEvent<>(
-                            path,
-                            Optional.ofNullable(newValue),
-                            Optional.ofNullable(previousValue)));
-                }
-            } else if (previousValue != null) {
-                eventsToPublish.add(new DocumentTreeEvent<>(
-                        path,
-                        Optional.empty(),
-                        Optional.of(previousValue)));
-            }
-        }
-
-        DocumentTreeEvent<byte[]> end = new DocumentTreeEvent<>(
-                DocumentPath.from(transactionScope.transactionLog().transactionId().toString()),
-                Type.TRANSACTION_END,
-                Optional.empty(),
-                Optional.empty());
-        eventsToPublish.add(end);
-        publish(eventsToPublish);
-
-        return CommitResult.OK;
-    }
-
-    /**
-     * Handles an commit commit (ha!).
-     *
-     * @param commit transaction commit commit
-     * @return commit result
-     */
-    protected CommitResult commit(Commit<? extends TransactionCommit> commit) {
-        TransactionId transactionId = commit.value().transactionId();
-        TransactionScope transactionScope = activeTransactions.remove(transactionId);
-        if (transactionScope == null) {
-            return CommitResult.UNKNOWN_TRANSACTION_ID;
-        }
-        try {
-            return commitTransaction(transactionScope);
-        } catch (Exception e) {
-            logger().warn("Failure applying {}", commit, e);
-            throw new IllegalStateException(e);
-        }
-    }
-
-    /**
-     * Handles an rollback commit (ha!).
-     *
-     * @param commit transaction rollback commit
-     * @return rollback result
-     */
-    protected RollbackResult rollback(Commit<? extends TransactionRollback> commit) {
-        TransactionId transactionId = commit.value().transactionId();
-        TransactionScope transactionScope = activeTransactions.remove(transactionId);
-        if (transactionScope == null) {
-            return RollbackResult.UNKNOWN_TRANSACTION_ID;
-        } else if (!transactionScope.isPrepared()) {
-            return RollbackResult.OK;
-        } else {
-            transactionScope.transactionLog().records()
-                    .forEach(record -> {
-                        preparedKeys.remove(record.path());
-                    });
-            return RollbackResult.OK;
-        }
-
-    }
-
-    /**
-     * Map transaction scope.
-     */
-    private static final class TransactionScope {
-        private final long version;
-        private final TransactionLog<NodeUpdate<byte[]>> transactionLog;
-
-        private TransactionScope(long version) {
-            this(version, null);
-        }
-
-        private TransactionScope(long version, TransactionLog<NodeUpdate<byte[]>> transactionLog) {
-            this.version = version;
-            this.transactionLog = transactionLog;
-        }
-
-        /**
-         * Returns the transaction version.
-         *
-         * @return the transaction version
-         */
-        long version() {
-            return version;
-        }
-
-        /**
-         * Returns whether this is a prepared transaction scope.
-         *
-         * @return whether this is a prepared transaction scope
-         */
-        boolean isPrepared() {
-            return transactionLog != null;
-        }
-
-        /**
-         * Returns the transaction commit log.
-         *
-         * @return the transaction commit log
-         */
-        TransactionLog<NodeUpdate<byte[]>> transactionLog() {
-            checkState(isPrepared(), "Transaction not prepared");
-            return transactionLog;
-        }
-
-        /**
-         * Returns a new transaction scope with a prepare commit.
-         *
-         * @param commit the prepare commit
-         * @return new transaction scope updated with the prepare commit
-         */
-        TransactionScope prepared(Commit<? extends TransactionPrepare> commit) {
-            return new TransactionScope(version, commit.value().transactionLog());
-        }
-    }
-
-    private void publish(List<DocumentTreeEvent<byte[]>> events) {
-        listeners.values().forEach(session -> {
-            session.publish(CHANGE, events);
-        });
-    }
-
-    private void notifyListeners(DocumentTreeEvent<byte[]> event) {
-        listeners.values()
-                .stream()
-                .filter(l -> event.path().isDescendentOf(l.leastCommonAncestorPath()))
-                .forEach(listener -> listener.publish(CHANGE, Arrays.asList(event)));
-    }
-
-    @Override
-    public void onExpire(RaftSession session) {
-        closeListener(session.sessionId().id());
-    }
-
-    @Override
-    public void onClose(RaftSession session) {
-        closeListener(session.sessionId().id());
-    }
-
-    private void closeListener(Long sessionId) {
-        listeners.remove(sessionId);
-    }
-
-    private class SessionListenCommits {
-        private final List<Listener> listeners = Lists.newArrayList();
-        private DocumentPath leastCommonAncestorPath;
-
-        public void add(Listener listener) {
-            listeners.add(listener);
-            recomputeLeastCommonAncestor();
-        }
-
-        public void remove(Commit<? extends Unlisten> commit) {
-            // Remove the first listen commit with path matching path in unlisten commit
-            Iterator<Listener> iterator = listeners.iterator();
-            while (iterator.hasNext()) {
-                Listener listener = iterator.next();
-                if (listener.path().equals(commit.value().path())) {
-                    iterator.remove();
-                }
-            }
-            recomputeLeastCommonAncestor();
-        }
-
-        public DocumentPath leastCommonAncestorPath() {
-            return leastCommonAncestorPath;
-        }
-
-        public <M> void publish(EventType topic, M message) {
-            listeners.stream().findAny().ifPresent(listener ->
-                    listener.session().publish(topic, serializer::encode, message));
-        }
-
-        private void recomputeLeastCommonAncestor() {
-            this.leastCommonAncestorPath = DocumentPath.leastCommonAncestor(listeners.stream()
-                    .map(Listener::path)
-                    .collect(Collectors.toList()));
-        }
-    }
-
-    private static class Listener {
-        private final RaftSession session;
-        private final DocumentPath path;
-
-        public Listener(RaftSession session, DocumentPath path) {
-            this.session = session;
-            this.path = path;
-        }
-
-        public DocumentPath path() {
-            return path;
-        }
-
-        public RaftSession session() {
-            return session;
-        }
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixIdGenerator.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixIdGenerator.java
deleted file mode 100644
index 0fa6c7b..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixIdGenerator.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.onosproject.store.service.AsyncAtomicCounter;
-import org.onosproject.store.service.AsyncAtomicIdGenerator;
-
-/**
- * {@code AsyncAtomicIdGenerator} implementation backed by Atomix
- * {@link AsyncAtomicCounter}.
- */
-public class AtomixIdGenerator implements AsyncAtomicIdGenerator {
-
-    private static final long DEFAULT_BATCH_SIZE = 1000;
-    private final AsyncAtomicCounter counter;
-    private final long batchSize;
-    private CompletableFuture<Long> reserveFuture;
-    private long base;
-    private final AtomicLong delta = new AtomicLong();
-
-    public AtomixIdGenerator(AsyncAtomicCounter counter) {
-        this(counter, DEFAULT_BATCH_SIZE);
-    }
-
-    AtomixIdGenerator(AsyncAtomicCounter counter, long batchSize) {
-        this.counter = counter;
-        this.batchSize = batchSize;
-    }
-
-    @Override
-    public String name() {
-        return counter.name();
-    }
-
-    @Override
-    public synchronized CompletableFuture<Long> nextId() {
-        long nextDelta = delta.incrementAndGet();
-        if ((base == 0 && reserveFuture == null) || nextDelta > batchSize) {
-            delta.set(0);
-            long delta = this.delta.incrementAndGet();
-            return reserve().thenApply(base -> base + delta);
-        } else {
-            return reserveFuture.thenApply(base -> base + nextDelta);
-        }
-    }
-
-    private CompletableFuture<Long> reserve() {
-        if (reserveFuture == null || reserveFuture.isDone()) {
-            reserveFuture = counter.getAndAdd(batchSize);
-        } else {
-            reserveFuture = reserveFuture.thenCompose(v -> counter.getAndAdd(batchSize));
-        }
-        reserveFuture = reserveFuture.thenApply(base -> {
-            this.base = base;
-            return base;
-        });
-        return reserveFuture;
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElector.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElector.java
deleted file mode 100644
index 53c7094..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElector.java
+++ /dev/null
@@ -1,139 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-import java.util.function.Consumer;
-
-import com.google.common.collect.Sets;
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.cluster.Leadership;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.event.Change;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.Anoint;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GetElectedTopics;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GetLeadership;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.Promote;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.Run;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.Withdraw;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.AsyncLeaderElector;
-import org.onosproject.store.service.Serializer;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorEvents.CHANGE;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.ADD_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.ANOINT;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.EVICT;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GET_ALL_LEADERSHIPS;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GET_ELECTED_TOPICS;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GET_LEADERSHIP;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.PROMOTE;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.REMOVE_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.RUN;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.WITHDRAW;
-
-/**
- * Distributed resource providing the {@link AsyncLeaderElector} primitive.
- */
-public class AtomixLeaderElector extends AbstractRaftPrimitive implements AsyncLeaderElector {
-    private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
-            .register(KryoNamespaces.API)
-            .register(AtomixLeaderElectorOperations.NAMESPACE)
-            .register(AtomixLeaderElectorEvents.NAMESPACE)
-            .build());
-
-    private final Set<Consumer<Change<Leadership>>> leadershipChangeListeners = Sets.newCopyOnWriteArraySet();
-
-    public AtomixLeaderElector(RaftProxy proxy) {
-        super(proxy);
-        proxy.addStateChangeListener(state -> {
-            if (state == RaftProxy.State.CONNECTED && isListening()) {
-                proxy.invoke(ADD_LISTENER);
-            }
-        });
-        proxy.addEventListener(CHANGE, SERIALIZER::decode, this::handleEvent);
-    }
-
-    private void handleEvent(List<Change<Leadership>> changes) {
-        changes.forEach(change -> leadershipChangeListeners.forEach(l -> l.accept(change)));
-    }
-
-    @Override
-    public CompletableFuture<Leadership> run(String topic, NodeId nodeId) {
-        return proxy.<Run, Leadership>invoke(RUN, SERIALIZER::encode, new Run(topic, nodeId), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Void> withdraw(String topic) {
-        return proxy.invoke(WITHDRAW, SERIALIZER::encode, new Withdraw(topic));
-    }
-
-    @Override
-    public CompletableFuture<Boolean> anoint(String topic, NodeId nodeId) {
-        return proxy.<Anoint, Boolean>invoke(ANOINT, SERIALIZER::encode, new Anoint(topic, nodeId), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Boolean> promote(String topic, NodeId nodeId) {
-        return proxy.<Promote, Boolean>invoke(
-                PROMOTE, SERIALIZER::encode, new Promote(topic, nodeId), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Void> evict(NodeId nodeId) {
-        return proxy.invoke(EVICT, SERIALIZER::encode, new AtomixLeaderElectorOperations.Evict(nodeId));
-    }
-
-    @Override
-    public CompletableFuture<Leadership> getLeadership(String topic) {
-        return proxy.invoke(GET_LEADERSHIP, SERIALIZER::encode, new GetLeadership(topic), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Map<String, Leadership>> getLeaderships() {
-        return proxy.invoke(GET_ALL_LEADERSHIPS, SERIALIZER::decode);
-    }
-
-    public CompletableFuture<Set<String>> getElectedTopics(NodeId nodeId) {
-        return proxy.invoke(GET_ELECTED_TOPICS, SERIALIZER::encode, new GetElectedTopics(nodeId), SERIALIZER::decode);
-    }
-
-    @Override
-    public synchronized CompletableFuture<Void> addChangeListener(Consumer<Change<Leadership>> consumer) {
-        if (leadershipChangeListeners.isEmpty()) {
-            return proxy.invoke(ADD_LISTENER).thenRun(() -> leadershipChangeListeners.add(consumer));
-        } else {
-            leadershipChangeListeners.add(consumer);
-            return CompletableFuture.completedFuture(null);
-        }
-    }
-
-    @Override
-    public synchronized CompletableFuture<Void> removeChangeListener(Consumer<Change<Leadership>> consumer) {
-        if (leadershipChangeListeners.remove(consumer) && leadershipChangeListeners.isEmpty()) {
-            return proxy.invoke(REMOVE_LISTENER).thenApply(v -> null);
-        }
-        return CompletableFuture.completedFuture(null);
-    }
-
-    private boolean isListening() {
-        return !leadershipChangeListeners.isEmpty();
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorEvents.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorEvents.java
deleted file mode 100644
index 3c5a2c7..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorEvents.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import io.atomix.protocols.raft.event.EventType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-
-/**
- * Atomix leader elector events.
- */
-public enum AtomixLeaderElectorEvents implements EventType {
-    CHANGE;
-
-    @Override
-    public String id() {
-        return name();
-    }
-
-    public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
-            .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 50)
-            .build("AtomixLeaderElectorEvents");
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorOperations.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorOperations.java
deleted file mode 100644
index c778cea..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorOperations.java
+++ /dev/null
@@ -1,331 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import com.google.common.base.MoreObjects;
-import io.atomix.protocols.raft.operation.OperationId;
-import io.atomix.protocols.raft.operation.OperationType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.store.serializers.KryoNamespaces;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * {@link AtomixLeaderElector} resource state machine operations.
- */
-public enum AtomixLeaderElectorOperations implements OperationId {
-    ADD_LISTENER(OperationType.COMMAND),
-    REMOVE_LISTENER(OperationType.COMMAND),
-    RUN(OperationType.COMMAND),
-    WITHDRAW(OperationType.COMMAND),
-    ANOINT(OperationType.COMMAND),
-    PROMOTE(OperationType.COMMAND),
-    EVICT(OperationType.COMMAND),
-    GET_LEADERSHIP(OperationType.QUERY),
-    GET_ALL_LEADERSHIPS(OperationType.QUERY),
-    GET_ELECTED_TOPICS(OperationType.QUERY);
-
-    private final OperationType type;
-
-    AtomixLeaderElectorOperations(OperationType type) {
-        this.type = type;
-    }
-
-    @Override
-    public String id() {
-        return name();
-    }
-
-    @Override
-    public OperationType type() {
-        return type;
-    }
-
-    public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
-            .register(KryoNamespaces.API)
-            .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
-            .register(Run.class)
-            .register(Withdraw.class)
-            .register(Anoint.class)
-            .register(Promote.class)
-            .register(Evict.class)
-            .register(GetLeadership.class)
-            .register(GetElectedTopics.class)
-            .build("AtomixLeaderElectorOperations");
-
-    /**
-     * Abstract election query.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class ElectionOperation {
-    }
-
-    /**
-     * Abstract election topic query.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class TopicOperation extends ElectionOperation {
-        String topic;
-
-        public TopicOperation() {
-        }
-
-        public TopicOperation(String topic) {
-            this.topic = checkNotNull(topic);
-        }
-
-        /**
-         * Returns the topic.
-         * @return topic
-         */
-        public String topic() {
-            return topic;
-        }
-    }
-
-    /**
-     * GetLeader query.
-     */
-    @SuppressWarnings("serial")
-    public static class GetLeadership extends TopicOperation {
-
-        public GetLeadership() {
-        }
-
-        public GetLeadership(String topic) {
-            super(topic);
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("topic", topic)
-                    .toString();
-        }
-    }
-
-    /**
-     * GetElectedTopics query.
-     */
-    @SuppressWarnings("serial")
-    public static class GetElectedTopics extends ElectionOperation {
-        private NodeId nodeId;
-
-        public GetElectedTopics() {
-        }
-
-        public GetElectedTopics(NodeId nodeId) {
-            checkArgument(nodeId != null, "nodeId cannot be null");
-            this.nodeId = nodeId;
-        }
-
-        /**
-         * Returns the nodeId to check.
-         *
-         * @return The nodeId to check.
-         */
-        public NodeId nodeId() {
-            return nodeId;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("nodeId", nodeId)
-                    .toString();
-        }
-    }
-
-    /**
-     * Enter and run for leadership.
-     */
-    @SuppressWarnings("serial")
-    public static class Run extends ElectionOperation {
-        private String topic;
-        private NodeId nodeId;
-
-        public Run() {
-        }
-
-        public Run(String topic, NodeId nodeId) {
-            this.topic = topic;
-            this.nodeId = nodeId;
-        }
-
-        /**
-         * Returns the topic.
-         *
-         * @return topic
-         */
-        public String topic() {
-            return topic;
-        }
-
-        /**
-         * Returns the nodeId.
-         *
-         * @return the nodeId
-         */
-        public NodeId nodeId() {
-            return nodeId;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("topic", topic)
-                    .add("nodeId", nodeId)
-                    .toString();
-        }
-    }
-
-    /**
-     * Withdraw from a leadership contest.
-     */
-    @SuppressWarnings("serial")
-    public static class Withdraw extends ElectionOperation {
-        private String topic;
-
-        public Withdraw() {
-        }
-
-        public Withdraw(String topic) {
-            this.topic = topic;
-        }
-
-        /**
-         * Returns the topic.
-         *
-         * @return The topic
-         */
-        public String topic() {
-            return topic;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("topic", topic)
-                    .toString();
-        }
-    }
-
-    /**
-     * Command for administratively changing the leadership state for a node.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class ElectionChangeOperation extends ElectionOperation  {
-        private String topic;
-        private NodeId nodeId;
-
-        ElectionChangeOperation() {
-            topic = null;
-            nodeId = null;
-        }
-
-        public ElectionChangeOperation(String topic, NodeId nodeId) {
-            this.topic = topic;
-            this.nodeId = nodeId;
-        }
-
-        /**
-         * Returns the topic.
-         *
-         * @return The topic
-         */
-        public String topic() {
-            return topic;
-        }
-
-        /**
-         * Returns the nodeId to make leader.
-         *
-         * @return The nodeId
-         */
-        public NodeId nodeId() {
-            return nodeId;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("topic", topic)
-                    .add("nodeId", nodeId)
-                    .toString();
-        }
-    }
-
-    /**
-     * Command for administratively anoint a node as leader.
-     */
-    @SuppressWarnings("serial")
-    public static class Anoint extends ElectionChangeOperation {
-
-        private Anoint() {
-        }
-
-        public Anoint(String topic, NodeId nodeId) {
-            super(topic, nodeId);
-        }
-    }
-
-    /**
-     * Command for administratively promote a node as top candidate.
-     */
-    @SuppressWarnings("serial")
-    public static class Promote extends ElectionChangeOperation {
-
-        private Promote() {
-        }
-
-        public Promote(String topic, NodeId nodeId) {
-            super(topic, nodeId);
-        }
-    }
-
-    /**
-     * Command for administratively evicting a node from all leadership topics.
-     */
-    @SuppressWarnings("serial")
-    public static class Evict extends ElectionOperation {
-        private NodeId nodeId;
-
-        public Evict() {
-        }
-
-        public Evict(NodeId nodeId) {
-            this.nodeId = nodeId;
-        }
-
-        /**
-         * Returns the node identifier.
-         *
-         * @return The nodeId
-         */
-        public NodeId nodeId() {
-            return nodeId;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("nodeId", nodeId)
-                    .toString();
-        }
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorService.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorService.java
deleted file mode 100644
index dcddcfc..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorService.java
+++ /dev/null
@@ -1,539 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.function.Supplier;
-import java.util.stream.Collectors;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Objects;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import io.atomix.protocols.raft.service.AbstractRaftService;
-import io.atomix.protocols.raft.service.Commit;
-import io.atomix.protocols.raft.service.RaftServiceExecutor;
-import io.atomix.protocols.raft.session.RaftSession;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.cluster.Leader;
-import org.onosproject.cluster.Leadership;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.event.Change;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.Anoint;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.Evict;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GetElectedTopics;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GetLeadership;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.Promote;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.Run;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.Withdraw;
-import org.onosproject.store.service.Serializer;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorEvents.CHANGE;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.ADD_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.ANOINT;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.EVICT;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GET_ALL_LEADERSHIPS;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GET_ELECTED_TOPICS;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.GET_LEADERSHIP;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.PROMOTE;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.REMOVE_LISTENER;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.RUN;
-import static org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorOperations.WITHDRAW;
-
-/**
- * State machine for {@link AtomixLeaderElector} resource.
- */
-public class AtomixLeaderElectorService extends AbstractRaftService {
-
-    private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
-            .register(AtomixLeaderElectorOperations.NAMESPACE)
-            .register(AtomixLeaderElectorEvents.NAMESPACE)
-            .register(ElectionState.class)
-            .register(Registration.class)
-            .register(new LinkedHashMap<>().keySet().getClass())
-            .build());
-
-    private Map<String, AtomicLong> termCounters = new HashMap<>();
-    private Map<String, ElectionState> elections = new HashMap<>();
-    private Map<Long, RaftSession> listeners = new LinkedHashMap<>();
-
-    @Override
-    public void snapshot(SnapshotWriter writer) {
-        writer.writeObject(Sets.newHashSet(listeners.keySet()), SERIALIZER::encode);
-        writer.writeObject(termCounters, SERIALIZER::encode);
-        writer.writeObject(elections, SERIALIZER::encode);
-        logger().debug("Took state machine snapshot");
-    }
-
-    @Override
-    public void install(SnapshotReader reader) {
-        listeners = new LinkedHashMap<>();
-        for (Long sessionId : reader.<Set<Long>>readObject(SERIALIZER::decode)) {
-            listeners.put(sessionId, sessions().getSession(sessionId));
-        }
-        termCounters = reader.readObject(SERIALIZER::decode);
-        elections = reader.readObject(SERIALIZER::decode);
-        logger().debug("Reinstated state machine from snapshot");
-    }
-
-    @Override
-    protected void configure(RaftServiceExecutor executor) {
-        // Notification
-        executor.register(ADD_LISTENER, this::listen);
-        executor.register(REMOVE_LISTENER, this::unlisten);
-        // Commands
-        executor.register(RUN, SERIALIZER::decode, this::run, SERIALIZER::encode);
-        executor.register(WITHDRAW, SERIALIZER::decode, this::withdraw);
-        executor.register(ANOINT, SERIALIZER::decode, this::anoint, SERIALIZER::encode);
-        executor.register(PROMOTE, SERIALIZER::decode, this::promote, SERIALIZER::encode);
-        executor.register(EVICT, SERIALIZER::decode, this::evict);
-        // Queries
-        executor.register(GET_LEADERSHIP, SERIALIZER::decode, this::getLeadership, SERIALIZER::encode);
-        executor.register(GET_ALL_LEADERSHIPS, this::allLeaderships, SERIALIZER::encode);
-        executor.register(GET_ELECTED_TOPICS, SERIALIZER::decode, this::electedTopics, SERIALIZER::encode);
-    }
-
-    private void notifyLeadershipChange(Leadership previousLeadership, Leadership newLeadership) {
-        notifyLeadershipChanges(Lists.newArrayList(new Change<>(previousLeadership, newLeadership)));
-    }
-
-    private void notifyLeadershipChanges(List<Change<Leadership>> changes) {
-        if (changes.isEmpty()) {
-            return;
-        }
-        listeners.values().forEach(session -> session.publish(CHANGE, SERIALIZER::encode, changes));
-    }
-
-    /**
-     * Applies listen commits.
-     *
-     * @param commit listen commit
-     */
-    public void listen(Commit<Void> commit) {
-        listeners.put(commit.session().sessionId().id(), commit.session());
-    }
-
-    /**
-     * Applies unlisten commits.
-     *
-     * @param commit unlisten commit
-     */
-    public void unlisten(Commit<Void> commit) {
-        listeners.remove(commit.session().sessionId().id());
-    }
-
-    /**
-     * Applies an {@link AtomixLeaderElectorOperations.Run} commit.
-     * @param commit commit entry
-     * @return topic leader. If no previous leader existed this is the node that just entered the race.
-     */
-    public Leadership run(Commit<? extends Run> commit) {
-        try {
-            String topic = commit.value().topic();
-            Leadership oldLeadership = leadership(topic);
-            Registration registration = new Registration(commit.value().nodeId(), commit.session().sessionId().id());
-            elections.compute(topic, (k, v) -> {
-                if (v == null) {
-                    return new ElectionState(registration, termCounter(topic)::incrementAndGet);
-                } else {
-                    if (!v.isDuplicate(registration)) {
-                        return new ElectionState(v).addRegistration(registration, termCounter(topic)::incrementAndGet);
-                    } else {
-                        return v;
-                    }
-                }
-            });
-            Leadership newLeadership = leadership(topic);
-
-            if (!Objects.equal(oldLeadership, newLeadership)) {
-                notifyLeadershipChange(oldLeadership, newLeadership);
-            }
-            return newLeadership;
-        } catch (Exception e) {
-            logger().error("State machine operation failed", e);
-            throw new IllegalStateException(e);
-        }
-    }
-
-    /**
-     * Applies an {@link AtomixLeaderElectorOperations.Withdraw} commit.
-     * @param commit withdraw commit
-     */
-    public void withdraw(Commit<? extends Withdraw> commit) {
-        try {
-            String topic = commit.value().topic();
-            Leadership oldLeadership = leadership(topic);
-            elections.computeIfPresent(topic, (k, v) -> v.cleanup(commit.session(),
-                    termCounter(topic)::incrementAndGet));
-            Leadership newLeadership = leadership(topic);
-            if (!Objects.equal(oldLeadership, newLeadership)) {
-                notifyLeadershipChange(oldLeadership, newLeadership);
-            }
-        } catch (Exception e) {
-            logger().error("State machine operation failed", e);
-            throw new IllegalStateException(e);
-        }
-    }
-
-    /**
-     * Applies an {@link AtomixLeaderElectorOperations.Anoint} commit.
-     * @param commit anoint commit
-     * @return {@code true} if changes were made and the transfer occurred; {@code false} if it did not.
-     */
-    public boolean anoint(Commit<? extends Anoint> commit) {
-        try {
-            String topic = commit.value().topic();
-            NodeId nodeId = commit.value().nodeId();
-            Leadership oldLeadership = leadership(topic);
-            ElectionState electionState = elections.computeIfPresent(topic,
-                    (k, v) -> v.transferLeadership(nodeId, termCounter(topic)));
-            Leadership newLeadership = leadership(topic);
-            if (!Objects.equal(oldLeadership, newLeadership)) {
-                notifyLeadershipChange(oldLeadership, newLeadership);
-            }
-            return (electionState != null &&
-                    electionState.leader() != null &&
-                    commit.value().nodeId().equals(electionState.leader().nodeId()));
-        } catch (Exception e) {
-            logger().error("State machine operation failed", e);
-            throw new IllegalStateException(e);
-        }
-    }
-
-    /**
-     * Applies an {@link AtomixLeaderElectorOperations.Promote} commit.
-     * @param commit promote commit
-     * @return {@code true} if changes desired end state is achieved.
-     */
-    public boolean promote(Commit<? extends Promote> commit) {
-        try {
-            String topic = commit.value().topic();
-            NodeId nodeId = commit.value().nodeId();
-            Leadership oldLeadership = leadership(topic);
-            if (oldLeadership == null || !oldLeadership.candidates().contains(nodeId)) {
-                return false;
-            }
-            elections.computeIfPresent(topic, (k, v) -> v.promote(nodeId));
-            Leadership newLeadership = leadership(topic);
-            if (!Objects.equal(oldLeadership, newLeadership)) {
-                notifyLeadershipChange(oldLeadership, newLeadership);
-            }
-            return true;
-        } catch (Exception e) {
-            logger().error("State machine operation failed", e);
-            throw new IllegalStateException(e);
-        }
-    }
-
-    /**
-     * Applies an {@link AtomixLeaderElectorOperations.Evict} commit.
-     * @param commit evict commit
-     */
-    public void evict(Commit<? extends Evict> commit) {
-        try {
-            List<Change<Leadership>> changes = Lists.newArrayList();
-            NodeId nodeId = commit.value().nodeId();
-            Set<String> topics = Maps.filterValues(elections, e -> e.candidates().contains(nodeId)).keySet();
-            topics.forEach(topic -> {
-                Leadership oldLeadership = leadership(topic);
-                elections.compute(topic, (k, v) -> v.evict(nodeId, termCounter(topic)::incrementAndGet));
-                Leadership newLeadership = leadership(topic);
-                if (!Objects.equal(oldLeadership, newLeadership)) {
-                    changes.add(new Change<>(oldLeadership, newLeadership));
-                }
-            });
-            notifyLeadershipChanges(changes);
-        } catch (Exception e) {
-            logger().error("State machine operation failed", e);
-            throw new IllegalStateException(e);
-        }
-    }
-
-    /**
-     * Applies an {@link AtomixLeaderElectorOperations.GetLeadership} commit.
-     * @param commit GetLeadership commit
-     * @return leader
-     */
-    public Leadership getLeadership(Commit<? extends GetLeadership> commit) {
-        String topic = commit.value().topic();
-        try {
-            return leadership(topic);
-        } catch (Exception e) {
-            logger().error("State machine operation failed", e);
-            throw new IllegalStateException(e);
-        }
-    }
-
-    /**
-     * Applies an {@link AtomixLeaderElectorOperations.GetElectedTopics} commit.
-     * @param commit commit entry
-     * @return set of topics for which the node is the leader
-     */
-    public Set<String> electedTopics(Commit<? extends GetElectedTopics> commit) {
-        try {
-            NodeId nodeId = commit.value().nodeId();
-            return ImmutableSet.copyOf(Maps.filterEntries(elections, e -> {
-                Leader leader = leadership(e.getKey()).leader();
-                return leader != null && leader.nodeId().equals(nodeId);
-            }).keySet());
-        } catch (Exception e) {
-            logger().error("State machine operation failed", e);
-            throw new IllegalStateException(e);
-        }
-    }
-
-    /**
-     * Applies an {@link AtomixLeaderElectorOperations#GET_ALL_LEADERSHIPS} commit.
-     * @param commit GetAllLeaderships commit
-     * @return topic to leader mapping
-     */
-    public Map<String, Leadership> allLeaderships(Commit<Void> commit) {
-        Map<String, Leadership> result = new HashMap<>();
-        try {
-            result.putAll(Maps.transformEntries(elections, (k, v) -> leadership(k)));
-            return result;
-        } catch (Exception e) {
-            logger().error("State machine operation failed", e);
-            throw new IllegalStateException(e);
-        }
-    }
-
-    private Leadership leadership(String topic) {
-        return new Leadership(topic,
-                leader(topic),
-                candidates(topic));
-    }
-
-    private Leader leader(String topic) {
-        ElectionState electionState = elections.get(topic);
-        return electionState == null ? null : electionState.leader();
-    }
-
-    private List<NodeId> candidates(String topic) {
-        ElectionState electionState = elections.get(topic);
-        return electionState == null ? new LinkedList<>() : electionState.candidates();
-    }
-
-    private void onSessionEnd(RaftSession session) {
-        listeners.remove(session.sessionId().id());
-        Set<String> topics = ImmutableSet.copyOf(elections.keySet());
-        List<Change<Leadership>> changes = Lists.newArrayList();
-        for (String topic: topics) {
-            Leadership oldLeadership = leadership(topic);
-            elections.compute(topic, (k, v) -> v.cleanup(session, termCounter(topic)::incrementAndGet));
-            Leadership newLeadership = leadership(topic);
-            if (!Objects.equal(oldLeadership, newLeadership)) {
-                changes.add(new Change<>(oldLeadership, newLeadership));
-            }
-        }
-        notifyLeadershipChanges(changes);
-    }
-
-    private static class Registration {
-        private final NodeId nodeId;
-        private final long sessionId;
-
-        public Registration(NodeId nodeId, long sessionId) {
-            this.nodeId = nodeId;
-            this.sessionId = sessionId;
-        }
-
-        public NodeId nodeId() {
-            return nodeId;
-        }
-
-        public long sessionId() {
-            return sessionId;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("nodeId", nodeId)
-                    .add("sessionId", sessionId)
-                    .toString();
-        }
-    }
-
-    private static class ElectionState {
-        final Registration leader;
-        final long term;
-        final long termStartTime;
-        final List<Registration> registrations;
-
-        public ElectionState(Registration registration, Supplier<Long> termCounter) {
-            registrations = Arrays.asList(registration);
-            term = termCounter.get();
-            termStartTime = System.currentTimeMillis();
-            leader = registration;
-        }
-
-        public ElectionState(ElectionState other) {
-            registrations = Lists.newArrayList(other.registrations);
-            leader = other.leader;
-            term = other.term;
-            termStartTime = other.termStartTime;
-        }
-
-        public ElectionState(List<Registration> registrations,
-                Registration leader,
-                long term,
-                long termStartTime) {
-            this.registrations = Lists.newArrayList(registrations);
-            this.leader = leader;
-            this.term = term;
-            this.termStartTime = termStartTime;
-        }
-
-        public ElectionState cleanup(RaftSession session, Supplier<Long> termCounter) {
-            Optional<Registration> registration =
-                    registrations.stream().filter(r -> r.sessionId() == session.sessionId().id()).findFirst();
-            if (registration.isPresent()) {
-                List<Registration> updatedRegistrations =
-                        registrations.stream()
-                                .filter(r -> r.sessionId() != session.sessionId().id())
-                                .collect(Collectors.toList());
-                if (leader.sessionId() == session.sessionId().id()) {
-                    if (!updatedRegistrations.isEmpty()) {
-                        return new ElectionState(updatedRegistrations,
-                                updatedRegistrations.get(0),
-                                termCounter.get(),
-                                System.currentTimeMillis());
-                    } else {
-                        return new ElectionState(updatedRegistrations, null, term, termStartTime);
-                    }
-                } else {
-                    return new ElectionState(updatedRegistrations, leader, term, termStartTime);
-                }
-            } else {
-                return this;
-            }
-        }
-
-        public ElectionState evict(NodeId nodeId, Supplier<Long> termCounter) {
-            Optional<Registration> registration =
-                    registrations.stream().filter(r -> r.nodeId.equals(nodeId)).findFirst();
-            if (registration.isPresent()) {
-                List<Registration> updatedRegistrations =
-                        registrations.stream()
-                                .filter(r -> !r.nodeId().equals(nodeId))
-                                .collect(Collectors.toList());
-                if (leader.nodeId().equals(nodeId)) {
-                    if (!updatedRegistrations.isEmpty()) {
-                        return new ElectionState(updatedRegistrations,
-                                updatedRegistrations.get(0),
-                                termCounter.get(),
-                                System.currentTimeMillis());
-                    } else {
-                        return new ElectionState(updatedRegistrations, null, term, termStartTime);
-                    }
-                } else {
-                    return new ElectionState(updatedRegistrations, leader, term, termStartTime);
-                }
-            } else {
-                return this;
-            }
-        }
-
-        public boolean isDuplicate(Registration registration) {
-            return registrations.stream().anyMatch(r -> r.sessionId() == registration.sessionId());
-        }
-
-        public Leader leader() {
-            if (leader == null) {
-                return null;
-            } else {
-                NodeId leaderNodeId = leader.nodeId();
-                return new Leader(leaderNodeId, term, termStartTime);
-            }
-        }
-
-        public List<NodeId> candidates() {
-            return registrations.stream().map(registration -> registration.nodeId()).collect(Collectors.toList());
-        }
-
-        public ElectionState addRegistration(Registration registration, Supplier<Long> termCounter) {
-            if (!registrations.stream().anyMatch(r -> r.sessionId() == registration.sessionId())) {
-                List<Registration> updatedRegistrations = new LinkedList<>(registrations);
-                updatedRegistrations.add(registration);
-                boolean newLeader = leader == null;
-                return new ElectionState(updatedRegistrations,
-                        newLeader ? registration : leader,
-                        newLeader ? termCounter.get() : term,
-                        newLeader ? System.currentTimeMillis() : termStartTime);
-            }
-            return this;
-        }
-
-        public ElectionState transferLeadership(NodeId nodeId, AtomicLong termCounter) {
-            Registration newLeader = registrations.stream()
-                    .filter(r -> r.nodeId().equals(nodeId))
-                    .findFirst()
-                    .orElse(null);
-            if (newLeader != null) {
-                return new ElectionState(registrations,
-                        newLeader,
-                        termCounter.incrementAndGet(),
-                        System.currentTimeMillis());
-            } else {
-                return this;
-            }
-        }
-
-        public ElectionState promote(NodeId nodeId) {
-            Registration registration = registrations.stream()
-                    .filter(r -> r.nodeId().equals(nodeId))
-                    .findFirst()
-                    .orElse(null);
-            List<Registration> updatedRegistrations = Lists.newArrayList();
-            updatedRegistrations.add(registration);
-            registrations.stream()
-                    .filter(r -> !r.nodeId().equals(nodeId))
-                    .forEach(updatedRegistrations::add);
-            return new ElectionState(updatedRegistrations,
-                    leader,
-                    term,
-                    termStartTime);
-
-        }
-    }
-
-    @Override
-    public void onExpire(RaftSession session) {
-        onSessionEnd(session);
-    }
-
-    @Override
-    public void onClose(RaftSession session) {
-        onSessionEnd(session);
-    }
-
-    private AtomicLong termCounter(String topic) {
-        return termCounters.computeIfAbsent(topic, k -> new AtomicLong(0));
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixValue.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixValue.java
deleted file mode 100644
index a4cdd52..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixValue.java
+++ /dev/null
@@ -1,109 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.Objects;
-import java.util.Set;
-import java.util.concurrent.CompletableFuture;
-
-import org.onlab.util.Tools;
-import org.onosproject.store.service.AsyncAtomicValue;
-import org.onosproject.store.service.AtomicValueEvent;
-import org.onosproject.store.service.AtomicValueEventListener;
-import org.onosproject.store.service.MapEventListener;
-import org.onosproject.store.service.Versioned;
-
-import com.google.common.collect.Sets;
-
-/**
- * Implementation of {@link AsyncAtomicValue} backed by {@link AtomixConsistentMap}.
- */
-public class AtomixValue implements AsyncAtomicValue<String> {
-
-    private final String name;
-    private final AtomixConsistentMap atomixMap;
-    private MapEventListener<String, byte[]> mapEventListener;
-    private final Set<AtomicValueEventListener<String>> listeners = Sets.newIdentityHashSet();
-
-    AtomixValue(String name, AtomixConsistentMap atomixMap) {
-        this.name = name;
-        this.atomixMap = atomixMap;
-    }
-
-    @Override
-    public CompletableFuture<Boolean> compareAndSet(String expect, String update) {
-        return atomixMap.replace(name, Tools.getBytesUtf8(expect), Tools.getBytesUtf8(update));
-    }
-
-    @Override
-    public CompletableFuture<String> get() {
-        return atomixMap.get(name)
-                        .thenApply(v -> v != null ? Tools.toStringUtf8(v.value()) : null);
-    }
-
-    @Override
-    public CompletableFuture<String> getAndSet(String value) {
-        return atomixMap.put(name, Tools.getBytesUtf8(value))
-                        .thenApply(v -> v != null ? Tools.toStringUtf8(v.value()) : null);
-    }
-
-    @Override
-    public CompletableFuture<Void> set(String value) {
-        return getAndSet(value).thenApply(v -> null);
-    }
-
-    @Override
-    public CompletableFuture<Void> addListener(AtomicValueEventListener<String> listener) {
-        // TODO: synchronization
-        if (mapEventListener == null) {
-            mapEventListener = event -> {
-                Versioned<byte[]> newValue = event.newValue();
-                Versioned<byte[]> oldValue = event.oldValue();
-                if (Objects.equals(event.key(), name)) {
-                    listener.event(new AtomicValueEvent<>(name,
-                            newValue == null ? null : Tools.toStringUtf8(newValue.value()),
-                            oldValue == null ? null : Tools.toStringUtf8(oldValue.value())));
-                }
-            };
-            return atomixMap.addListener(mapEventListener).whenComplete((r, e) -> {
-                if (e == null) {
-                    listeners.add(listener);
-                } else {
-                    mapEventListener = null;
-                }
-            });
-        } else {
-            listeners.add(listener);
-            return CompletableFuture.completedFuture(null);
-        }
-    }
-
-    @Override
-    public CompletableFuture<Void> removeListener(AtomicValueEventListener<String> listener) {
-        // TODO: synchronization
-        listeners.remove(listener);
-        if (listeners.isEmpty()) {
-            return atomixMap.removeListener(mapEventListener);
-        } else {
-            return CompletableFuture.completedFuture(null);
-        }
-    }
-
-    @Override
-    public String name() {
-        return null;
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueue.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueue.java
deleted file mode 100644
index 73bc802..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueue.java
+++ /dev/null
@@ -1,207 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.Collection;
-import java.util.List;
-import java.util.Timer;
-import java.util.concurrent.CompletableFuture;
-import java.util.concurrent.Executor;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.function.Consumer;
-
-import com.google.common.collect.ImmutableList;
-import io.atomix.protocols.raft.proxy.RaftProxy;
-import org.onlab.util.AbstractAccumulator;
-import org.onlab.util.Accumulator;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.Add;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.Complete;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.Take;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.Task;
-import org.onosproject.store.service.WorkQueue;
-import org.onosproject.store.service.WorkQueueStats;
-import org.slf4j.Logger;
-
-import static java.util.concurrent.Executors.newSingleThreadExecutor;
-import static org.onlab.util.Tools.groupedThreads;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueEvents.TASK_AVAILABLE;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.ADD;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.CLEAR;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.COMPLETE;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.REGISTER;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.STATS;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.TAKE;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.UNREGISTER;
-import static org.slf4j.LoggerFactory.getLogger;
-
-/**
- * Distributed resource providing the {@link WorkQueue} primitive.
- */
-public class AtomixWorkQueue extends AbstractRaftPrimitive implements WorkQueue<byte[]> {
-    private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
-            .register(KryoNamespaces.BASIC)
-            .register(AtomixWorkQueueOperations.NAMESPACE)
-            .register(AtomixWorkQueueEvents.NAMESPACE)
-            .build());
-
-    private final Logger log = getLogger(getClass());
-    private final ExecutorService executor = newSingleThreadExecutor(groupedThreads("AtomixWorkQueue", "%d", log));
-    private final AtomicReference<TaskProcessor> taskProcessor = new AtomicReference<>();
-    private final Timer timer = new Timer("atomix-work-queue-completer");
-    private final AtomicBoolean isRegistered = new AtomicBoolean(false);
-
-    public AtomixWorkQueue(RaftProxy proxy) {
-        super(proxy);
-        proxy.addStateChangeListener(state -> {
-            if (state == RaftProxy.State.CONNECTED && isRegistered.get()) {
-                proxy.invoke(REGISTER);
-            }
-        });
-        proxy.addEventListener(TASK_AVAILABLE, this::resumeWork);
-    }
-
-    @Override
-    public CompletableFuture<Void> destroy() {
-        executor.shutdown();
-        timer.cancel();
-        return proxy.invoke(CLEAR);
-    }
-
-    @Override
-    public CompletableFuture<Void> addMultiple(Collection<byte[]> items) {
-        if (items.isEmpty()) {
-            return CompletableFuture.completedFuture(null);
-        }
-        return proxy.invoke(ADD, SERIALIZER::encode, new Add(items));
-    }
-
-    @Override
-    public CompletableFuture<Collection<Task<byte[]>>> take(int maxTasks) {
-        if (maxTasks <= 0) {
-            return CompletableFuture.completedFuture(ImmutableList.of());
-        }
-        return proxy.invoke(TAKE, SERIALIZER::encode, new Take(maxTasks), SERIALIZER::decode);
-    }
-
-    @Override
-    public CompletableFuture<Void> complete(Collection<String> taskIds) {
-        if (taskIds.isEmpty()) {
-            return CompletableFuture.completedFuture(null);
-        }
-        return proxy.invoke(COMPLETE, SERIALIZER::encode, new Complete(taskIds));
-    }
-
-    @Override
-    public CompletableFuture<Void> registerTaskProcessor(Consumer<byte[]> callback,
-            int parallelism,
-            Executor executor) {
-        Accumulator<String> completedTaskAccumulator =
-                new CompletedTaskAccumulator(timer, 50, 50); // TODO: make configurable
-        taskProcessor.set(new TaskProcessor(callback,
-                parallelism,
-                executor,
-                completedTaskAccumulator));
-        return register().thenCompose(v -> take(parallelism))
-                .thenAccept(taskProcessor.get());
-    }
-
-    @Override
-    public CompletableFuture<Void> stopProcessing() {
-        return unregister();
-    }
-
-    @Override
-    public CompletableFuture<WorkQueueStats> stats() {
-        return proxy.invoke(STATS, SERIALIZER::decode);
-    }
-
-    private void resumeWork() {
-        TaskProcessor activeProcessor = taskProcessor.get();
-        if (activeProcessor == null) {
-            return;
-        }
-        this.take(activeProcessor.headRoom())
-                .whenCompleteAsync((tasks, e) -> activeProcessor.accept(tasks), executor);
-    }
-
-    private CompletableFuture<Void> register() {
-        return proxy.invoke(REGISTER).thenRun(() -> isRegistered.set(true));
-    }
-
-    private CompletableFuture<Void> unregister() {
-        return proxy.invoke(UNREGISTER).thenRun(() -> isRegistered.set(false));
-    }
-
-    // TaskId accumulator for paced triggering of task completion calls.
-    private class CompletedTaskAccumulator extends AbstractAccumulator<String> {
-        CompletedTaskAccumulator(Timer timer, int maxTasksToBatch, int maxBatchMillis) {
-            super(timer, maxTasksToBatch, maxBatchMillis, Integer.MAX_VALUE);
-        }
-
-        @Override
-        public void processItems(List<String> items) {
-            complete(items);
-        }
-    }
-
-    private class TaskProcessor implements Consumer<Collection<Task<byte[]>>> {
-
-        private final AtomicInteger headRoom;
-        private final Consumer<byte[]> backingConsumer;
-        private final Executor executor;
-        private final Accumulator<String> taskCompleter;
-
-        public TaskProcessor(Consumer<byte[]> backingConsumer,
-                int parallelism,
-                Executor executor,
-                Accumulator<String> taskCompleter) {
-            this.backingConsumer = backingConsumer;
-            this.headRoom = new AtomicInteger(parallelism);
-            this.executor = executor;
-            this.taskCompleter = taskCompleter;
-        }
-
-        public int headRoom() {
-            return headRoom.get();
-        }
-
-        @Override
-        public void accept(Collection<Task<byte[]>> tasks) {
-            if (tasks == null) {
-                return;
-            }
-            headRoom.addAndGet(-1 * tasks.size());
-            tasks.forEach(task ->
-                    executor.execute(() -> {
-                        try {
-                            backingConsumer.accept(task.payload());
-                            taskCompleter.add(task.taskId());
-                        } catch (Exception e) {
-                            log.debug("Task execution failed", e);
-                        } finally {
-                            headRoom.incrementAndGet();
-                            resumeWork();
-                        }
-                    }));
-        }
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueEvents.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueEvents.java
deleted file mode 100644
index 5458300..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueEvents.java
+++ /dev/null
@@ -1,36 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import io.atomix.protocols.raft.event.EventType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-
-/**
- * Atomix work queue events.
- */
-public enum AtomixWorkQueueEvents implements EventType {
-    TASK_AVAILABLE;
-
-    @Override
-    public String id() {
-        return name();
-    }
-
-    public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
-            .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID + 50)
-            .build("AtomixWorkQueueEvents");
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueOperations.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueOperations.java
deleted file mode 100644
index 2452cf8..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueOperations.java
+++ /dev/null
@@ -1,146 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.Collection;
-
-import com.google.common.base.MoreObjects;
-import io.atomix.protocols.raft.operation.OperationId;
-import io.atomix.protocols.raft.operation.OperationType;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.Task;
-import org.onosproject.store.service.WorkQueueStats;
-
-/**
- * {@link AtomixWorkQueue} resource state machine operations.
- */
-public enum AtomixWorkQueueOperations implements OperationId {
-    STATS(OperationType.QUERY),
-    REGISTER(OperationType.COMMAND),
-    UNREGISTER(OperationType.COMMAND),
-    ADD(OperationType.COMMAND),
-    TAKE(OperationType.COMMAND),
-    COMPLETE(OperationType.COMMAND),
-    CLEAR(OperationType.COMMAND);
-
-    private final OperationType type;
-
-    AtomixWorkQueueOperations(OperationType type) {
-        this.type = type;
-    }
-
-    @Override
-    public String id() {
-        return name();
-    }
-
-    @Override
-    public OperationType type() {
-        return type;
-    }
-
-    public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
-            .register(KryoNamespaces.BASIC)
-            .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
-            .register(Add.class)
-            .register(Take.class)
-            .register(Complete.class)
-            .register(Task.class)
-            .register(WorkQueueStats.class)
-            .build("AtomixWorkQueueOperations");
-
-    /**
-     * Work queue operation.
-     */
-    public abstract static class WorkQueueOperation {
-    }
-
-    /**
-     * Command to add a collection of tasks to the queue.
-     */
-    @SuppressWarnings("serial")
-    public static class Add extends WorkQueueOperation {
-        private Collection<byte[]> items;
-
-        private Add() {
-        }
-
-        public Add(Collection<byte[]> items) {
-            this.items = items;
-        }
-
-        public Collection<byte[]> items() {
-            return items;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("items", items)
-                    .toString();
-        }
-    }
-
-    /**
-     * Command to take a task from the queue.
-     */
-    @SuppressWarnings("serial")
-    public static class Take extends WorkQueueOperation {
-        private int maxTasks;
-
-        private Take() {
-        }
-
-        public Take(int maxTasks) {
-            this.maxTasks = maxTasks;
-        }
-
-        public int maxTasks() {
-            return maxTasks;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("maxTasks", maxTasks)
-                    .toString();
-        }
-    }
-
-    @SuppressWarnings("serial")
-    public static class Complete extends WorkQueueOperation {
-        private Collection<String> taskIds;
-
-        private Complete() {
-        }
-
-        public Complete(Collection<String> taskIds) {
-            this.taskIds = taskIds;
-        }
-
-        public Collection<String> taskIds() {
-            return taskIds;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("taskIds", taskIds)
-                    .toString();
-        }
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueService.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueService.java
deleted file mode 100644
index 4540aa5..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueService.java
+++ /dev/null
@@ -1,242 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import java.util.ArrayDeque;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Queue;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Queues;
-import com.google.common.collect.Sets;
-import io.atomix.protocols.raft.service.AbstractRaftService;
-import io.atomix.protocols.raft.service.Commit;
-import io.atomix.protocols.raft.service.RaftServiceExecutor;
-import io.atomix.protocols.raft.session.RaftSession;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotReader;
-import io.atomix.protocols.raft.storage.snapshot.SnapshotWriter;
-import org.onlab.util.KryoNamespace;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.Add;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.Complete;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.Take;
-import org.onosproject.store.serializers.KryoNamespaces;
-import org.onosproject.store.service.Serializer;
-import org.onosproject.store.service.Task;
-import org.onosproject.store.service.WorkQueueStats;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueEvents.TASK_AVAILABLE;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.ADD;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.CLEAR;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.COMPLETE;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.REGISTER;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.STATS;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.TAKE;
-import static org.onosproject.store.primitives.resources.impl.AtomixWorkQueueOperations.UNREGISTER;
-
-/**
- * State machine for {@link AtomixWorkQueue} resource.
- */
-public class AtomixWorkQueueService extends AbstractRaftService {
-
-    private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
-            .register(KryoNamespaces.BASIC)
-            .register(AtomixWorkQueueOperations.NAMESPACE)
-            .register(AtomixWorkQueueEvents.NAMESPACE)
-            .register(TaskAssignment.class)
-            .register(new HashMap().keySet().getClass())
-            .register(ArrayDeque.class)
-            .build());
-
-    private final AtomicLong totalCompleted = new AtomicLong(0);
-
-    private Queue<Task<byte[]>> unassignedTasks = Queues.newArrayDeque();
-    private Map<String, TaskAssignment> assignments = Maps.newHashMap();
-    private Map<Long, RaftSession> registeredWorkers = Maps.newHashMap();
-
-    @Override
-    public void snapshot(SnapshotWriter writer) {
-        writer.writeObject(Sets.newHashSet(registeredWorkers.keySet()), SERIALIZER::encode);
-        writer.writeObject(assignments, SERIALIZER::encode);
-        writer.writeObject(unassignedTasks, SERIALIZER::encode);
-        writer.writeLong(totalCompleted.get());
-    }
-
-    @Override
-    public void install(SnapshotReader reader) {
-        registeredWorkers = Maps.newHashMap();
-        for (Long sessionId : reader.<Set<Long>>readObject(SERIALIZER::decode)) {
-            registeredWorkers.put(sessionId, sessions().getSession(sessionId));
-        }
-        assignments = reader.readObject(SERIALIZER::decode);
-        unassignedTasks = reader.readObject(SERIALIZER::decode);
-        totalCompleted.set(reader.readLong());
-    }
-
-    @Override
-    protected void configure(RaftServiceExecutor executor) {
-        executor.register(STATS, this::stats, SERIALIZER::encode);
-        executor.register(REGISTER, this::register);
-        executor.register(UNREGISTER, this::unregister);
-        executor.register(ADD, SERIALIZER::decode, this::add);
-        executor.register(TAKE, SERIALIZER::decode, this::take, SERIALIZER::encode);
-        executor.register(COMPLETE, SERIALIZER::decode, this::complete);
-        executor.register(CLEAR, this::clear);
-    }
-
-    protected WorkQueueStats stats(Commit<Void> commit) {
-        return WorkQueueStats.builder()
-                .withTotalCompleted(totalCompleted.get())
-                .withTotalPending(unassignedTasks.size())
-                .withTotalInProgress(assignments.size())
-                .build();
-    }
-
-    protected void clear(Commit<Void> commit) {
-        unassignedTasks.clear();
-        assignments.clear();
-        registeredWorkers.clear();
-        totalCompleted.set(0);
-    }
-
-    protected void register(Commit<Void> commit) {
-        registeredWorkers.put(commit.session().sessionId().id(), commit.session());
-    }
-
-    protected void unregister(Commit<Void> commit) {
-        registeredWorkers.remove(commit.session().sessionId().id());
-    }
-
-    protected void add(Commit<? extends Add> commit) {
-        Collection<byte[]> items = commit.value().items();
-
-        AtomicInteger itemIndex = new AtomicInteger(0);
-        items.forEach(item -> {
-            String taskId = String.format("%d:%d:%d", commit.session().sessionId().id(),
-                    commit.index(),
-                    itemIndex.getAndIncrement());
-            unassignedTasks.add(new Task<>(taskId, item));
-        });
-
-        // Send an event to all sessions that have expressed interest in task processing
-        // and are not actively processing a task.
-        registeredWorkers.values().forEach(session -> session.publish(TASK_AVAILABLE));
-        // FIXME: This generates a lot of event traffic.
-    }
-
-    protected Collection<Task<byte[]>> take(Commit<? extends Take> commit) {
-        try {
-            if (unassignedTasks.isEmpty()) {
-                return ImmutableList.of();
-            }
-            long sessionId = commit.session().sessionId().id();
-            int maxTasks = commit.value().maxTasks();
-            return IntStream.range(0, Math.min(maxTasks, unassignedTasks.size()))
-                    .mapToObj(i -> {
-                        Task<byte[]> task = unassignedTasks.poll();
-                        String taskId = task.taskId();
-                        TaskAssignment assignment = new TaskAssignment(sessionId, task);
-
-                        // bookkeeping
-                        assignments.put(taskId, assignment);
-
-                        return task;
-                    })
-                    .collect(Collectors.toCollection(ArrayList::new));
-        } catch (Exception e) {
-            logger().warn("State machine update failed", e);
-            throw new IllegalStateException(e);
-        }
-    }
-
-    protected void complete(Commit<? extends Complete> commit) {
-        long sessionId = commit.session().sessionId().id();
-        try {
-            commit.value().taskIds().forEach(taskId -> {
-                TaskAssignment assignment = assignments.get(taskId);
-                if (assignment != null && assignment.sessionId() == sessionId) {
-                    assignments.remove(taskId);
-                    // bookkeeping
-                    totalCompleted.incrementAndGet();
-                }
-            });
-        } catch (Exception e) {
-            logger().warn("State machine update failed", e);
-            throw new IllegalStateException(e);
-        }
-    }
-
-    @Override
-    public void onExpire(RaftSession session) {
-        evictWorker(session.sessionId().id());
-    }
-
-    @Override
-    public void onClose(RaftSession session) {
-        evictWorker(session.sessionId().id());
-    }
-
-    private void evictWorker(long sessionId) {
-        registeredWorkers.remove(sessionId);
-
-        // TODO: Maintain an index of tasks by session for efficient access.
-        Iterator<Map.Entry<String, TaskAssignment>> iter = assignments.entrySet().iterator();
-        while (iter.hasNext()) {
-            Map.Entry<String, TaskAssignment> entry = iter.next();
-            TaskAssignment assignment = entry.getValue();
-            if (assignment.sessionId() == sessionId) {
-                unassignedTasks.add(assignment.task());
-                iter.remove();
-            }
-        }
-    }
-
-    private static class TaskAssignment {
-        private final long sessionId;
-        private final Task<byte[]> task;
-
-        public TaskAssignment(long sessionId, Task<byte[]> task) {
-            this.sessionId = sessionId;
-            this.task = task;
-        }
-
-        public long sessionId() {
-            return sessionId;
-        }
-
-        public Task<byte[]> task() {
-            return task;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("sessionId", sessionId)
-                    .add("task", task)
-                    .toString();
-        }
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/CommitResult.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/CommitResult.java
deleted file mode 100644
index 5b1da77..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/CommitResult.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-/**
- * Response enum for two phase commit operation.
- */
-public enum CommitResult {
-    /**
-     * Signifies a successful commit execution.
-     */
-    OK,
-
-    /**
-     * Signifies a failure due to unrecognized transaction identifier.
-     */
-    UNKNOWN_TRANSACTION_ID,
-
-    /**
-     * Signifies a failure to get participants to agree to commit (during prepare stage).
-     */
-    FAILURE_TO_PREPARE,
-
-    /**
-     * Failure during commit phase.
-     */
-    FAILURE_DURING_COMMIT
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/DefaultDocumentTree.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/DefaultDocumentTree.java
deleted file mode 100644
index 2d5832c..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/DefaultDocumentTree.java
+++ /dev/null
@@ -1,213 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Objects;
-import java.util.Optional;
-import java.util.concurrent.atomic.AtomicLong;
-
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTree;
-import org.onosproject.store.service.DocumentTreeListener;
-import org.onosproject.store.service.DocumentTreeNode;
-import org.onosproject.store.service.IllegalDocumentModificationException;
-import org.onosproject.store.service.NoSuchDocumentPathException;
-import org.onosproject.store.service.Ordering;
-import org.onosproject.store.service.Versioned;
-
-import com.google.common.base.Supplier;
-import com.google.common.collect.Maps;
-
-/**
- * Simple implementation of a {@link DocumentTree}.
- *
- * @param <V> tree node value type
- */
-public class DefaultDocumentTree<V> implements DocumentTree<V> {
-
-    private static final DocumentPath ROOT_PATH = DocumentPath.from("root");
-    final DefaultDocumentTreeNode<V> root;
-    private final Supplier<Long> versionSupplier;
-
-    public DefaultDocumentTree() {
-        AtomicLong versionCounter = new AtomicLong(0);
-        versionSupplier = versionCounter::incrementAndGet;
-        root = new DefaultDocumentTreeNode<>(ROOT_PATH, null, versionSupplier.get(), Ordering.NATURAL, null);
-    }
-
-    public DefaultDocumentTree(Supplier<Long> versionSupplier, Ordering ordering) {
-        root = new DefaultDocumentTreeNode<>(ROOT_PATH, null, versionSupplier.get(), ordering, null);
-        this.versionSupplier = versionSupplier;
-    }
-
-    DefaultDocumentTree(Supplier<Long> versionSupplier, DefaultDocumentTreeNode<V> root) {
-        this.root = root;
-        this.versionSupplier = versionSupplier;
-    }
-
-    @Override
-    public String name() {
-        return null;
-    }
-
-    @Override
-    public DocumentPath root() {
-        return ROOT_PATH;
-    }
-
-    @Override
-    public Map<String, Versioned<V>> getChildren(DocumentPath path) {
-        DocumentTreeNode<V> node = getNode(path);
-        if (node != null) {
-            Map<String, Versioned<V>> childrenValues = Maps.newLinkedHashMap();
-            node.children().forEachRemaining(n -> childrenValues.put(simpleName(n.path()), n.value()));
-            return childrenValues;
-        }
-        throw new NoSuchDocumentPathException();
-    }
-
-    @Override
-    public Versioned<V> get(DocumentPath path) {
-        DocumentTreeNode<V> currentNode = getNode(path);
-        return currentNode != null ? currentNode.value() : null;
-    }
-
-    @Override
-    public Versioned<V> set(DocumentPath path, V value) {
-        checkRootModification(path);
-        DefaultDocumentTreeNode<V> node = getNode(path);
-        if (node != null) {
-            return node.update(value, versionSupplier.get());
-        } else {
-            create(path, value);
-            return null;
-        }
-    }
-
-    @Override
-    public boolean create(DocumentPath path, V value) {
-        checkRootModification(path);
-        DocumentTreeNode<V> node = getNode(path);
-        if (node != null) {
-            return false;
-        }
-        DocumentPath parentPath = path.parent();
-        DefaultDocumentTreeNode<V> parentNode =  getNode(parentPath);
-        if (parentNode == null) {
-            throw new IllegalDocumentModificationException();
-        }
-        parentNode.addChild(simpleName(path), value, versionSupplier.get());
-        return true;
-    }
-
-    @Override
-    public boolean createRecursive(DocumentPath path, V value) {
-        checkRootModification(path);
-        DocumentTreeNode<V> node = getNode(path);
-        if (node != null) {
-            return false;
-        }
-        DocumentPath parentPath = path.parent();
-        if (getNode(parentPath) == null) {
-            createRecursive(parentPath, null);
-        }
-        DefaultDocumentTreeNode<V> parentNode =  getNode(parentPath);
-        if (parentNode == null) {
-            throw new IllegalDocumentModificationException();
-        }
-        parentNode.addChild(simpleName(path), value, versionSupplier.get());
-        return true;
-    }
-
-    @Override
-    public boolean replace(DocumentPath path, V newValue, long version) {
-        checkRootModification(path);
-        DocumentTreeNode<V> node = getNode(path);
-        if (node != null && node.value() != null && node.value().version() == version) {
-            set(path, newValue);
-            return true;
-        }
-        return false;
-    }
-
-    @Override
-    public boolean replace(DocumentPath path, V newValue, V expectedValue) {
-        checkRootModification(path);
-        if (Objects.equals(newValue, expectedValue)) {
-            return false;
-        }
-        DocumentTreeNode<V> node = getNode(path);
-        V prevValue = Optional.ofNullable(node)
-                    .map(DocumentTreeNode::value)
-                    .map(Versioned::valueOrNull)
-                    .orElse(null);
-        if (Objects.equals(prevValue, expectedValue)) {
-            set(path, newValue);
-            return true;
-        }
-        return false;
-    }
-
-    @Override
-    public Versioned<V> removeNode(DocumentPath path) {
-        checkRootModification(path);
-        DefaultDocumentTreeNode<V> nodeToRemove = getNode(path);
-        if (nodeToRemove == null) {
-            throw new NoSuchDocumentPathException();
-        }
-        if (nodeToRemove.hasChildren()) {
-            throw new IllegalDocumentModificationException();
-        }
-        DefaultDocumentTreeNode<V> parent = (DefaultDocumentTreeNode<V>) nodeToRemove.parent();
-        parent.removeChild(simpleName(path));
-        return nodeToRemove.value();
-    }
-
-    @Override
-    public void addListener(DocumentPath path, DocumentTreeListener<V> listener) {
-        // TODO Auto-generated method stub
-    }
-
-    @Override
-    public void removeListener(DocumentTreeListener<V> listener) {
-        // TODO Auto-generated method stub
-    }
-
-    private DefaultDocumentTreeNode<V> getNode(DocumentPath path) {
-        Iterator<String> pathElements = path.pathElements().iterator();
-        DefaultDocumentTreeNode<V> currentNode = root;
-        checkArgument("root".equals(pathElements.next()), "Path should start with root: %s", path);
-        while (pathElements.hasNext() &&  currentNode != null) {
-            currentNode = (DefaultDocumentTreeNode<V>) currentNode.child(pathElements.next());
-        }
-        return currentNode;
-    }
-
-    private String simpleName(DocumentPath path) {
-        return path.pathElements().get(path.pathElements().size() - 1);
-    }
-
-    private void checkRootModification(DocumentPath path) {
-        if (ROOT_PATH.equals(path)) {
-            throw new IllegalDocumentModificationException();
-        }
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/DefaultDocumentTreeNode.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/DefaultDocumentTreeNode.java
deleted file mode 100644
index 0f5d66a..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/DefaultDocumentTreeNode.java
+++ /dev/null
@@ -1,165 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Objects;
-
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.DocumentTreeNode;
-import org.onosproject.store.service.Ordering;
-import org.onosproject.store.service.Versioned;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-
-/**
- * A {@code DocumentTree} node.
- */
-public class DefaultDocumentTreeNode<V> implements DocumentTreeNode<V> {
-    private final DocumentPath key;
-    private Versioned<V> value;
-    private final Map<String, DocumentTreeNode<V>> children;
-    private final Ordering ordering;
-    private final DocumentTreeNode<V> parent;
-
-    public DefaultDocumentTreeNode(DocumentPath key,
-            V value,
-            long version,
-            Ordering ordering,
-            DocumentTreeNode<V> parent) {
-        this.key = checkNotNull(key);
-        this.value = new Versioned<>(value, version);
-        this.ordering = ordering;
-        this.parent = parent;
-
-        switch (ordering) {
-            case INSERTION:
-                children = Maps.newLinkedHashMap();
-                break;
-            case NATURAL:
-            default:
-                children = Maps.newTreeMap();
-                break;
-        }
-    }
-
-    @Override
-    public DocumentPath path() {
-        return key;
-    }
-
-    @Override
-    public Versioned<V> value() {
-        return value;
-    }
-
-    @Override
-    public Iterator<DocumentTreeNode<V>> children() {
-        return ImmutableList.copyOf(children.values()).iterator();
-    }
-
-    @Override
-    public DocumentTreeNode<V> child(String name) {
-        return children.get(name);
-    }
-
-
-    public DocumentTreeNode<V> parent() {
-        return parent;
-    }
-
-    /**
-     * Adds a new child only if one does not exist with the name.
-     * @param name relative path name of the child node
-     * @param newValue new value to set
-     * @param newVersion new version to set
-     * @return previous value; can be {@code null} if no child currently exists with that relative path name.
-     * a non null return value indicates child already exists and no modification occurred.
-     */
-    public Versioned<V> addChild(String name, V newValue, long newVersion) {
-        DefaultDocumentTreeNode<V> child = (DefaultDocumentTreeNode<V>) children.get(name);
-        if (child != null) {
-            return child.value();
-        }
-        children.put(name, new DefaultDocumentTreeNode<>(
-                new DocumentPath(name, path()), newValue, newVersion, ordering, this));
-        return null;
-    }
-
-    /**
-     * Updates the node value.
-     *
-     * @param newValue new value to set
-     * @param newVersion new version to set
-     * @return previous value
-     */
-    public Versioned<V> update(V newValue, long newVersion) {
-        Versioned<V> previousValue = value;
-        value = new Versioned<>(newValue, newVersion);
-        return previousValue;
-    }
-
-
-    /**
-     * Removes a child node.
-     *
-     * @param name the name of child node to be removed
-     * @return {@code true} if the child set was modified as a result of this call, {@code false} otherwise
-     */
-    public boolean removeChild(String name) {
-        return children.remove(name) != null;
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hash(this.key);
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (obj instanceof DefaultDocumentTreeNode) {
-            DefaultDocumentTreeNode<V> that = (DefaultDocumentTreeNode<V>) obj;
-            if (this.parent.equals(that.parent)) {
-                if (this.children.size() == that.children.size()) {
-                    return Sets.symmetricDifference(this.children.keySet(), that.children.keySet()).isEmpty();
-                }
-            }
-        }
-        return false;
-    }
-
-    @Override
-    public String toString() {
-        MoreObjects.ToStringHelper helper =
-                MoreObjects.toStringHelper(getClass())
-                        .add("parent", this.parent)
-                        .add("key", this.key)
-                        .add("value", this.value);
-        for (DocumentTreeNode<V> child : children.values()) {
-            helper = helper.add("child", "\n" + child.path().pathElements()
-                    .get(child.path().pathElements().size() - 1) +
-                    " : " + child.value());
-        }
-        return helper.toString();
-    }
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/DocumentTreeResult.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/DocumentTreeResult.java
deleted file mode 100644
index bf544dd..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/DocumentTreeResult.java
+++ /dev/null
@@ -1,142 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import com.google.common.base.MoreObjects;
-
-/**
- * Result of a document tree operation.
- *
- * @param <V> value type
- */
-public class DocumentTreeResult<V> {
-
-    public enum Status {
-        /**
-         * Indicates a successful update.
-         */
-        OK,
-
-        /**
-         * Indicates a noop i.e. existing and new value are both same.
-         */
-        NOOP,
-
-        /**
-         * Indicates a failed update due to a write lock.
-         */
-        WRITE_LOCK,
-
-        /**
-         * Indicates a failed update due to a invalid path.
-         */
-        INVALID_PATH,
-
-        /**
-         * Indicates a failed update due to a illegal modification attempt.
-         */
-        ILLEGAL_MODIFICATION,
-    }
-
-    @SuppressWarnings("unchecked")
-    public static final DocumentTreeResult WRITE_LOCK =
-            new DocumentTreeResult(Status.WRITE_LOCK, null);
-
-    @SuppressWarnings("unchecked")
-    public static final DocumentTreeResult INVALID_PATH =
-            new DocumentTreeResult(Status.INVALID_PATH, null);
-
-    @SuppressWarnings("unchecked")
-    public static final DocumentTreeResult ILLEGAL_MODIFICATION =
-            new DocumentTreeResult(Status.ILLEGAL_MODIFICATION, null);
-
-    /**
-     * Returns a successful result.
-     *
-     * @param result the operation result
-     * @param <V> the result value type
-     * @return successful result
-     */
-    public static <V> DocumentTreeResult<V> ok(V result) {
-        return new DocumentTreeResult<V>(Status.OK, result);
-    }
-
-    /**
-     * Returns a {@code WRITE_LOCK} error result.
-     *
-     * @param <V> the result value type
-     * @return write lock result
-     */
-    @SuppressWarnings("unchecked")
-    public static <V> DocumentTreeResult<V> writeLock() {
-        return WRITE_LOCK;
-    }
-
-    /**
-     * Returns an {@code INVALID_PATH} result.
-     *
-     * @param <V> the result value type
-     * @return invalid path result
-     */
-    @SuppressWarnings("unchecked")
-    public static <V> DocumentTreeResult<V> invalidPath() {
-        return INVALID_PATH;
-    }
-
-    /**
-     * Returns an {@code ILLEGAL_MODIFICATION} result.
-     *
-     * @param <V> the result value type
-     * @return illegal modification result
-     */
-    @SuppressWarnings("unchecked")
-    public static <V> DocumentTreeResult<V> illegalModification() {
-        return ILLEGAL_MODIFICATION;
-    }
-
-    private final Status status;
-    private final V result;
-
-    public DocumentTreeResult(Status status, V result) {
-        this.status = status;
-        this.result = result;
-    }
-
-    public Status status() {
-        return status;
-    }
-
-    public V result() {
-        return result;
-    }
-
-    public boolean updated() {
-        return status == Status.OK;
-    }
-
-    public boolean created() {
-        return updated() && result == null;
-    }
-
-    @Override
-    public String toString() {
-        return MoreObjects.toStringHelper(getClass())
-                .add("status", status)
-                .add("value", result)
-                .toString();
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/LockEvent.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/LockEvent.java
deleted file mode 100644
index cddb41f..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/LockEvent.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Copyright 2018-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import static com.google.common.base.MoreObjects.toStringHelper;
-
-/**
- * Locked event.
- */
-public class LockEvent {
-  private final int id;
-  private final long version;
-
-  public LockEvent() {
-    this(0, 0);
-  }
-
-  public LockEvent(int id, long version) {
-    this.id = id;
-    this.version = version;
-  }
-
-  /**
-   * Returns the lock ID.
-   *
-   * @return The lock ID.
-   */
-  public int id() {
-    return id;
-  }
-
-  /**
-   * Returns the lock version.
-   *
-   * @return The lock version.
-   */
-  public long version() {
-    return version;
-  }
-
-  @Override
-  public String toString() {
-    return toStringHelper(this)
-        .add("id", id)
-        .add("version", version)
-        .toString();
-  }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/MapEntryUpdateResult.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/MapEntryUpdateResult.java
deleted file mode 100644
index 4bc783b..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/MapEntryUpdateResult.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-import com.google.common.base.MoreObjects;
-import org.onosproject.store.service.Versioned;
-
-/**
- * Result of a map entry update operation.
- * <p>
- * Both old and new values are accessible along with a flag that indicates if the
- * the value was updated. If flag is false, oldValue and newValue both
- * point to the same unmodified value.
- *
- * @param <K> key type
- * @param <V> result type
- */
-public class MapEntryUpdateResult<K, V> {
-
-    public enum Status {
-
-        /**
-         * Indicates a successful update.
-         */
-        OK,
-
-        /**
-         * Indicates a noop i.e. existing and new value are both null.
-         */
-        NOOP,
-
-        /**
-         * Indicates a failed update due to a write lock.
-         */
-        WRITE_LOCK,
-
-        /**
-         * Indicates a failed update due to a precondition check failure.
-         */
-        PRECONDITION_FAILED
-    }
-
-    private final Status status;
-    private final long version;
-    private final K key;
-    private final Versioned<V> result;
-
-    public MapEntryUpdateResult(Status status, long version, K key, Versioned<V> result) {
-        this.status = status;
-        this.version = version;
-        this.key = key;
-        this.result = result;
-    }
-
-    /**
-     * Returns {@code true} if the update was successful.
-     * @return {@code true} if yes, {@code false} otherwise
-     */
-    public boolean updated() {
-        return status == Status.OK;
-    }
-
-    /**
-     * Returns the update status.
-     * @return update status
-     */
-    public Status status() {
-        return status;
-    }
-
-    /**
-     * Returns the result version.
-     * @return result version
-     */
-    public long version() {
-        return version;
-    }
-
-    /**
-     * Returns the value.
-     * @return the value associated with key if updated was successful, otherwise current value
-     */
-    public Versioned<V> result() {
-        return result;
-    }
-
-    @Override
-    public String toString() {
-        return MoreObjects.toStringHelper(MapEntryUpdateResult.class)
-                .add("status", status)
-                .add("key", key)
-                .add("result", result)
-                .toString();
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/PrepareResult.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/PrepareResult.java
deleted file mode 100644
index f4e2bdc..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/PrepareResult.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-/**
- * Response enum for two phase commit prepare operation.
- */
-public enum PrepareResult {
-    /**
-     * Signifies a successful execution of the prepare operation.
-     */
-    OK,
-
-    /**
-     * Signifies some participants in a distributed prepare operation failed.
-     */
-    PARTIAL_FAILURE,
-
-    /**
-     * Signifies a failure to another transaction locking the underlying state.
-     */
-    CONCURRENT_TRANSACTION,
-
-    /**
-     * Signifies a optimistic lock failure. This can happen if underlying state has changed since it was last read.
-     */
-    OPTIMISTIC_LOCK_FAILURE,
-}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/RollbackResult.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/RollbackResult.java
deleted file mode 100644
index 3c6558b..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/RollbackResult.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Foundation
- *
- * 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.resources.impl;
-
-/**
- * Response enum for two phase commit rollback operation.
- */
-public enum RollbackResult {
-    /**
-     * Signifies a successful rollback execution.
-     */
-    OK,
-
-    /**
-     * Signifies a failure due to unrecognized transaction identifier.
-     */
-    UNKNOWN_TRANSACTION_ID,
-}