[ONOS-6594] Upgrade to Atomix 2.0.0

Change-Id: I6534bca1c8570b4e017f682953b876da29146675
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
new file mode 100644
index 0000000..d023788
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AbstractRaftPrimitive.java
@@ -0,0 +1,91 @@
+/*
+ * Copyright 2017-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.store.primitives.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
index ad1de2e..c84e03a 100644
--- 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
@@ -15,117 +15,131 @@
  */
 package org.onosproject.store.primitives.resources.impl;
 
-import io.atomix.copycat.client.CopycatClient;
-import io.atomix.resource.AbstractResource;
-import io.atomix.resource.ResourceTypeInfo;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.AddAndGet;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.Clear;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.DecrementAndGet;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.Get;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.GetAndAdd;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.GetAndDecrement;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.GetAndIncrement;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.IncrementAndGet;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.IsEmpty;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.Put;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.PutIfAbsent;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.Remove;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.RemoveValue;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.Replace;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.Size;
-import org.onosproject.store.service.AsyncAtomicCounterMap;
-
-import java.util.Properties;
 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.
  */
-@ResourceTypeInfo(id = -157, factory = AtomixAtomicCounterMapFactory.class)
-public class AtomixAtomicCounterMap extends AbstractResource<AtomixAtomicCounterMap>
-    implements AsyncAtomicCounterMap<String> {
+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(CopycatClient client, Properties options) {
-        super(client, options);
-    }
-
-    @Override
-    public String name() {
-        return null;
+    public AtomixAtomicCounterMap(RaftProxy proxy) {
+        super(proxy);
     }
 
     @Override
     public CompletableFuture<Long> incrementAndGet(String key) {
-        return client.submit(new IncrementAndGet(key));
+        return proxy.invoke(INCREMENT_AND_GET, SERIALIZER::encode, new IncrementAndGet(key), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Long> decrementAndGet(String key) {
-        return client.submit(new DecrementAndGet(key));
+        return proxy.invoke(DECREMENT_AND_GET, SERIALIZER::encode, new DecrementAndGet(key), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Long> getAndIncrement(String key) {
-        return client.submit(new GetAndIncrement(key));
+        return proxy.invoke(GET_AND_INCREMENT, SERIALIZER::encode, new GetAndIncrement(key), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Long> getAndDecrement(String key) {
-        return client.submit(new GetAndDecrement(key));
+        return proxy.invoke(GET_AND_DECREMENT, SERIALIZER::encode, new GetAndDecrement(key), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Long> addAndGet(String key, long delta) {
-        return client.submit(new AddAndGet(key, 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 client.submit(new GetAndAdd(key, delta));
+        return proxy.invoke(GET_AND_ADD, SERIALIZER::encode, new GetAndAdd(key, delta), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Long> get(String key) {
-        return client.submit(new Get(key));
+        return proxy.invoke(GET, SERIALIZER::encode, new Get(key), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Long> put(String key, long newValue) {
-        return client.submit(new Put(key, newValue));
+        return proxy.invoke(PUT, SERIALIZER::encode, new Put(key, newValue), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Long> putIfAbsent(String key, long newValue) {
-        return client.submit(new PutIfAbsent(key, 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 client.submit(new Replace(key, expectedOldValue, newValue));
+        return proxy.invoke(
+                REPLACE,
+                SERIALIZER::encode,
+                new Replace(key, expectedOldValue, newValue),
+                SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Long> remove(String key) {
-        return client.submit(new Remove(key));
+        return proxy.invoke(REMOVE, SERIALIZER::encode, new Remove(key), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Boolean> remove(String key, long value) {
-        return client.submit(new RemoveValue(key, value));
+        return proxy.invoke(REMOVE_VALUE, SERIALIZER::encode, new RemoveValue(key, value), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Integer> size() {
-        return client.submit(new Size());
+        return proxy.invoke(SIZE, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Boolean> isEmpty() {
-        return client.submit(new IsEmpty());
+        return proxy.invoke(IS_EMPTY, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Void> clear() {
-        return client.submit(new 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/AtomixAtomicCounterMapCommands.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapCommands.java
deleted file mode 100644
index 3a7f696..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapCommands.java
+++ /dev/null
@@ -1,332 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Laboratory
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.onosproject.store.primitives.resources.impl;
-
-import io.atomix.catalyst.buffer.BufferInput;
-import io.atomix.catalyst.buffer.BufferOutput;
-import io.atomix.catalyst.serializer.CatalystSerializable;
-import io.atomix.catalyst.serializer.SerializableTypeResolver;
-import io.atomix.catalyst.serializer.Serializer;
-import io.atomix.catalyst.serializer.SerializerRegistry;
-import io.atomix.copycat.Command;
-import io.atomix.copycat.Query;
-
-/**
- * Atomic counter map commands.
- */
-public final class AtomixAtomicCounterMapCommands {
-    private AtomixAtomicCounterMapCommands() {
-    }
-
-    public abstract static class AtomicCounterMapCommand<V> implements Command<V>, CatalystSerializable {
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.SNAPSHOT;
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-        }
-    }
-
-    public abstract static class AtomicCounterMapQuery<V> implements Query<V>, CatalystSerializable {
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-        }
-    }
-
-    public abstract static class KeyCommand<V> extends AtomicCounterMapCommand<V> {
-        private String key;
-
-        public KeyCommand() {
-        }
-
-        public KeyCommand(String key) {
-            this.key = key;
-        }
-
-        public String key() {
-            return key;
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            buffer.writeString(key);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            key = buffer.readString();
-        }
-    }
-
-    public abstract static class KeyQuery<V> extends AtomicCounterMapQuery<V> {
-        private String key;
-
-        public KeyQuery() {
-        }
-
-        public KeyQuery(String key) {
-            this.key = key;
-        }
-
-        public String key() {
-            return key;
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            buffer.writeString(key);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            key = buffer.readString();
-        }
-    }
-
-    public static class KeyValueCommand<V> extends KeyCommand<V> {
-        private long value;
-
-        public KeyValueCommand() {
-        }
-
-        public KeyValueCommand(String key, long value) {
-            super(key);
-            this.value = value;
-        }
-
-        public long value() {
-            return value;
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            buffer.writeLong(value);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            value = buffer.readLong();
-        }
-    }
-
-    public static class Get extends KeyQuery<Long> {
-        public Get() {
-        }
-
-        public Get(String key) {
-            super(key);
-        }
-    }
-
-    public static class Put extends KeyValueCommand<Long> {
-        public Put() {
-        }
-
-        public Put(String key, long value) {
-            super(key, value);
-        }
-    }
-
-    public static class PutIfAbsent extends KeyValueCommand<Long> {
-        public PutIfAbsent() {
-        }
-
-        public PutIfAbsent(String key, long value) {
-            super(key, value);
-        }
-    }
-
-    public static class Replace extends KeyCommand<Boolean> {
-        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;
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            buffer.writeLong(replace);
-            buffer.writeLong(value);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            replace = buffer.readLong();
-            value = buffer.readLong();
-        }
-    }
-
-    public static class Remove extends KeyCommand<Long> {
-        public Remove() {
-        }
-
-        public Remove(String key) {
-            super(key);
-        }
-    }
-
-    public static class RemoveValue extends KeyValueCommand<Boolean> {
-        public RemoveValue() {
-        }
-
-        public RemoveValue(String key, long value) {
-            super(key, value);
-        }
-    }
-
-    public static class IncrementAndGet extends KeyCommand<Long> {
-        public IncrementAndGet() {
-        }
-
-        public IncrementAndGet(String key) {
-            super(key);
-        }
-    }
-
-    public static class DecrementAndGet extends KeyCommand<Long> {
-        public DecrementAndGet(String key) {
-            super(key);
-        }
-
-        public DecrementAndGet() {
-        }
-    }
-
-    public static class GetAndIncrement extends KeyCommand<Long> {
-        public GetAndIncrement() {
-        }
-
-        public GetAndIncrement(String key) {
-            super(key);
-        }
-    }
-
-    public static class GetAndDecrement extends KeyCommand<Long> {
-        public GetAndDecrement() {
-        }
-
-        public GetAndDecrement(String key) {
-            super(key);
-        }
-    }
-
-    public abstract static class DeltaCommand extends KeyCommand<Long> {
-        private long delta;
-
-        public DeltaCommand() {
-        }
-
-        public DeltaCommand(String key, long delta) {
-            super(key);
-            this.delta = delta;
-        }
-
-        public long delta() {
-            return delta;
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            buffer.writeLong(delta);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            delta = buffer.readLong();
-        }
-    }
-
-    public static class AddAndGet extends DeltaCommand {
-        public AddAndGet() {
-        }
-
-        public AddAndGet(String key, long delta) {
-            super(key, delta);
-        }
-    }
-
-    public static class GetAndAdd extends DeltaCommand {
-        public GetAndAdd() {
-        }
-
-        public GetAndAdd(String key, long delta) {
-            super(key, delta);
-        }
-    }
-
-    public static class Size extends AtomicCounterMapQuery<Integer> {
-    }
-
-    public static class IsEmpty extends AtomicCounterMapQuery<Boolean> {
-    }
-
-    public static class Clear extends AtomicCounterMapCommand<Void> {
-    }
-
-    /**
-     * Counter map command type resolver.
-     */
-    public static class TypeResolver implements SerializableTypeResolver {
-        @Override
-        public void resolve(SerializerRegistry registry) {
-            registry.register(Get.class, -790);
-            registry.register(Put.class, -791);
-            registry.register(PutIfAbsent.class, -792);
-            registry.register(Replace.class, -793);
-            registry.register(Remove.class, -794);
-            registry.register(RemoveValue.class, -795);
-            registry.register(IncrementAndGet.class, -796);
-            registry.register(DecrementAndGet.class, -797);
-            registry.register(GetAndIncrement.class, -798);
-            registry.register(GetAndDecrement.class, -799);
-            registry.register(AddAndGet.class, -800);
-            registry.register(GetAndAdd.class, -801);
-            registry.register(Size.class, -801);
-            registry.register(IsEmpty.class, -801);
-            registry.register(Clear.class, -801);
-        }
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapFactory.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapFactory.java
deleted file mode 100644
index 4caf68e..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapFactory.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Laboratory
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.onosproject.store.primitives.resources.impl;
-
-import io.atomix.catalyst.serializer.SerializableTypeResolver;
-import io.atomix.copycat.client.CopycatClient;
-import io.atomix.resource.ResourceFactory;
-import io.atomix.resource.ResourceStateMachine;
-
-import java.util.Properties;
-
-/**
- * Atomic counter map factory.
- */
-public class AtomixAtomicCounterMapFactory implements ResourceFactory<AtomixAtomicCounterMap> {
-
-    @Override
-    public SerializableTypeResolver createSerializableTypeResolver() {
-        return new AtomixAtomicCounterMapCommands.TypeResolver();
-    }
-
-    @Override
-    public ResourceStateMachine createStateMachine(Properties config) {
-        return new AtomixAtomicCounterMapState(config);
-    }
-
-    @Override
-    public AtomixAtomicCounterMap createInstance(CopycatClient client, Properties options) {
-        return new AtomixAtomicCounterMap(client, options);
-    }
-}
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
new file mode 100644
index 0000000..df3508f
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapOperations.java
@@ -0,0 +1,248 @@
+/*
+ * Copyright 2017-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.store.primitives.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("put", OperationType.COMMAND),
+    PUT_IF_ABSENT("putIfAbsent", OperationType.COMMAND),
+    GET("get", OperationType.QUERY),
+    REPLACE("replace", OperationType.COMMAND),
+    REMOVE("remove", OperationType.COMMAND),
+    REMOVE_VALUE("removeValue", OperationType.COMMAND),
+    GET_AND_INCREMENT("getAndIncrement", OperationType.COMMAND),
+    GET_AND_DECREMENT("getAndDecrement", OperationType.COMMAND),
+    INCREMENT_AND_GET("incrementAndGet", OperationType.COMMAND),
+    DECREMENT_AND_GET("decrementAndGet", OperationType.COMMAND),
+    ADD_AND_GET("addAndGet", OperationType.COMMAND),
+    GET_AND_ADD("getAndAdd", OperationType.COMMAND),
+    SIZE("size", OperationType.QUERY),
+    IS_EMPTY("isEmpty", OperationType.QUERY),
+    CLEAR("clear", OperationType.COMMAND);
+
+    private final String id;
+    private final OperationType type;
+
+    AtomixAtomicCounterMapOperations(String id, OperationType type) {
+        this.id = id;
+        this.type = type;
+    }
+
+    @Override
+    public String id() {
+        return id;
+    }
+
+    @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
new file mode 100644
index 0000000..b895d71
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapService.java
@@ -0,0 +1,303 @@
+/*
+ * Copyright 2017-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.store.primitives.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/AtomixAtomicCounterMapState.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapState.java
deleted file mode 100644
index 1aed4a8..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixAtomicCounterMapState.java
+++ /dev/null
@@ -1,347 +0,0 @@
-/*
- * Copyright 2017-present Open Networking Laboratory
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.onosproject.store.primitives.resources.impl;
-
-import io.atomix.copycat.server.Commit;
-import io.atomix.copycat.server.Snapshottable;
-import io.atomix.copycat.server.StateMachineExecutor;
-import io.atomix.copycat.server.storage.snapshot.SnapshotReader;
-import io.atomix.copycat.server.storage.snapshot.SnapshotWriter;
-import io.atomix.resource.ResourceStateMachine;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.AddAndGet;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.Clear;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.DecrementAndGet;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.Get;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.GetAndAdd;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.GetAndDecrement;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.GetAndIncrement;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.IncrementAndGet;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.IsEmpty;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.Put;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.PutIfAbsent;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.Remove;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.RemoveValue;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.Replace;
-import org.onosproject.store.primitives.resources.impl.AtomixAtomicCounterMapCommands.Size;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Properties;
-
-/**
- * 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 AtomixAtomicCounterMapState extends ResourceStateMachine implements Snapshottable {
-    private Map<String, Long> map = new HashMap<>();
-
-    public AtomixAtomicCounterMapState(Properties config) {
-        super(config);
-    }
-
-    @Override
-    protected void configure(StateMachineExecutor executor) {
-        executor.register(Put.class, this::put);
-        executor.register(PutIfAbsent.class, this::putIfAbsent);
-        executor.register(Get.class, this::get);
-        executor.register(Replace.class, this::replace);
-        executor.register(Remove.class, this::remove);
-        executor.register(RemoveValue.class, this::removeValue);
-        executor.register(GetAndIncrement.class, this::getAndIncrement);
-        executor.register(GetAndDecrement.class, this::getAndDecrement);
-        executor.register(IncrementAndGet.class, this::incrementAndGet);
-        executor.register(DecrementAndGet.class, this::decrementAndGet);
-        executor.register(AddAndGet.class, this::addAndGet);
-        executor.register(GetAndAdd.class, this::getAndAdd);
-        executor.register(Size.class, this::size);
-        executor.register(IsEmpty.class, this::isEmpty);
-        executor.register(Clear.class, this::clear);
-    }
-
-    @Override
-    public void snapshot(SnapshotWriter writer) {
-        writer.writeObject(map);
-    }
-
-    @Override
-    public void install(SnapshotReader reader) {
-        map = reader.readObject();
-    }
-
-    /**
-     * 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) {
-        try {
-            return primitive(map.put(commit.operation().key(), commit.operation().value()));
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * 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) {
-        try {
-            return primitive(map.putIfAbsent(commit.operation().key(), commit.operation().value()));
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles a {@link Get} query which implements {@link AtomixAtomicCounterMap#get(String)}}.
-     *
-     * @param commit get commit
-     * @return get result
-     */
-    protected long get(Commit<Get> commit) {
-        try {
-            return primitive(map.get(commit.operation().key()));
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * 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) {
-        try {
-            Long value = map.get(commit.operation().key());
-            if (value == null) {
-                if (commit.operation().replace() == 0) {
-                    map.put(commit.operation().key(), commit.operation().value());
-                    return true;
-                } else {
-                    return false;
-                }
-            } else if (value == commit.operation().replace()) {
-                map.put(commit.operation().key(), commit.operation().value());
-                return true;
-            }
-            return false;
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles a {@link Remove} command which implements {@link AtomixAtomicCounterMap#remove(String)}.
-     *
-     * @param commit remove commit
-     * @return remove result
-     */
-    protected long remove(Commit<Remove> commit) {
-        try {
-            return primitive(map.remove(commit.operation().key()));
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * 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) {
-        try {
-            Long value = map.get(commit.operation().key());
-            if (value == null) {
-                if (commit.operation().value() == 0) {
-                    map.remove(commit.operation().key());
-                    return true;
-                }
-                return false;
-            } else if (value == commit.operation().value()) {
-                map.remove(commit.operation().key());
-                return true;
-            }
-            return false;
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles a {@link GetAndIncrement} command which implements
-     * {@link AtomixAtomicCounterMap#getAndIncrement(String)}.
-     *
-     * @param commit getAndIncrement commit
-     * @return getAndIncrement result
-     */
-    protected long getAndIncrement(Commit<GetAndIncrement> commit) {
-        try {
-            long value = primitive(map.get(commit.operation().key()));
-            map.put(commit.operation().key(), value + 1);
-            return value;
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles a {@link GetAndDecrement} command which implements
-     * {@link AtomixAtomicCounterMap#getAndDecrement(String)}.
-     *
-     * @param commit getAndDecrement commit
-     * @return getAndDecrement result
-     */
-    protected long getAndDecrement(Commit<GetAndDecrement> commit) {
-        try {
-            long value = primitive(map.get(commit.operation().key()));
-            map.put(commit.operation().key(), value - 1);
-            return value;
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles a {@link IncrementAndGet} command which implements
-     * {@link AtomixAtomicCounterMap#incrementAndGet(String)}.
-     *
-     * @param commit incrementAndGet commit
-     * @return incrementAndGet result
-     */
-    protected long incrementAndGet(Commit<IncrementAndGet> commit) {
-        try {
-            long value = primitive(map.get(commit.operation().key()));
-            map.put(commit.operation().key(), ++value);
-            return value;
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles a {@link DecrementAndGet} command which implements
-     * {@link AtomixAtomicCounterMap#decrementAndGet(String)}.
-     *
-     * @param commit decrementAndGet commit
-     * @return decrementAndGet result
-     */
-    protected long decrementAndGet(Commit<DecrementAndGet> commit) {
-        try {
-            long value = primitive(map.get(commit.operation().key()));
-            map.put(commit.operation().key(), --value);
-            return value;
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * 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) {
-        try {
-            long value = primitive(map.get(commit.operation().key()));
-            value += commit.operation().delta();
-            map.put(commit.operation().key(), value);
-            return value;
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * 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) {
-        try {
-            long value = primitive(map.get(commit.operation().key()));
-            map.put(commit.operation().key(), value + commit.operation().delta());
-            return value;
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles a {@link Size} query which implements {@link AtomixAtomicCounterMap#size()}.
-     *
-     * @param commit size commit
-     * @return size result
-     */
-    protected int size(Commit<Size> commit) {
-        try {
-            return map.size();
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles an {@link IsEmpty} query which implements {@link AtomixAtomicCounterMap#isEmpty()}.
-     *
-     * @param commit isEmpty commit
-     * @return isEmpty result
-     */
-    protected boolean isEmpty(Commit<IsEmpty> commit) {
-        try {
-            return map.isEmpty();
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles a {@link Clear} command which implements {@link AtomixAtomicCounterMap#clear()}.
-     *
-     * @param commit clear commit
-     */
-    protected void clear(Commit<Clear> commit) {
-        try {
-            map.clear();
-        } finally {
-            commit.close();
-        }
-    }
-}
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
index 7220c37..f3a1ea2 100644
--- 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
@@ -15,146 +15,149 @@
  */
 package org.onosproject.store.primitives.resources.impl;
 
-import io.atomix.copycat.client.CopycatClient;
-import io.atomix.resource.AbstractResource;
-import io.atomix.resource.ResourceTypeInfo;
-
-import java.util.concurrent.ConcurrentHashMap;
 import java.util.Collection;
 import java.util.ConcurrentModificationException;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
-import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.Executor;
 import java.util.concurrent.atomic.AtomicReference;
 import java.util.function.BiFunction;
-import java.util.function.Consumer;
 import java.util.function.Predicate;
 
+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.MapUpdate;
 import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.Clear;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.ContainsKey;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.ContainsValue;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.EntrySet;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.Get;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.GetOrDefault;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.IsEmpty;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.KeySet;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.Listen;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.Size;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.TransactionBegin;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.TransactionCommit;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.TransactionPrepare;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.TransactionPrepareAndCommit;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.TransactionRollback;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.Unlisten;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.UpdateAndGet;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.Values;
+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.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.primitives.resources.impl.AtomixConsistentMapOperations.UpdateAndGet;
+import org.onosproject.store.serializers.KryoNamespaces;
 import org.onosproject.store.service.AsyncConsistentMap;
 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 com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
+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.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.KEY_SET;
+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.REMOVE_LISTENER;
+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.UPDATE_AND_GET;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.VALUES;
 
 /**
  * Distributed resource providing the {@link AsyncConsistentMap} primitive.
  */
-@ResourceTypeInfo(id = -151, factory = AtomixConsistentMapFactory.class)
-public class AtomixConsistentMap extends AbstractResource<AtomixConsistentMap>
-    implements AsyncConsistentMap<String, byte[]> {
+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)
+            .build());
 
-    private final Set<Consumer<Status>> statusChangeListeners = Sets.newCopyOnWriteArraySet();
     private final Map<MapEventListener<String, byte[]>, Executor> mapEventListeners = new ConcurrentHashMap<>();
 
-    public static final String CHANGE_SUBJECT = "changeEvents";
-
-    public AtomixConsistentMap(CopycatClient client, Properties properties) {
-        super(client, properties);
-    }
-
-    @Override
-    public String name() {
-        return null;
-    }
-
-    @Override
-    public CompletableFuture<AtomixConsistentMap> open() {
-        return super.open().thenApply(result -> {
-            client.onStateChange(state -> {
-                if (state == CopycatClient.State.CONNECTED && isListening()) {
-                    client.submit(new Listen());
-                }
-            });
-            client.onEvent(CHANGE_SUBJECT, this::handleEvent);
-            return result;
+    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);
+            }
         });
     }
 
     private void handleEvent(List<MapEvent<String, byte[]>> events) {
         events.forEach(event ->
-            mapEventListeners.forEach((listener, executor) -> executor.execute(() -> listener.event(event))));
+                mapEventListeners.forEach((listener, executor) -> executor.execute(() -> listener.event(event))));
     }
 
     @Override
     public CompletableFuture<Boolean> isEmpty() {
-        return client.submit(new IsEmpty());
+        return proxy.invoke(IS_EMPTY, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Integer> size() {
-        return client.submit(new Size());
+        return proxy.invoke(SIZE, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Boolean> containsKey(String key) {
-        return client.submit(new ContainsKey(key));
+        return proxy.invoke(CONTAINS_KEY, SERIALIZER::encode, new ContainsKey(key), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Boolean> containsValue(byte[] value) {
-        return client.submit(new ContainsValue(value));
+        return proxy.invoke(CONTAINS_VALUE, SERIALIZER::encode, new ContainsValue(value), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Versioned<byte[]>> get(String key) {
-        return client.submit(new Get(key));
+        return proxy.invoke(GET, SERIALIZER::encode, new Get(key), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Versioned<byte[]>> getOrDefault(String key, byte[] defaultValue) {
-        return client.submit(new GetOrDefault(key, defaultValue));
+        return proxy.invoke(
+                GET_OR_DEFAULT,
+                SERIALIZER::encode,
+                new GetOrDefault(key, defaultValue),
+                SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Set<String>> keySet() {
-        return client.submit(new KeySet());
+        return proxy.invoke(KEY_SET, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Collection<Versioned<byte[]>>> values() {
-        return client.submit(new Values());
+        return proxy.invoke(VALUES, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Set<Entry<String, Versioned<byte[]>>>> entrySet() {
-        return client.submit(new EntrySet());
+        return proxy.invoke(ENTRY_SET, SERIALIZER::decode);
     }
 
     @Override
     @SuppressWarnings("unchecked")
     public CompletableFuture<Versioned<byte[]>> put(String key, byte[] value) {
-        return client.submit(new UpdateAndGet(key, value, Match.ANY, Match.ANY))
+        return proxy.<UpdateAndGet, MapEntryUpdateResult<String, byte[]>>invoke(
+                UPDATE_AND_GET,
+                SERIALIZER::encode,
+                new UpdateAndGet(key, value, Match.ANY, Match.ANY),
+                SERIALIZER::decode)
                 .whenComplete((r, e) -> throwIfLocked(r.status()))
                 .thenApply(v -> v.oldValue());
     }
@@ -162,7 +165,11 @@
     @Override
     @SuppressWarnings("unchecked")
     public CompletableFuture<Versioned<byte[]>> putAndGet(String key, byte[] value) {
-        return client.submit(new UpdateAndGet(key, value, Match.ANY, Match.ANY))
+        return proxy.<UpdateAndGet, MapEntryUpdateResult<String, byte[]>>invoke(
+                UPDATE_AND_GET,
+                SERIALIZER::encode,
+                new UpdateAndGet(key, value, Match.ANY, Match.ANY),
+                SERIALIZER::decode)
                 .whenComplete((r, e) -> throwIfLocked(r.status()))
                 .thenApply(v -> v.newValue());
     }
@@ -170,14 +177,23 @@
     @Override
     @SuppressWarnings("unchecked")
     public CompletableFuture<Versioned<byte[]>> putIfAbsent(String key, byte[] value) {
-        return client.submit(new UpdateAndGet(key, value, Match.NULL, Match.ANY))
+        return proxy.<UpdateAndGet, MapEntryUpdateResult<String, byte[]>>invoke(
+                UPDATE_AND_GET,
+                SERIALIZER::encode,
+                new UpdateAndGet(key, value, Match.NULL, Match.ANY),
+                SERIALIZER::decode)
                 .whenComplete((r, e) -> throwIfLocked(r.status()))
                 .thenApply(v -> v.oldValue());
     }
+
     @Override
     @SuppressWarnings("unchecked")
     public CompletableFuture<Versioned<byte[]>> remove(String key) {
-        return client.submit(new UpdateAndGet(key, null, Match.ANY, Match.ANY))
+        return proxy.<UpdateAndGet, MapEntryUpdateResult<String, byte[]>>invoke(
+                UPDATE_AND_GET,
+                SERIALIZER::encode,
+                new UpdateAndGet(key, null, Match.ANY, Match.ANY),
+                SERIALIZER::decode)
                 .whenComplete((r, e) -> throwIfLocked(r.status()))
                 .thenApply(v -> v.oldValue());
     }
@@ -185,7 +201,11 @@
     @Override
     @SuppressWarnings("unchecked")
     public CompletableFuture<Boolean> remove(String key, byte[] value) {
-        return client.submit(new UpdateAndGet(key, null, Match.ifValue(value), Match.ANY))
+        return proxy.<UpdateAndGet, MapEntryUpdateResult<String, byte[]>>invoke(
+                UPDATE_AND_GET,
+                SERIALIZER::encode,
+                new UpdateAndGet(key, null, Match.ifValue(value), Match.ANY),
+                SERIALIZER::decode)
                 .whenComplete((r, e) -> throwIfLocked(r.status()))
                 .thenApply(v -> v.updated());
     }
@@ -193,7 +213,11 @@
     @Override
     @SuppressWarnings("unchecked")
     public CompletableFuture<Boolean> remove(String key, long version) {
-        return client.submit(new UpdateAndGet(key, null, Match.ANY, Match.ifValue(version)))
+        return proxy.<UpdateAndGet, MapEntryUpdateResult<String, byte[]>>invoke(
+                UPDATE_AND_GET,
+                SERIALIZER::encode,
+                new UpdateAndGet(key, null, Match.ANY, Match.ifValue(version)),
+                SERIALIZER::decode)
                 .whenComplete((r, e) -> throwIfLocked(r.status()))
                 .thenApply(v -> v.updated());
     }
@@ -201,7 +225,11 @@
     @Override
     @SuppressWarnings("unchecked")
     public CompletableFuture<Versioned<byte[]>> replace(String key, byte[] value) {
-        return client.submit(new UpdateAndGet(key, value, Match.NOT_NULL, Match.ANY))
+        return proxy.<UpdateAndGet, MapEntryUpdateResult<String, byte[]>>invoke(
+                UPDATE_AND_GET,
+                SERIALIZER::encode,
+                new UpdateAndGet(key, value, Match.NOT_NULL, Match.ANY),
+                SERIALIZER::decode)
                 .whenComplete((r, e) -> throwIfLocked(r.status()))
                 .thenApply(v -> v.oldValue());
     }
@@ -209,7 +237,11 @@
     @Override
     @SuppressWarnings("unchecked")
     public CompletableFuture<Boolean> replace(String key, byte[] oldValue, byte[] newValue) {
-        return client.submit(new UpdateAndGet(key, newValue, Match.ifValue(oldValue), Match.ANY))
+        return proxy.<UpdateAndGet, MapEntryUpdateResult<String, byte[]>>invoke(
+                UPDATE_AND_GET,
+                SERIALIZER::encode,
+                new UpdateAndGet(key, newValue, Match.ifValue(oldValue), Match.ANY),
+                SERIALIZER::decode)
                 .whenComplete((r, e) -> throwIfLocked(r.status()))
                 .thenApply(v -> v.updated());
     }
@@ -217,14 +249,18 @@
     @Override
     @SuppressWarnings("unchecked")
     public CompletableFuture<Boolean> replace(String key, long oldVersion, byte[] newValue) {
-        return client.submit(new UpdateAndGet(key, newValue, Match.ANY, Match.ifValue(oldVersion)))
+        return proxy.<UpdateAndGet, MapEntryUpdateResult<String, byte[]>>invoke(
+                UPDATE_AND_GET,
+                SERIALIZER::encode,
+                new UpdateAndGet(key, newValue, Match.ANY, Match.ifValue(oldVersion)),
+                SERIALIZER::decode)
                 .whenComplete((r, e) -> throwIfLocked(r.status()))
                 .thenApply(v -> v.updated());
     }
 
     @Override
     public CompletableFuture<Void> clear() {
-        return client.submit(new Clear())
+        return proxy.<MapEntryUpdateResult.Status>invoke(CLEAR, SERIALIZER::decode)
                 .whenComplete((r, e) -> throwIfLocked(r))
                 .thenApply(v -> null);
     }
@@ -233,7 +269,7 @@
     @SuppressWarnings("unchecked")
     public CompletableFuture<Versioned<byte[]>> computeIf(String key,
             Predicate<? super byte[]> condition,
-                    BiFunction<? super String, ? super byte[], ? extends byte[]> remappingFunction) {
+            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.
@@ -255,27 +291,31 @@
             }
             Match<byte[]> valueMatch = r1 == null ? Match.NULL : Match.ANY;
             Match<Long> versionMatch = r1 == null ? Match.ANY : Match.ifValue(r1.version());
-            return client.submit(new UpdateAndGet(key,
-                                                  computedValue.get(),
-                                                  valueMatch,
-                                                  versionMatch))
-                         .whenComplete((r, e) -> throwIfLocked(r.status()))
-                         .thenCompose(r -> {
-                             if (r.status() == MapEntryUpdateResult.Status.PRECONDITION_FAILED ||
-                                     r.status() == MapEntryUpdateResult.Status.WRITE_LOCK) {
-                                 return Tools.exceptionalFuture(new ConsistentMapException.ConcurrentModification());
-                             }
-                             return CompletableFuture.completedFuture(r);
-                         })
-                         .thenApply(v -> v.newValue());
+            return proxy.<UpdateAndGet, MapEntryUpdateResult<String, byte[]>>invoke(
+                    UPDATE_AND_GET,
+                    SERIALIZER::encode,
+                    new UpdateAndGet(key,
+                            computedValue.get(),
+                            valueMatch,
+                            versionMatch),
+                    SERIALIZER::decode)
+                    .whenComplete((r, e) -> throwIfLocked(r.status()))
+                    .thenCompose(r -> {
+                        if (r.status() == MapEntryUpdateResult.Status.PRECONDITION_FAILED ||
+                                r.status() == MapEntryUpdateResult.Status.WRITE_LOCK) {
+                            return Tools.exceptionalFuture(new ConsistentMapException.ConcurrentModification());
+                        }
+                        return CompletableFuture.completedFuture(r);
+                    })
+                    .thenApply(v -> v.newValue());
         });
     }
 
     @Override
     public synchronized CompletableFuture<Void> addListener(MapEventListener<String, byte[]> listener,
-                                                            Executor executor) {
+            Executor executor) {
         if (mapEventListeners.isEmpty()) {
-            return client.submit(new Listen()).thenRun(() -> mapEventListeners.put(listener, executor));
+            return proxy.invoke(ADD_LISTENER).thenRun(() -> mapEventListeners.put(listener, executor));
         } else {
             mapEventListeners.put(listener, executor);
             return CompletableFuture.completedFuture(null);
@@ -285,7 +325,7 @@
     @Override
     public synchronized CompletableFuture<Void> removeListener(MapEventListener<String, byte[]> listener) {
         if (mapEventListeners.remove(listener) != null && mapEventListeners.isEmpty()) {
-            return client.submit(new Unlisten()).thenApply(v -> null);
+            return proxy.invoke(REMOVE_LISTENER).thenApply(v -> null);
         }
         return CompletableFuture.completedFuture(null);
     }
@@ -298,49 +338,55 @@
 
     @Override
     public CompletableFuture<Version> begin(TransactionId transactionId) {
-        return client.submit(new TransactionBegin(transactionId)).thenApply(Version::new);
+        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 client.submit(new TransactionPrepare(transactionLog))
+    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 client.submit(new TransactionPrepareAndCommit(transactionLog))
+    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 client.submit(new TransactionCommit(transactionId)).thenApply(v -> null);
+        return proxy.<TransactionCommit, CommitResult>invoke(
+                COMMIT,
+                SERIALIZER::encode,
+                new TransactionCommit(transactionId),
+                SERIALIZER::decode)
+                .thenApply(v -> null);
     }
 
     @Override
     public CompletableFuture<Void> rollback(TransactionId transactionId) {
-        return client.submit(new TransactionRollback(transactionId)).thenApply(v -> null);
-    }
-
-    @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);
+        return proxy.invoke(
+                ROLLBACK,
+                SERIALIZER::encode,
+                new TransactionRollback(transactionId),
+                SERIALIZER::decode)
+                .thenApply(v -> null);
     }
 
     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/AtomixConsistentMapCommands.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapCommands.java
deleted file mode 100644
index 8dd4096..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapCommands.java
+++ /dev/null
@@ -1,632 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Laboratory
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.onosproject.store.primitives.resources.impl;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.Set;
-
-import com.google.common.base.MoreObjects;
-import io.atomix.catalyst.buffer.BufferInput;
-import io.atomix.catalyst.buffer.BufferOutput;
-import io.atomix.catalyst.serializer.CatalystSerializable;
-import io.atomix.catalyst.serializer.SerializableTypeResolver;
-import io.atomix.catalyst.serializer.Serializer;
-import io.atomix.catalyst.serializer.SerializerRegistry;
-import io.atomix.catalyst.util.Assert;
-import io.atomix.copycat.Command;
-import io.atomix.copycat.Query;
-import org.onlab.util.Match;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Versioned;
-
-/**
- * {@link AtomixConsistentMap} resource state machine operations.
- */
-public final class AtomixConsistentMapCommands {
-
-    private AtomixConsistentMapCommands() {
-    }
-
-    /**
-     * Abstract map command.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class MapCommand<V> implements Command<V>, CatalystSerializable {
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .toString();
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-        }
-    }
-
-    /**
-     * Abstract map query.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class MapQuery<V> implements Query<V>, CatalystSerializable {
-
-        @Override
-        public ConsistencyLevel consistency() {
-          return ConsistencyLevel.SEQUENTIAL;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .toString();
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-        }
-    }
-
-    /**
-     * Abstract key-based query.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class KeyQuery<V> extends MapQuery<V> {
-        protected String key;
-
-        public KeyQuery() {
-        }
-
-        public KeyQuery(String key) {
-            this.key = Assert.notNull(key, "key");
-        }
-
-        /**
-         * Returns the key.
-         * @return key
-         */
-        public String key() {
-            return key;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("key", key)
-                    .toString();
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            serializer.writeObject(key, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            key = serializer.readObject(buffer);
-        }
-    }
-
-    /**
-     * Abstract value-based query.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class ValueQuery<V> extends MapQuery<V> {
-        protected byte[] value;
-
-        public ValueQuery() {
-        }
-
-        public ValueQuery(byte[] value) {
-            this.value = Assert.notNull(value, "value");
-        }
-
-        /**
-         * Returns the value.
-         * @return value
-         */
-        public byte[] value() {
-            return value;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("value", value)
-                    .toString();
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            serializer.writeObject(value, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            value = serializer.readObject(buffer);
-        }
-    }
-
-    /**
-     * Contains key command.
-     */
-    @SuppressWarnings("serial")
-    public static class ContainsKey extends KeyQuery<Boolean> {
-        public ContainsKey() {
-        }
-
-        public ContainsKey(String key) {
-            super(key);
-        }
-    }
-
-    /**
-     * Contains value command.
-     */
-    @SuppressWarnings("serial")
-    public static class ContainsValue extends ValueQuery<Boolean> {
-        public ContainsValue() {
-        }
-
-        public ContainsValue(byte[] value) {
-            super(value);
-        }
-    }
-
-    /**
-     * Transaction begin command.
-     */
-    public static class TransactionBegin extends MapCommand<Long> {
-        private TransactionId transactionId;
-
-        public TransactionBegin() {
-        }
-
-        public TransactionBegin(TransactionId transactionId) {
-            this.transactionId = transactionId;
-        }
-
-        public TransactionId transactionId() {
-            return transactionId;
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            serializer.writeObject(transactionId, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            transactionId = serializer.readObject(buffer);
-        }
-    }
-
-    /**
-     * Map prepare command.
-     */
-    @SuppressWarnings("serial")
-    public static class TransactionPrepare extends MapCommand<PrepareResult> {
-        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 void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            serializer.writeObject(transactionLog, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            transactionLog = serializer.readObject(buffer);
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.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);
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.TOMBSTONE;
-        }
-    }
-
-    /**
-     * Map transaction commit command.
-     */
-    @SuppressWarnings("serial")
-    public static class TransactionCommit extends MapCommand<CommitResult> {
-        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 void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            serializer.writeObject(transactionId, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            transactionId = serializer.readObject(buffer);
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.TOMBSTONE;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("transactionId", transactionId)
-                    .toString();
-        }
-    }
-
-    /**
-     * Map transaction rollback command.
-     */
-    @SuppressWarnings("serial")
-    public static class TransactionRollback extends MapCommand<RollbackResult> {
-        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 void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            serializer.writeObject(transactionId, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            transactionId = serializer.readObject(buffer);
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.TOMBSTONE;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("transactionId", transactionId)
-                    .toString();
-        }
-    }
-
-    /**
-     * Map update command.
-     */
-    @SuppressWarnings("serial")
-    public static class UpdateAndGet extends MapCommand<MapEntryUpdateResult<String, byte[]>> {
-        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;
-        }
-
-        /**
-         * Returns the key.
-         * @return key
-         */
-        public String key() {
-            return this.key;
-        }
-
-        /**
-         * Returns the value.
-         * @return value
-         */
-        public byte[] value() {
-            return this.value;
-        }
-
-        /**
-         * Returns the value match.
-         * @return value match
-         */
-        public Match<byte[]> valueMatch() {
-            return this.valueMatch;
-        }
-
-        /**
-         * Returns the version match.
-         * @return version match
-         */
-        public Match<Long> versionMatch() {
-            return this.versionMatch;
-        }
-
-        @Override
-        public CompactionMode compaction() {
-          return value == null ? CompactionMode.TOMBSTONE : CompactionMode.FULL;
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            serializer.writeObject(key, buffer);
-            serializer.writeObject(value, buffer);
-            serializer.writeObject(valueMatch, buffer);
-            serializer.writeObject(versionMatch, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            key = serializer.readObject(buffer);
-            value = serializer.readObject(buffer);
-            valueMatch = serializer.readObject(buffer);
-            versionMatch = serializer.readObject(buffer);
-        }
-
-        @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 KeyQuery<Versioned<byte[]>> {
-        public Get() {
-        }
-
-        public Get(String key) {
-            super(key);
-        }
-    }
-
-    /**
-     * Get or default query.
-     */
-    @SuppressWarnings("serial")
-    public static class GetOrDefault extends KeyQuery<Versioned<byte[]>> {
-        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 void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            serializer.writeObject(defaultValue, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            defaultValue = serializer.readObject(buffer);
-        }
-    }
-
-    /**
-     * Is empty query.
-     */
-    @SuppressWarnings("serial")
-    public static class IsEmpty extends MapQuery<Boolean> {
-    }
-
-    /**
-     * KeySet query.
-     */
-    @SuppressWarnings("serial")
-    public static class KeySet extends MapQuery<Set<String>> {
-    }
-
-    /**
-     * ValueSet query.
-     */
-    @SuppressWarnings("serial")
-    public static class Values extends MapQuery<Collection<Versioned<byte[]>>> {
-    }
-
-    /**
-     * EntrySet query.
-     */
-    @SuppressWarnings("serial")
-    public static class EntrySet extends MapQuery<Set<Map.Entry<String, Versioned<byte[]>>>> {
-    }
-
-    /**
-     * Size query.
-     */
-    @SuppressWarnings("serial")
-    public static class Size extends MapQuery<Integer> {
-    }
-
-    /**
-     * Clear command.
-     */
-    @SuppressWarnings("serial")
-    public static class Clear extends MapCommand<MapEntryUpdateResult.Status> {
-        @Override
-        public CompactionMode compaction() {
-          return CompactionMode.TOMBSTONE;
-        }
-    }
-
-    /**
-     * Change listen.
-     */
-    @SuppressWarnings("serial")
-    public static class Listen implements Command<Void>, CatalystSerializable {
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.QUORUM;
-        }
-    }
-
-    /**
-     * Change unlisten.
-     */
-    @SuppressWarnings("serial")
-    public static class Unlisten implements Command<Void>, CatalystSerializable {
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.TOMBSTONE;
-        }
-    }
-
-    /**
-     * Map command type resolver.
-     */
-    public static class TypeResolver implements SerializableTypeResolver {
-        @Override
-        public void resolve(SerializerRegistry registry) {
-            registry.register(ContainsKey.class, -761);
-            registry.register(ContainsValue.class, -762);
-            registry.register(Get.class, -763);
-            registry.register(GetOrDefault.class, -778);
-            registry.register(EntrySet.class, -764);
-            registry.register(Values.class, -765);
-            registry.register(KeySet.class, -766);
-            registry.register(Clear.class, -767);
-            registry.register(IsEmpty.class, -768);
-            registry.register(Size.class, -769);
-            registry.register(Listen.class, -770);
-            registry.register(Unlisten.class, -771);
-            registry.register(TransactionBegin.class, -777);
-            registry.register(TransactionPrepare.class, -772);
-            registry.register(TransactionCommit.class, -773);
-            registry.register(TransactionRollback.class, -774);
-            registry.register(TransactionPrepareAndCommit.class, -775);
-            registry.register(UpdateAndGet.class, -776);
-        }
-    }
-}
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
new file mode 100644
index 0000000..f0e6a42
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapEvents.java
@@ -0,0 +1,47 @@
+/*
+ * Copyright 2017-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.store.primitives.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("change");
+
+    private final String id;
+
+    AtomixConsistentMapEvents(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(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/AtomixConsistentMapFactory.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapFactory.java
deleted file mode 100644
index bf1523a..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapFactory.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Laboratory
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.onosproject.store.primitives.resources.impl;
-
-import io.atomix.catalyst.serializer.SerializableTypeResolver;
-import io.atomix.copycat.client.CopycatClient;
-import io.atomix.resource.ResourceFactory;
-import io.atomix.resource.ResourceStateMachine;
-
-import java.util.Properties;
-
-/**
- * {@link AtomixConsistentMap} resource factory.
- *
- */
-public class AtomixConsistentMapFactory implements ResourceFactory<AtomixConsistentMap> {
-
-    @Override
-    public SerializableTypeResolver createSerializableTypeResolver() {
-      return new AtomixConsistentMapCommands.TypeResolver();
-    }
-
-    @Override
-    public ResourceStateMachine createStateMachine(Properties config) {
-      return new AtomixConsistentMapState(config);
-    }
-
-    @Override
-    public AtomixConsistentMap createInstance(CopycatClient client, Properties options) {
-      return new AtomixConsistentMap(client, options);
-    }
-  }
\ No newline at end of file
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
new file mode 100644
index 0000000..2d81787
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapOperations.java
@@ -0,0 +1,417 @@
+/*
+ * Copyright 2016-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.store.primitives.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.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.Preconditions.checkNotNull;
+
+/**
+ * {@link AtomixConsistentMap} resource state machine operations.
+ */
+public enum AtomixConsistentMapOperations implements OperationId {
+    IS_EMPTY("isEmpty", OperationType.QUERY),
+    SIZE("size", OperationType.QUERY),
+    CONTAINS_KEY("containsKey", OperationType.QUERY),
+    CONTAINS_VALUE("containsValue", OperationType.QUERY),
+    GET("get", OperationType.QUERY),
+    GET_OR_DEFAULT("getOrDefault", OperationType.QUERY),
+    KEY_SET("keySet", OperationType.QUERY),
+    VALUES("values", OperationType.QUERY),
+    ENTRY_SET("entrySet", OperationType.QUERY),
+    UPDATE_AND_GET("updateAndGet", OperationType.COMMAND),
+    CLEAR("clear", OperationType.COMMAND),
+    ADD_LISTENER("addListener", OperationType.COMMAND),
+    REMOVE_LISTENER("removeListener", OperationType.COMMAND),
+    BEGIN("begin", OperationType.COMMAND),
+    PREPARE("prepare", OperationType.COMMAND),
+    PREPARE_AND_COMMIT("prepareAndCommit", OperationType.COMMAND),
+    COMMIT("commit", OperationType.COMMAND),
+    ROLLBACK("rollback", OperationType.COMMAND);
+
+    private final String id;
+    private final OperationType type;
+
+    AtomixConsistentMapOperations(String id, OperationType type) {
+        this.id = id;
+        this.type = type;
+    }
+
+    @Override
+    public String id() {
+        return id;
+    }
+
+    @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(UpdateAndGet.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)
+            .build("AtomixConsistentMapOperations");
+
+    /**
+     * Abstract map command.
+     */
+    @SuppressWarnings("serial")
+    public abstract static class MapOperation {
+        @Override
+        public String toString() {
+            return MoreObjects.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 MoreObjects.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 = checkNotNull(value, "value cannot be null");
+        }
+
+        /**
+         * Returns the value.
+         * @return 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() {
+        }
+
+        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);
+        }
+    }
+
+    /**
+     * 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 MoreObjects.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 MoreObjects.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 MoreObjects.toStringHelper(getClass())
+                    .add("transactionId", transactionId)
+                    .toString();
+        }
+    }
+
+    /**
+     * Map update command.
+     */
+    @SuppressWarnings("serial")
+    public static class UpdateAndGet extends MapOperation {
+        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;
+        }
+
+        /**
+         * Returns the key.
+         * @return key
+         */
+        public String key() {
+            return this.key;
+        }
+
+        /**
+         * Returns the value.
+         * @return value
+         */
+        public byte[] value() {
+            return this.value;
+        }
+
+        /**
+         * Returns the value match.
+         * @return value match
+         */
+        public Match<byte[]> valueMatch() {
+            return this.valueMatch;
+        }
+
+        /**
+         * Returns the version match.
+         * @return version match
+         */
+        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;
+        }
+    }
+}
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
new file mode 100644
index 0000000..6d8c1b0
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapService.java
@@ -0,0 +1,743 @@
+/*
+ * Copyright 2016-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.store.primitives.resources.impl;
+
+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.stream.Collectors;
+
+import com.google.common.base.Throwables;
+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.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.primitives.resources.impl.AtomixConsistentMapOperations.UpdateAndGet;
+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.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.KEY_SET;
+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.REMOVE_LISTENER;
+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.UPDATE_AND_GET;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMapOperations.VALUES;
+import static org.onosproject.store.service.MapEvent.Type.INSERT;
+import static org.onosproject.store.service.MapEvent.Type.REMOVE;
+import static org.onosproject.store.service.MapEvent.Type.UPDATE;
+
+/**
+ * State Machine for {@link AtomixConsistentMap} resource.
+ */
+public class AtomixConsistentMapService extends AbstractRaftService {
+
+    private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
+            .register(KryoNamespaces.BASIC)
+            .register(AtomixConsistentMapOperations.NAMESPACE)
+            .register(AtomixConsistentMapEvents.NAMESPACE)
+            .nextId(KryoNamespace.FLOATING_ID)
+            .register(TransactionScope.class)
+            .register(TransactionLog.class)
+            .register(TransactionId.class)
+            .register(MapEntryValue.class)
+            .register(MapEntryValue.Type.class)
+            .register(new HashMap().keySet().getClass())
+            .build());
+
+    private Map<Long, RaftSession> listeners = new LinkedHashMap<>();
+    private Map<String, MapEntryValue> mapEntries = new HashMap<>();
+    private Set<String> preparedKeys = Sets.newHashSet();
+    private Map<TransactionId, TransactionScope> activeTransactions = Maps.newHashMap();
+    private long currentVersion;
+
+    @Override
+    public void snapshot(SnapshotWriter writer) {
+        writer.writeObject(Sets.newHashSet(listeners.keySet()), SERIALIZER::encode);
+        writer.writeObject(preparedKeys, SERIALIZER::encode);
+        writer.writeObject(mapEntries, SERIALIZER::encode);
+        writer.writeObject(activeTransactions, SERIALIZER::encode);
+        writer.writeLong(currentVersion);
+    }
+
+    @Override
+    public void install(SnapshotReader reader) {
+        listeners = new LinkedHashMap<>();
+        for (Long sessionId : reader.<Set<Long>>readObject(SERIALIZER::decode)) {
+            listeners.put(sessionId, getSessions().getSession(sessionId));
+        }
+        preparedKeys = reader.readObject(SERIALIZER::decode);
+        mapEntries = reader.readObject(SERIALIZER::decode);
+        activeTransactions = reader.readObject(SERIALIZER::decode);
+        currentVersion = reader.readLong();
+    }
+
+    @Override
+    protected void configure(RaftServiceExecutor executor) {
+        // Listeners
+        executor.register(ADD_LISTENER, this::listen);
+        executor.register(REMOVE_LISTENER, this::unlisten);
+        // 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, this::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, this::isEmpty, SERIALIZER::encode);
+        executor.register(KEY_SET, this::keySet, SERIALIZER::encode);
+        executor.register(SIZE, this::size, SERIALIZER::encode);
+        executor.register(VALUES, this::values, SERIALIZER::encode);
+        // Commands
+        executor.register(UPDATE_AND_GET, SERIALIZER::decode, this::updateAndGet, SERIALIZER::encode);
+        executor.register(CLEAR, this::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);
+    }
+
+    /**
+     * 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 = mapEntries.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 mapEntries.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(mapEntries.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 = mapEntries.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 count commit.
+     *
+     * @param commit size commit
+     * @return number of entries in map
+     */
+    protected int size(Commit<Void> commit) {
+        return (int) mapEntries.values().stream()
+                .filter(value -> value.type() != MapEntryValue.Type.TOMBSTONE)
+                .count();
+    }
+
+    /**
+     * Handles an is empty commit.
+     *
+     * @param commit isEmpty commit
+     * @return {@code true} if map is empty
+     */
+    protected boolean isEmpty(Commit<Void> commit) {
+        return mapEntries.values().stream()
+                .noneMatch(value -> value.type() != MapEntryValue.Type.TOMBSTONE);
+    }
+
+    /**
+     * Handles a keySet commit.
+     *
+     * @param commit keySet commit
+     * @return set of keys in map
+     */
+    protected Set<String> keySet(Commit<Void> commit) {
+        return mapEntries.entrySet().stream()
+                .filter(entry -> entry.getValue().type() != MapEntryValue.Type.TOMBSTONE)
+                .map(Map.Entry::getKey)
+                .collect(Collectors.toSet());
+    }
+
+    /**
+     * Handles a values commit.
+     *
+     * @param commit values commit
+     * @return collection of values in map
+     */
+    protected Collection<Versioned<byte[]>> values(Commit<Void> commit) {
+        return mapEntries.entrySet().stream()
+                .filter(entry -> entry.getValue().type() != MapEntryValue.Type.TOMBSTONE)
+                .map(entry -> toVersioned(entry.getValue()))
+                .collect(Collectors.toList());
+    }
+
+    /**
+     * Handles a entry set commit.
+     *
+     * @param commit entrySet commit
+     * @return set of map entries
+     */
+    protected Set<Map.Entry<String, Versioned<byte[]>>> entrySet(Commit<Void> commit) {
+        return mapEntries.entrySet().stream()
+                .filter(entry -> entry.getValue().type() != MapEntryValue.Type.TOMBSTONE)
+                .map(e -> Maps.immutableEntry(e.getKey(), toVersioned(e.getValue())))
+                .collect(Collectors.toSet());
+    }
+
+    /**
+     * Handles a update and get commit.
+     *
+     * @param commit updateAndGet commit
+     * @return update result
+     */
+    protected MapEntryUpdateResult<String, byte[]> updateAndGet(Commit<? extends UpdateAndGet> commit) {
+        try {
+            MapEntryUpdateResult.Status updateStatus = validate(commit.value());
+            String key = commit.value().key();
+            MapEntryValue oldCommitValue = mapEntries.get(commit.value().key());
+            Versioned<byte[]> oldMapValue = toVersioned(oldCommitValue);
+
+            if (updateStatus != MapEntryUpdateResult.Status.OK) {
+                return new MapEntryUpdateResult<>(updateStatus, "", key, oldMapValue, oldMapValue);
+            }
+
+            byte[] newValue = commit.value().value();
+            currentVersion = commit.index();
+            Versioned<byte[]> newMapValue = newValue == null ? null
+                    : new Versioned<>(newValue, currentVersion);
+
+            MapEvent.Type updateType = newValue == null ? REMOVE
+                    : oldCommitValue == null ? INSERT : UPDATE;
+
+            // If a value existed in the map, remove and discard the value to ensure disk can be freed.
+            if (updateType == REMOVE || updateType == UPDATE) {
+                mapEntries.remove(key);
+            }
+
+            // If this is an insert/update commit, add the commit to the map entries.
+            if (updateType == INSERT || updateType == UPDATE) {
+                mapEntries.put(key, new MapEntryValue(
+                        MapEntryValue.Type.VALUE,
+                        commit.index(),
+                        commit.value().value()));
+            } else if (!activeTransactions.isEmpty()) {
+                // If this is a delete but transactions are currently running, ensure tombstones are retained
+                // for version checks.
+                mapEntries.put(key, new MapEntryValue(MapEntryValue.Type.TOMBSTONE, commit.index(), null));
+            }
+
+            publish(Lists.newArrayList(new MapEvent<>("", key, newMapValue, oldMapValue)));
+            return new MapEntryUpdateResult<>(updateStatus, "", key, oldMapValue, newMapValue);
+        } catch (Exception e) {
+            getLogger().error("State machine operation failed", e);
+            throw Throwables.propagate(e);
+        }
+    }
+
+    /**
+     * Handles a clear commit.
+     *
+     * @param commit clear commit
+     * @return clear result
+     */
+    protected MapEntryUpdateResult.Status clear(Commit<Void> commit) {
+        Iterator<Map.Entry<String, MapEntryValue>> iterator = mapEntries
+                .entrySet().iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<String, MapEntryValue> entry = iterator.next();
+            String key = entry.getKey();
+            MapEntryValue value = entry.getValue();
+            Versioned<byte[]> removedValue = new Versioned<>(value.value(),
+                    value.version());
+            publish(Lists.newArrayList(new MapEvent<>("", key, null, removedValue)));
+            iterator.remove();
+        }
+        return MapEntryUpdateResult.Status.OK;
+    }
+
+    /**
+     * 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 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 = mapEntries.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) {
+            getLogger().warn("Failure applying {}", commit, e);
+            throw Throwables.propagate(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) {
+            getLogger().warn("Failure applying {}", commit, e);
+            throw Throwables.propagate(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 = mapEntries.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);
+            }
+
+            eventsToPublish.add(new MapEvent<>("", key, toVersioned(newValue), toVersioned(previousValue)));
+
+            if (newValue != null) {
+                mapEntries.put(key, newValue);
+            }
+        }
+        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 = mapEntries.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 = mapEntries.entrySet().iterator();
+            while (iterator.hasNext()) {
+                MapEntryValue value = iterator.next().getValue();
+                if (value.type() == MapEntryValue.Type.TOMBSTONE && value.version < lowWaterMark) {
+                    iterator.remove();
+                }
+            }
+        }
+    }
+
+    /**
+     * Computes the update status that would result if the specified update were to applied to
+     * the state machine.
+     *
+     * @param update update
+     * @return status
+     */
+    private MapEntryUpdateResult.Status validate(UpdateAndGet update) {
+        MapEntryValue existingValue = mapEntries.get(update.key());
+        boolean isEmpty = existingValue == null || existingValue.type() == MapEntryValue.Type.TOMBSTONE;
+        if (isEmpty && update.value() == null) {
+            return MapEntryUpdateResult.Status.NOOP;
+        }
+        if (preparedKeys.contains(update.key())) {
+            return MapEntryUpdateResult.Status.WRITE_LOCK;
+        }
+        byte[] existingRawValue = isEmpty ? null : existingValue.value();
+        Long existingVersion = isEmpty ? null : existingValue.version();
+        return update.valueMatch().matches(existingRawValue)
+                && update.versionMatch().matches(existingVersion) ? MapEntryUpdateResult.Status.OK
+                : MapEntryUpdateResult.Status.PRECONDITION_FAILED;
+    }
+
+    /**
+     * Utility for turning a {@code MapEntryValue} to {@code Versioned}.
+     * @param value map entry value
+     * @return versioned instance
+     */
+    private Versioned<byte[]> toVersioned(MapEntryValue value) {
+        return value != null && value.type() != MapEntryValue.Type.TOMBSTONE
+                ? new Versioned<>(value.value(), value.version()) : null;
+    }
+
+    /**
+     * 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.
+     */
+    private 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.
+     */
+    private 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());
+        }
+    }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapState.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapState.java
deleted file mode 100644
index ea62e72..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMapState.java
+++ /dev/null
@@ -1,946 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Laboratory
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.onosproject.store.primitives.resources.impl;
-
-import static com.google.common.base.Preconditions.checkState;
-import static org.onosproject.store.service.MapEvent.Type.INSERT;
-import static org.onosproject.store.service.MapEvent.Type.REMOVE;
-import static org.onosproject.store.service.MapEvent.Type.UPDATE;
-import static org.slf4j.LoggerFactory.getLogger;
-import io.atomix.copycat.server.Commit;
-import io.atomix.copycat.server.Snapshottable;
-import io.atomix.copycat.server.StateMachineExecutor;
-import io.atomix.copycat.server.session.ServerSession;
-import io.atomix.copycat.server.session.SessionListener;
-import io.atomix.copycat.server.storage.snapshot.SnapshotReader;
-import io.atomix.copycat.server.storage.snapshot.SnapshotWriter;
-import io.atomix.resource.ResourceStateMachine;
-
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.stream.Collectors;
-
-import org.onlab.util.CountDownCompleter;
-import org.onlab.util.Match;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.Clear;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.ContainsKey;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.ContainsValue;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.EntrySet;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.Get;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.GetOrDefault;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.IsEmpty;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.KeySet;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.Listen;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.Size;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.TransactionBegin;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.TransactionCommit;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.TransactionPrepare;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.TransactionPrepareAndCommit;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.TransactionRollback;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.Unlisten;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.UpdateAndGet;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentMapCommands.Values;
-import org.onosproject.store.service.MapEvent;
-import org.onosproject.store.service.TransactionLog;
-import org.onosproject.store.service.Versioned;
-import org.slf4j.Logger;
-
-import com.google.common.base.Throwables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-
-/**
- * State Machine for {@link AtomixConsistentMap} resource.
- */
-public class AtomixConsistentMapState extends ResourceStateMachine implements SessionListener, Snapshottable {
-
-    private final Logger log = getLogger(getClass());
-    private final Map<Long, Commit<? extends Listen>> listeners = new HashMap<>();
-    private final Map<String, MapEntryValue> mapEntries = new HashMap<>();
-    private final Set<String> preparedKeys = Sets.newHashSet();
-    private final Map<TransactionId, TransactionScope> activeTransactions = Maps.newHashMap();
-    private long currentVersion;
-
-    public AtomixConsistentMapState(Properties properties) {
-        super(properties);
-    }
-
-    @Override
-    public void snapshot(SnapshotWriter writer) {
-    }
-
-    @Override
-    public void install(SnapshotReader reader) {
-    }
-
-    @Override
-    protected void configure(StateMachineExecutor executor) {
-        // Listeners
-        executor.register(Listen.class, this::listen);
-        executor.register(Unlisten.class, this::unlisten);
-        // Queries
-        executor.register(ContainsKey.class, this::containsKey);
-        executor.register(ContainsValue.class, this::containsValue);
-        executor.register(EntrySet.class, this::entrySet);
-        executor.register(Get.class, this::get);
-        executor.register(GetOrDefault.class, this::getOrDefault);
-        executor.register(IsEmpty.class, this::isEmpty);
-        executor.register(KeySet.class, this::keySet);
-        executor.register(Size.class, this::size);
-        executor.register(Values.class, this::values);
-        // Commands
-        executor.register(UpdateAndGet.class, this::updateAndGet);
-        executor.register(AtomixConsistentMapCommands.Clear.class, this::clear);
-        executor.register(TransactionBegin.class, this::begin);
-        executor.register(TransactionPrepare.class, this::prepare);
-        executor.register(TransactionCommit.class, this::commit);
-        executor.register(TransactionRollback.class, this::rollback);
-        executor.register(TransactionPrepareAndCommit.class, this::prepareAndCommit);
-    }
-
-    @Override
-    public void delete() {
-        // Delete Listeners
-        listeners.values().forEach(Commit::close);
-        listeners.clear();
-
-        // Delete Map entries
-        mapEntries.values().forEach(MapEntryValue::discard);
-        mapEntries.clear();
-    }
-
-    /**
-     * Handles a contains key commit.
-     *
-     * @param commit containsKey commit
-     * @return {@code true} if map contains key
-     */
-    protected boolean containsKey(Commit<? extends ContainsKey> commit) {
-        try {
-            MapEntryValue value = mapEntries.get(commit.operation().key());
-            return value != null && value.type() != MapEntryValue.Type.TOMBSTONE;
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles a contains value commit.
-     *
-     * @param commit containsValue commit
-     * @return {@code true} if map contains value
-     */
-    protected boolean containsValue(Commit<? extends ContainsValue> commit) {
-        try {
-            Match<byte[]> valueMatch = Match.ifValue(commit.operation().value());
-            return mapEntries.values().stream()
-                    .filter(value -> value.type() != MapEntryValue.Type.TOMBSTONE)
-                    .anyMatch(value -> valueMatch.matches(value.value()));
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles a get commit.
-     *
-     * @param commit get commit
-     * @return value mapped to key
-     */
-    protected Versioned<byte[]> get(Commit<? extends Get> commit) {
-        try {
-            return toVersioned(mapEntries.get(commit.operation().key()));
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * 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) {
-        try {
-            MapEntryValue value = mapEntries.get(commit.operation().key());
-            if (value == null) {
-                return new Versioned<>(commit.operation().defaultValue(), 0);
-            } else if (value.type() == MapEntryValue.Type.TOMBSTONE) {
-                return new Versioned<>(commit.operation().defaultValue(), value.version);
-            } else {
-                return new Versioned<>(value.value(), value.version);
-            }
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles a count commit.
-     *
-     * @param commit size commit
-     * @return number of entries in map
-     */
-    protected int size(Commit<? extends Size> commit) {
-        try {
-            return (int) mapEntries.values().stream()
-                    .filter(value -> value.type() != MapEntryValue.Type.TOMBSTONE)
-                    .count();
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles an is empty commit.
-     *
-     * @param commit isEmpty commit
-     * @return {@code true} if map is empty
-     */
-    protected boolean isEmpty(Commit<? extends IsEmpty> commit) {
-        try {
-            return mapEntries.values().stream()
-                    .noneMatch(value -> value.type() != MapEntryValue.Type.TOMBSTONE);
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles a keySet commit.
-     *
-     * @param commit keySet commit
-     * @return set of keys in map
-     */
-    protected Set<String> keySet(Commit<? extends KeySet> commit) {
-        try {
-            return mapEntries.entrySet().stream()
-                    .filter(entry -> entry.getValue().type() != MapEntryValue.Type.TOMBSTONE)
-                    .map(Map.Entry::getKey)
-                    .collect(Collectors.toSet());
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles a values commit.
-     *
-     * @param commit values commit
-     * @return collection of values in map
-     */
-    protected Collection<Versioned<byte[]>> values(Commit<? extends Values> commit) {
-        try {
-            return mapEntries.entrySet().stream()
-                    .filter(entry -> entry.getValue().type() != MapEntryValue.Type.TOMBSTONE)
-                    .map(entry -> toVersioned(entry.getValue()))
-                    .collect(Collectors.toList());
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles a entry set commit.
-     *
-     * @param commit
-     *            entrySet commit
-     * @return set of map entries
-     */
-    protected Set<Map.Entry<String, Versioned<byte[]>>> entrySet(Commit<? extends EntrySet> commit) {
-        try {
-            return mapEntries.entrySet().stream()
-                    .filter(entry -> entry.getValue().type() != MapEntryValue.Type.TOMBSTONE)
-                    .map(e -> Maps.immutableEntry(e.getKey(), toVersioned(e.getValue())))
-                    .collect(Collectors.toSet());
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles a update and get commit.
-     *
-     * @param commit updateAndGet commit
-     * @return update result
-     */
-    protected MapEntryUpdateResult<String, byte[]> updateAndGet(Commit<? extends UpdateAndGet> commit) {
-        try {
-            MapEntryUpdateResult.Status updateStatus = validate(commit.operation());
-            String key = commit.operation().key();
-            MapEntryValue oldCommitValue = mapEntries.get(commit.operation().key());
-            Versioned<byte[]> oldMapValue = toVersioned(oldCommitValue);
-
-            if (updateStatus != MapEntryUpdateResult.Status.OK) {
-                commit.close();
-                return new MapEntryUpdateResult<>(updateStatus, "", key, oldMapValue, oldMapValue);
-            }
-
-            byte[] newValue = commit.operation().value();
-            currentVersion = commit.index();
-            Versioned<byte[]> newMapValue = newValue == null ? null
-                    : new Versioned<>(newValue, currentVersion);
-
-            MapEvent.Type updateType = newValue == null ? REMOVE
-                    : oldCommitValue == null ? INSERT : UPDATE;
-
-            // If a value existed in the map, remove and discard the value to ensure disk can be freed.
-            if (updateType == REMOVE || updateType == UPDATE) {
-                mapEntries.remove(key);
-                oldCommitValue.discard();
-            }
-
-            // If this is an insert/update commit, add the commit to the map entries.
-            if (updateType == INSERT || updateType == UPDATE) {
-                mapEntries.put(key, new NonTransactionalCommit(commit));
-            } else if (!activeTransactions.isEmpty()) {
-                // If this is a delete but transactions are currently running, ensure tombstones are retained
-                // for version checks.
-                TombstoneCommit tombstone = new TombstoneCommit(
-                        commit.index(),
-                        new CountDownCompleter<>(commit, 1, Commit::close));
-                mapEntries.put(key, tombstone);
-            } else {
-                // If no transactions are in progress, we can safely delete the key from memory.
-                commit.close();
-            }
-
-            publish(Lists.newArrayList(new MapEvent<>("", key, newMapValue, oldMapValue)));
-            return new MapEntryUpdateResult<>(updateStatus, "", key, oldMapValue, newMapValue);
-        } catch (Exception e) {
-            log.error("State machine operation failed", e);
-            throw Throwables.propagate(e);
-        }
-    }
-
-    /**
-     * Handles a clear commit.
-     *
-     * @param commit clear commit
-     * @return clear result
-     */
-    protected MapEntryUpdateResult.Status clear(Commit<? extends Clear> commit) {
-        try {
-            Iterator<Map.Entry<String, MapEntryValue>> iterator = mapEntries
-                    .entrySet().iterator();
-            while (iterator.hasNext()) {
-                Map.Entry<String, MapEntryValue> entry = iterator.next();
-                String key = entry.getKey();
-                MapEntryValue value = entry.getValue();
-                Versioned<byte[]> removedValue = new Versioned<>(value.value(),
-                        value.version());
-                publish(Lists.newArrayList(new MapEvent<>("", key, null, removedValue)));
-                value.discard();
-                iterator.remove();
-            }
-            return MapEntryUpdateResult.Status.OK;
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles a listen commit.
-     *
-     * @param commit listen commit
-     */
-    protected void listen(Commit<? extends Listen> commit) {
-        Long sessionId = commit.session().id();
-        if (listeners.putIfAbsent(sessionId, commit) != null) {
-            commit.close();
-            return;
-        }
-        commit.session()
-                .onStateChange(
-                        state -> {
-                            if (state == ServerSession.State.CLOSED
-                                    || state == ServerSession.State.EXPIRED) {
-                                Commit<? extends Listen> listener = listeners.remove(sessionId);
-                                if (listener != null) {
-                                    listener.close();
-                                }
-                            }
-                        });
-    }
-
-    /**
-     * Handles an unlisten commit.
-     *
-     * @param commit unlisten commit
-     */
-    protected void unlisten(Commit<? extends Unlisten> commit) {
-        try {
-            Commit<? extends Listen> listener = listeners.remove(commit.session().id());
-            if (listener != null) {
-                listener.close();
-            }
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles a begin commit.
-     *
-     * @param commit transaction begin commit
-     * @return transaction state version
-     */
-    protected long begin(Commit<? extends TransactionBegin> commit) {
-        try {
-            long version = commit.index();
-            activeTransactions.put(commit.operation().transactionId(), new TransactionScope(version));
-            return version;
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * 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.operation().transactionLog().transactionId();
-        PrepareResult prepareResult = prepare(commit);
-        TransactionScope transactionScope = activeTransactions.remove(transactionId);
-        if (prepareResult == PrepareResult.OK) {
-            this.currentVersion = commit.index();
-            transactionScope = transactionScope.prepared(commit);
-            commit(transactionScope);
-        } else if (transactionScope != null) {
-            transactionScope.close();
-        }
-        discardTombstones();
-        return prepareResult;
-    }
-
-    /**
-     * Handles an prepare commit.
-     *
-     * @param commit transaction prepare commit
-     * @return prepare result
-     */
-    protected PrepareResult prepare(Commit<? extends TransactionPrepare> commit) {
-        boolean ok = false;
-
-        try {
-            TransactionLog<MapUpdate<String, byte[]>> transactionLog = commit.operation().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 = mapEntries.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());
-                }
-            });
-
-            ok = true;
-
-            // 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));
-                return PrepareResult.PARTIAL_FAILURE;
-            } else {
-                activeTransactions.put(
-                        transactionLog.transactionId(),
-                        transactionScope.prepared(commit));
-                return PrepareResult.OK;
-            }
-        } catch (Exception e) {
-            log.warn("Failure applying {}", commit, e);
-            throw Throwables.propagate(e);
-        } finally {
-            if (!ok) {
-                commit.close();
-            }
-        }
-    }
-
-    /**
-     * Handles an commit commit (ha!).
-     *
-     * @param commit transaction commit commit
-     * @return commit result
-     */
-    protected CommitResult commit(Commit<? extends TransactionCommit> commit) {
-        TransactionId transactionId = commit.operation().transactionId();
-        TransactionScope transactionScope = activeTransactions.remove(transactionId);
-        if (transactionScope == null) {
-            return CommitResult.UNKNOWN_TRANSACTION_ID;
-        }
-
-        try {
-            this.currentVersion = commit.index();
-            return commit(transactionScope.committed(commit));
-        } catch (Exception e) {
-            log.warn("Failure applying {}", commit, e);
-            throw Throwables.propagate(e);
-        } finally {
-            discardTombstones();
-        }
-    }
-
-    /**
-     * Applies committed operations to the state machine.
-     */
-    private CommitResult commit(TransactionScope transactionScope) {
-        TransactionLog<MapUpdate<String, byte[]>> transactionLog = transactionScope.transactionLog();
-        boolean retainTombstones = !activeTransactions.isEmpty();
-
-        // Count the total number of keys that will be set by this transaction. This is necessary to do reference
-        // counting for garbage collection.
-        long totalReferencesToCommit = transactionLog.records().stream()
-                // No keys are set for version checks. For deletes, references are only retained of tombstones
-                // need to be retained for concurrent transactions.
-                .filter(record -> record.type() != MapUpdate.Type.VERSION_MATCH && record.type() != MapUpdate.Type.LOCK
-                        && (record.type() != MapUpdate.Type.REMOVE_IF_VERSION_MATCH || retainTombstones))
-                .count();
-
-        // Create a count down completer that counts references to the transaction commit for garbage collection.
-        CountDownCompleter<TransactionScope> completer = new CountDownCompleter<>(
-                transactionScope, totalReferencesToCommit, TransactionScope::close);
-
-        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 = mapEntries.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 TransactionalCommit(currentVersion, record.value(), completer);
-            } else if (retainTombstones) {
-                // For deletes, if tombstones need to be retained then create and store a tombstone commit.
-                newValue = new TombstoneCommit(currentVersion, completer);
-            }
-
-            eventsToPublish.add(new MapEvent<>("", key, toVersioned(newValue), toVersioned(previousValue)));
-
-            if (newValue != null) {
-                mapEntries.put(key, newValue);
-            }
-
-            if (previousValue != null) {
-                previousValue.discard();
-            }
-        }
-        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.operation().transactionId();
-        TransactionScope transactionScope = activeTransactions.remove(transactionId);
-        if (transactionScope == null) {
-            return RollbackResult.UNKNOWN_TRANSACTION_ID;
-        } else if (!transactionScope.isPrepared()) {
-            discardTombstones();
-            transactionScope.close();
-            commit.close();
-            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();
-                transactionScope.close();
-                commit.close();
-            }
-        }
-
-    }
-
-    /**
-     * Discards tombstones no longer needed by active transactions.
-     */
-    private void discardTombstones() {
-        if (activeTransactions.isEmpty()) {
-            Iterator<Map.Entry<String, MapEntryValue>> iterator = mapEntries.entrySet().iterator();
-            while (iterator.hasNext()) {
-                MapEntryValue value = iterator.next().getValue();
-                if (value.type() == MapEntryValue.Type.TOMBSTONE) {
-                    iterator.remove();
-                    value.discard();
-                }
-            }
-        } else {
-            long lowWaterMark = activeTransactions.values().stream()
-                    .mapToLong(TransactionScope::version)
-                    .min().getAsLong();
-            Iterator<Map.Entry<String, MapEntryValue>> iterator = mapEntries.entrySet().iterator();
-            while (iterator.hasNext()) {
-                MapEntryValue value = iterator.next().getValue();
-                if (value.type() == MapEntryValue.Type.TOMBSTONE && value.version < lowWaterMark) {
-                    iterator.remove();
-                    value.discard();
-                }
-            }
-        }
-    }
-
-    /**
-     * Computes the update status that would result if the specified update were to applied to
-     * the state machine.
-     *
-     * @param update update
-     * @return status
-     */
-    private MapEntryUpdateResult.Status validate(UpdateAndGet update) {
-        MapEntryValue existingValue = mapEntries.get(update.key());
-        boolean isEmpty = existingValue == null || existingValue.type() == MapEntryValue.Type.TOMBSTONE;
-        if (isEmpty && update.value() == null) {
-            return MapEntryUpdateResult.Status.NOOP;
-        }
-        if (preparedKeys.contains(update.key())) {
-            return MapEntryUpdateResult.Status.WRITE_LOCK;
-        }
-        byte[] existingRawValue = isEmpty ? null : existingValue.value();
-        Long existingVersion = isEmpty ? null : existingValue.version();
-        return update.valueMatch().matches(existingRawValue)
-                && update.versionMatch().matches(existingVersion) ? MapEntryUpdateResult.Status.OK
-                : MapEntryUpdateResult.Status.PRECONDITION_FAILED;
-    }
-
-    /**
-     * Utility for turning a {@code MapEntryValue} to {@code Versioned}.
-     * @param value map entry value
-     * @return versioned instance
-     */
-    private Versioned<byte[]> toVersioned(MapEntryValue value) {
-        return value != null && value.type() != MapEntryValue.Type.TOMBSTONE
-                ? new Versioned<>(value.value(), value.version()) : null;
-    }
-
-    /**
-     * Publishes events to listeners.
-     *
-     * @param events list of map event to publish
-     */
-    private void publish(List<MapEvent<String, byte[]>> events) {
-        listeners.values().forEach(commit -> commit.session().publish(AtomixConsistentMap.CHANGE_SUBJECT, events));
-    }
-
-    @Override
-    public void register(ServerSession session) {
-    }
-
-    @Override
-    public void unregister(ServerSession session) {
-        closeListener(session.id());
-    }
-
-    @Override
-    public void expire(ServerSession session) {
-        closeListener(session.id());
-    }
-
-    @Override
-    public void close(ServerSession session) {
-        closeListener(session.id());
-    }
-
-    private void closeListener(Long sessionId) {
-        Commit<? extends Listen> commit = listeners.remove(sessionId);
-        if (commit != null) {
-            commit.close();
-        }
-    }
-
-    /**
-     * Interface implemented by map values.
-     */
-    private abstract static class MapEntryValue {
-        protected final Type type;
-        protected final long version;
-
-        MapEntryValue(Type type, long version) {
-            this.type = type;
-            this.version = version;
-        }
-
-        /**
-         * 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
-         */
-        abstract byte[] value();
-
-        /**
-         * Discards the value by invoke appropriate clean up actions.
-         */
-        abstract void discard();
-
-        /**
-         * Value type.
-         */
-        enum Type {
-            VALUE,
-            TOMBSTONE,
-        }
-    }
-
-    /**
-     * A {@code MapEntryValue} that is derived from a non-transactional update
-     * i.e. via any standard map update operation.
-     */
-    private static class NonTransactionalCommit extends MapEntryValue {
-        private final Commit<? extends UpdateAndGet> commit;
-
-        NonTransactionalCommit(Commit<? extends UpdateAndGet> commit) {
-            super(Type.VALUE, commit.index());
-            this.commit = commit;
-        }
-
-        @Override
-        byte[] value() {
-            return commit.operation().value();
-        }
-
-        @Override
-        void discard() {
-            commit.close();
-        }
-    }
-
-    /**
-     * A {@code MapEntryValue} that is derived from updates submitted via a
-     * transaction.
-     */
-    private static class TransactionalCommit extends MapEntryValue {
-        private final byte[] value;
-        private final CountDownCompleter<?> completer;
-
-        TransactionalCommit(long version, byte[] value, CountDownCompleter<?> completer) {
-            super(Type.VALUE, version);
-            this.value = value;
-            this.completer = completer;
-        }
-
-        @Override
-        byte[] value() {
-            return value;
-        }
-
-        @Override
-        void discard() {
-            completer.countDown();
-        }
-    }
-
-    /**
-     * A {@code MapEntryValue} that represents a deleted entry.
-     */
-    private static class TombstoneCommit extends MapEntryValue {
-        private final CountDownCompleter<?> completer;
-
-        public TombstoneCommit(long version, CountDownCompleter<?> completer) {
-            super(Type.TOMBSTONE, version);
-            this.completer = completer;
-        }
-
-        @Override
-        byte[] value() {
-            throw new UnsupportedOperationException();
-        }
-
-        @Override
-        void discard() {
-            completer.countDown();
-        }
-    }
-
-    /**
-     * Map transaction scope.
-     */
-    private static final class TransactionScope {
-        private final long version;
-        private final Commit<? extends TransactionPrepare> prepareCommit;
-        private final Commit<? extends TransactionCommit> commitCommit;
-
-        private TransactionScope(long version) {
-            this(version, null, null);
-        }
-
-        private TransactionScope(
-                long version,
-                Commit<? extends TransactionPrepare> prepareCommit) {
-            this(version, prepareCommit, null);
-        }
-
-        private TransactionScope(
-                long version,
-                Commit<? extends TransactionPrepare> prepareCommit,
-                Commit<? extends TransactionCommit> commitCommit) {
-            this.version = version;
-            this.prepareCommit = prepareCommit;
-            this.commitCommit = commitCommit;
-        }
-
-        /**
-         * 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 prepareCommit != null;
-        }
-
-        /**
-         * Returns the transaction commit log.
-         *
-         * @return the transaction commit log
-         */
-        TransactionLog<MapUpdate<String, byte[]>> transactionLog() {
-            checkState(isPrepared());
-            return prepareCommit.operation().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);
-        }
-
-        /**
-         * Returns a new transaction scope with a commit commit.
-         *
-         * @param commit the commit commit ;-)
-         * @return new transaction scope updated with the commit commit
-         */
-        TransactionScope committed(Commit<? extends TransactionCommit> commit) {
-            checkState(isPrepared());
-            return new TransactionScope(version, prepareCommit, commit);
-        }
-
-        /**
-         * Closes the transaction and all associated commits.
-         */
-        void close() {
-            if (prepareCommit != null) {
-                prepareCommit.close();
-            }
-            if (commitCommit != null) {
-                commitCommit.close();
-            }
-        }
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMultimapCommands.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMultimapCommands.java
deleted file mode 100644
index b8e5d4c..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentMultimapCommands.java
+++ /dev/null
@@ -1,629 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Laboratory
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.onosproject.store.primitives.resources.impl;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.collect.Multiset;
-import io.atomix.catalyst.buffer.BufferInput;
-import io.atomix.catalyst.buffer.BufferOutput;
-import io.atomix.catalyst.serializer.CatalystSerializable;
-import io.atomix.catalyst.serializer.SerializableTypeResolver;
-import io.atomix.catalyst.serializer.Serializer;
-import io.atomix.catalyst.serializer.SerializerRegistry;
-import io.atomix.catalyst.util.Assert;
-import io.atomix.copycat.Command;
-import io.atomix.copycat.Query;
-import org.onlab.util.Match;
-import org.onosproject.store.service.Versioned;
-
-import java.util.Collection;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * AsyncConsistentMultimap state machine commands.
- */
-public final class AtomixConsistentMultimapCommands {
-
-    private AtomixConsistentMultimapCommands() {
-    }
-
-    /**
-     * Abstract multimap command.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class MultimapCommand<V> implements Command<V>,
-            CatalystSerializable {
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .toString();
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer,
-                                Serializer serializer) {
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-        }
-    }
-
-    /**
-     * Abstract multimap query.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class MultimapQuery<V> implements Query<V>,
-            CatalystSerializable {
-        @Override
-        public ConsistencyLevel consistency() {
-            return ConsistencyLevel.SEQUENTIAL;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .toString();
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer,
-                                Serializer serializer) {
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer,
-                               Serializer serializer) {
-        }
-    }
-
-    /**
-     * Abstract key-based multimap query.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class KeyQuery<V> extends MultimapQuery<V> {
-        protected String key;
-
-        public KeyQuery() {
-        }
-
-        public KeyQuery(String key) {
-            this.key = Assert.notNull(key, "key");
-        }
-
-        public String key() {
-            return key;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("key", key)
-                    .toString();
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer,
-                                Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            serializer.writeObject(key, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            key = serializer.readObject(buffer);
-        }
-    }
-
-    /**
-     * Abstract value-based query.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class ValueQuery<V> extends MultimapQuery<V> {
-        protected byte[] value;
-
-        public ValueQuery() {
-        }
-
-        public ValueQuery(byte[] value) {
-            this.value = Assert.notNull(value, "value");
-        }
-
-        /**
-         * Returns the value.
-         *
-         * @return value.
-         */
-        public byte[] value() {
-            return value;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("value", value)
-                    .toString();
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer,
-                                Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            serializer.writeObject(value, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            value = serializer.readObject(buffer);
-        }
-    }
-
-    /**
-     * Size query.
-     */
-    public static class Size extends MultimapQuery<Integer> {
-    }
-
-    /**
-     * Is empty query.
-     */
-    public static class IsEmpty extends MultimapQuery<Boolean> {
-    }
-
-    /**
-     * Contains key query.
-     */
-    @SuppressWarnings("serial")
-    public static class ContainsKey extends KeyQuery<Boolean> {
-        public ContainsKey() {
-        }
-
-        public ContainsKey(String key) {
-            super(key);
-        }
-
-    }
-
-    /**
-     * Contains value query.
-     */
-    @SuppressWarnings("serial")
-    public static class ContainsValue extends ValueQuery<Boolean> {
-        public ContainsValue() {
-        }
-
-        public ContainsValue(byte[] value) {
-            super(value);
-        }
-    }
-
-    /**
-     * Contains entry query.
-     */
-    @SuppressWarnings("serial")
-    public static class ContainsEntry extends MultimapQuery<Boolean> {
-        protected String key;
-        protected byte[] value;
-
-        public ContainsEntry() {
-        }
-
-        public ContainsEntry(String key, byte[] value) {
-            this.key = Assert.notNull(key, "key");
-            this.value = Assert.notNull(value, "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();
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer,
-                                Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            serializer.writeObject(key, buffer);
-            serializer.writeObject(value, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            key = serializer.readObject(buffer);
-            value = serializer.readObject(buffer);
-
-        }
-    }
-
-    /**
-     * Remove command, backs remove and removeAll's that return booleans.
-     */
-    @SuppressWarnings("serial")
-    public static class RemoveAll extends
-            MultimapCommand<Versioned<Collection<? extends byte[]>>> {
-        private String key;
-        private Match<Long> versionMatch;
-
-        public RemoveAll() {
-        }
-
-        public RemoveAll(String key, Match<Long> versionMatch) {
-            this.key = Assert.notNull(key, "key");
-            this.versionMatch = versionMatch;
-        }
-
-        public String key() {
-            return this.key;
-        }
-
-        public Match<Long> versionMatch() {
-            return versionMatch;
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.TOMBSTONE;
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer,
-                                Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            serializer.writeObject(key, buffer);
-            serializer.writeObject(versionMatch, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            key = serializer.readObject(buffer);
-            versionMatch = serializer.readObject(buffer);
-        }
-
-        @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
-            MultimapCommand<Boolean> {
-        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 = Assert.notNull(key, "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 CompactionMode compaction() {
-            return CompactionMode.TOMBSTONE;
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer,
-                                Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            serializer.writeObject(key, buffer);
-            serializer.writeObject(values, buffer);
-            serializer.writeObject(versionMatch, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            key = serializer.readObject(buffer);
-            values = serializer.readObject(buffer);
-            versionMatch = serializer.readObject(buffer);
-        }
-
-        @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 MultimapCommand<Boolean> {
-        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 = Assert.notNull(key, "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 CompactionMode compaction() {
-            return CompactionMode.QUORUM;
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer,
-                                Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            serializer.writeObject(key, buffer);
-            serializer.writeObject(values, buffer);
-            serializer.writeObject(versionMatch, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            key = serializer.readObject(buffer);
-            values = serializer.readObject(buffer);
-            versionMatch = serializer.readObject(buffer);
-        }
-
-        @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
-            MultimapCommand<Versioned<Collection<? extends byte[]>>> {
-        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 = Assert.notNull(key, "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 CompactionMode compaction() {
-            return CompactionMode.QUORUM;
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer,
-                                Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            serializer.writeObject(key, buffer);
-            serializer.writeObject(values, buffer);
-            serializer.writeObject(versionMatch, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            key = serializer.readObject(buffer);
-            values = serializer.readObject(buffer);
-            versionMatch = serializer.readObject(buffer);
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("key", key)
-                    .add("values", values)
-                    .add("versionMatch", versionMatch)
-                    .toString();
-        }
-    }
-
-    /**
-     * Clear multimap command.
-     */
-    @SuppressWarnings("serial")
-    public static class Clear extends MultimapCommand<Void> {
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.TOMBSTONE;
-        }
-    }
-
-    /**
-     * Key set query.
-     */
-    @SuppressWarnings("serial")
-    public static class KeySet extends MultimapQuery<Set<String>> {
-    }
-
-    /**
-     * Key multiset query.
-     */
-    @SuppressWarnings("serial")
-    public static class Keys extends MultimapQuery<Multiset<String>> {
-    }
-
-    /**
-     * Value collection query.
-     */
-    @SuppressWarnings("serial")
-    public static class Values extends MultimapQuery<Multiset<byte[]>> {
-    }
-
-    /**
-     * Entry set query.
-     */
-    @SuppressWarnings("serial")
-    public static class Entries extends
-            MultimapQuery<Collection<Map.Entry<String, byte[]>>> {
-    }
-
-    /**
-     * Get value query.
-     */
-    public static class Get extends
-            KeyQuery<Versioned<Collection<? extends byte[]>>> {
-        public Get() {
-        }
-
-        public Get(String key) {
-            super(key);
-        }
-    }
-
-    /**
-     * Change listen.
-     */
-    @SuppressWarnings("serial")
-    public static class Listen implements Command<Void>, CatalystSerializable {
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.QUORUM;
-        }
-    }
-
-    /**
-     * Change unlisten.
-     */
-    @SuppressWarnings("serial")
-    public static class Unlisten implements Command<Void>, CatalystSerializable {
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.TOMBSTONE;
-        }
-    }
-
-    /**
-     * Multimap command type resolver.
-     */
-    @SuppressWarnings("serial")
-    public static class TypeResolver implements SerializableTypeResolver {
-        @Override
-        public void resolve(SerializerRegistry registry) {
-            registry.register(ContainsKey.class, -1000);
-            registry.register(ContainsValue.class, -1001);
-            registry.register(ContainsEntry.class, -1002);
-            registry.register(Replace.class, -1003);
-            registry.register(Clear.class, -1004);
-            registry.register(KeySet.class, -1005);
-            registry.register(Keys.class, -1006);
-            registry.register(Values.class, -1007);
-            registry.register(Entries.class, -1008);
-            registry.register(Size.class, -1009);
-            registry.register(IsEmpty.class, -1010);
-            registry.register(Get.class, -1011);
-            registry.register(Put.class, -1012);
-            registry.register(RemoveAll.class, -1013);
-            registry.register(MultiRemove.class, -1014);
-            registry.register(Listen.class, -1015);
-            registry.register(Unlisten.class, -1016);
-        }
-    }
-}
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
index 22a0345..a9ea68b 100644
--- 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
@@ -16,43 +16,52 @@
 
 package org.onosproject.store.primitives.resources.impl;
 
-import com.google.common.collect.Lists;
-import com.google.common.collect.Multiset;
-import io.atomix.copycat.client.CopycatClient;
-import io.atomix.resource.AbstractResource;
-import io.atomix.resource.ResourceTypeInfo;
-import org.onosproject.store.service.AsyncConsistentMultimap;
-import org.onosproject.store.service.MultimapEvent;
-import org.onosproject.store.service.MultimapEventListener;
-import org.onosproject.store.service.Versioned;
-
 import java.util.Collection;
 import java.util.ConcurrentModificationException;
 import java.util.List;
 import java.util.Map;
-import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.ConcurrentHashMap;
 import java.util.concurrent.Executor;
 
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.Clear;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.ContainsEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.ContainsKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.ContainsValue;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.Entries;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.Get;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.IsEmpty;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.KeySet;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.Keys;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.Listen;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.MultiRemove;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.Put;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.RemoveAll;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.Replace;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.Size;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.Unlisten;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.Values;
+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.onosproject.store.serializers.KryoNamespaces;
+import org.onosproject.store.service.AsyncConsistentMultimap;
+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.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.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.PUT;
+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_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;
 
 
 /**
@@ -60,30 +69,25 @@
  * <p>
  * Note: this implementation does not allow null entries or duplicate entries.
  */
-@ResourceTypeInfo(id = -153, factory = AtomixConsistentSetMultimapFactory.class)
 public class AtomixConsistentSetMultimap
-        extends AbstractResource<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 static final String CHANGE_SUBJECT = "multimapChangeEvents";
-
-    public AtomixConsistentSetMultimap(CopycatClient client,
-                                       Properties properties) {
-        super(client, properties);
-    }
-
-    @Override
-    public CompletableFuture<AtomixConsistentSetMultimap> open() {
-        return super.open().thenApply(result -> {
-            client.onStateChange(state -> {
-                if (state == CopycatClient.State.CONNECTED && isListening()) {
-                    client.submit(new Listen());
-                }
-            });
-            client.onEvent(CHANGE_SUBJECT, this::handleEvent);
-            return result;
+    public AtomixConsistentSetMultimap(RaftProxy proxy) {
+        super(proxy);
+        proxy.addEventListener(CHANGE, SERIALIZER::decode, this::handleEvent);
+        proxy.addStateChangeListener(state -> {
+            if (state == RaftProxy.State.CONNECTED && isListening()) {
+                proxy.invoke(ADD_LISTENER);
+            }
         });
     }
 
@@ -94,97 +98,109 @@
 
     @Override
     public CompletableFuture<Integer> size() {
-        return client.submit(new Size());
+        return proxy.invoke(SIZE, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Boolean> isEmpty() {
-        return client.submit(new IsEmpty());
+        return proxy.invoke(IS_EMPTY, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Boolean> containsKey(String key) {
-        return client.submit(new ContainsKey(key));
+        return proxy.invoke(CONTAINS_KEY, SERIALIZER::encode, new ContainsKey(key), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Boolean> containsValue(byte[] value) {
-        return client.submit(new ContainsValue(value));
+        return proxy.invoke(CONTAINS_VALUE, SERIALIZER::encode, new ContainsValue(value), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Boolean> containsEntry(String key, byte[] value) {
-        return client.submit(new ContainsEntry(key, value));
+        return proxy.invoke(CONTAINS_ENTRY, SERIALIZER::encode, new ContainsEntry(key, value), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Boolean> put(String key, byte[] value) {
-        return client.submit(new Put(key, Lists.newArrayList(value), null));
+        return proxy.invoke(
+                PUT,
+                SERIALIZER::encode,
+                new Put(key, Lists.newArrayList(value), null),
+                SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Boolean> remove(String key, byte[] value) {
-        return client.submit(new MultiRemove(key,
-                                             Lists.newArrayList(value),
-                                             null));
+        return proxy.invoke(REMOVE, SERIALIZER::encode, new MultiRemove(key,
+                Lists.newArrayList(value),
+                null), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Boolean> removeAll(String key, Collection<? extends byte[]> values) {
-        return client.submit(new MultiRemove(key, (Collection<byte[]>) values, null));
+        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 client.submit(new RemoveAll(key, null));
+        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 client.submit(new Put(key, values, null));
+        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 client.submit(new Replace(key, values, null));
+        return proxy.invoke(
+                REPLACE,
+                SERIALIZER::encode,
+                new Replace(key, values, null),
+                SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Void> clear() {
-        return client.submit(new Clear());
+        return proxy.invoke(CLEAR);
     }
 
     @Override
     public CompletableFuture<Versioned<Collection<? extends byte[]>>> get(String key) {
-        return client.submit(new Get(key));
+        return proxy.invoke(GET, SERIALIZER::encode, new Get(key), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Set<String>> keySet() {
-        return client.submit(new KeySet());
+        return proxy.invoke(KEY_SET, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Multiset<String>> keys() {
-        return client.submit(new Keys());
+        return proxy.invoke(KEYS, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Multiset<byte[]>> values() {
-        return client.submit(new Values());
+        return proxy.invoke(VALUES, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Collection<Map.Entry<String, byte[]>>> entries() {
-        return client.submit(new Entries());
+        return proxy.invoke(ENTRIES, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Void> addListener(MultimapEventListener<String, byte[]> listener, Executor executor) {
         if (mapEventListeners.isEmpty()) {
-            return client.submit(new Listen()).thenRun(() -> mapEventListeners.put(listener, executor));
+            return proxy.invoke(ADD_LISTENER).thenRun(() -> mapEventListeners.put(listener, executor));
         } else {
             mapEventListeners.put(listener, executor);
             return CompletableFuture.completedFuture(null);
@@ -194,7 +210,7 @@
     @Override
     public CompletableFuture<Void> removeListener(MultimapEventListener<String, byte[]> listener) {
         if (mapEventListeners.remove(listener) != null && mapEventListeners.isEmpty()) {
-            return client.submit(new Unlisten()).thenApply(v -> null);
+            return proxy.invoke(REMOVE_LISTENER).thenApply(v -> null);
         }
         return CompletableFuture.completedFuture(null);
     }
@@ -204,11 +220,6 @@
         throw new UnsupportedOperationException("Expensive operation.");
     }
 
-    @Override
-    public String name() {
-        return null;
-    }
-
     /**
      * Helper to check if there was a lock based issue.
      * @param status the status of an update result
@@ -216,12 +227,12 @@
     private void throwIfLocked(MapEntryUpdateResult.Status status) {
         if (status == MapEntryUpdateResult.Status.WRITE_LOCK) {
             throw new ConcurrentModificationException("Cannot update map: " +
-                                                      "Another transaction " +
-                                                      "in progress");
+                    "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
new file mode 100644
index 0000000..6c71915
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapEvents.java
@@ -0,0 +1,46 @@
+/*
+ * Copyright 2017-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.store.primitives.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("change");
+
+    private final String id;
+
+    AtomixConsistentSetMultimapEvents(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(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/AtomixConsistentSetMultimapFactory.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapFactory.java
deleted file mode 100644
index e6ebbda..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapFactory.java
+++ /dev/null
@@ -1,46 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Laboratory
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.onosproject.store.primitives.resources.impl;
-
-import io.atomix.catalyst.serializer.SerializableTypeResolver;
-import io.atomix.copycat.client.CopycatClient;
-import io.atomix.resource.ResourceFactory;
-import io.atomix.resource.ResourceStateMachine;
-
-import java.util.Properties;
-
-/**
- * {@link AtomixConsistentSetMultimap} resource factory.
- */
-public class AtomixConsistentSetMultimapFactory implements
-        ResourceFactory<AtomixConsistentSetMultimap> {
-    @Override
-    public SerializableTypeResolver createSerializableTypeResolver() {
-        return new AtomixConsistentMultimapCommands.TypeResolver();
-    }
-
-    @Override
-    public ResourceStateMachine createStateMachine(Properties config) {
-        return new AtomixConsistentSetMultimapState(config);
-    }
-
-    @Override
-    public AtomixConsistentSetMultimap createInstance(CopycatClient client,
-                                                      Properties properties) {
-        return new AtomixConsistentSetMultimap(client, properties);
-    }
-}
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
new file mode 100644
index 0000000..ddbbc5b
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapOperations.java
@@ -0,0 +1,384 @@
+/*
+ * Copyright 2016-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.onosproject.store.primitives.resources.impl;
+
+import java.util.ArrayList;
+import java.util.Collection;
+
+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("get", OperationType.QUERY),
+    SIZE("size", OperationType.QUERY),
+    IS_EMPTY("isEmpty", OperationType.QUERY),
+    CONTAINS_KEY("containsKey", OperationType.QUERY),
+    CONTAINS_VALUE("containsValue", OperationType.QUERY),
+    CONTAINS_ENTRY("containsEntry", OperationType.QUERY),
+    KEY_SET("keySet", OperationType.QUERY),
+    KEYS("keys", OperationType.QUERY),
+    VALUES("values", OperationType.QUERY),
+    ENTRIES("entries", OperationType.QUERY),
+    PUT("put", OperationType.COMMAND),
+    REMOVE("remove", OperationType.COMMAND),
+    REMOVE_ALL("removeAll", OperationType.COMMAND),
+    REPLACE("replace", OperationType.COMMAND),
+    CLEAR("clear", OperationType.COMMAND),
+    ADD_LISTENER("addListener", OperationType.COMMAND),
+    REMOVE_LISTENER("removeListener", OperationType.COMMAND);
+
+    private final String id;
+    private final OperationType type;
+
+    AtomixConsistentSetMultimapOperations(String id, OperationType type) {
+        this.id = id;
+        this.type = type;
+    }
+
+    @Override
+    public String id() {
+        return id;
+    }
+
+    @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())
+            .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);
+        }
+    }
+}
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
new file mode 100644
index 0000000..3029bd8
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapService.java
@@ -0,0 +1,706 @@
+/*
+ * Copyright 2016-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.onosproject.store.primitives.resources.impl;
+
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.EnumSet;
+import java.util.HashMap;
+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.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.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.PUT;
+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_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 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.newHashMap();
+
+    @Override
+    public void snapshot(SnapshotWriter writer) {
+        writer.writeLong(globalVersion.get());
+        writer.writeObject(Sets.newHashSet(listeners.keySet()), serializer::encode);
+        writer.writeObject(backingMap, 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, getSessions().getSession(sessionId));
+        }
+
+        backingMap = reader.readObject(serializer::decode);
+    }
+
+    @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(PUT, serializer::decode, this::put, serializer::encode);
+        executor.register(REPLACE, serializer::decode, this::replace, serializer::encode);
+        executor.register(ADD_LISTENER, this::listen);
+        executor.register(REMOVE_LISTENER, this::unlisten);
+    }
+
+    @Override
+    public void onExpire(RaftSession session) {
+        listeners.remove(session.sessionId().id());
+    }
+
+    @Override
+    public void onClose(RaftSession session) {
+        listeners.remove(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 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;
+    }
+
+    protected Versioned<Collection<? extends byte[]>> replace(
+            Commit<? extends Replace> commit) {
+        if (!backingMap.containsKey(commit.value().key())) {
+            backingMap.put(commit.value().key(),
+                    new NonTransactionalCommit());
+        }
+        return backingMap.get(commit.value().key()).addCommit(commit);
+    }
+
+    /**
+     * 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());
+    }
+
+    /**
+     * 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;
+            }
+        }
+    }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapState.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapState.java
deleted file mode 100644
index 47972d1..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentSetMultimapState.java
+++ /dev/null
@@ -1,805 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Laboratory
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.onosproject.store.primitives.resources.impl;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.HashMultimap;
-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.copycat.server.Commit;
-import io.atomix.copycat.server.Snapshottable;
-import io.atomix.copycat.server.StateMachineExecutor;
-import io.atomix.copycat.server.session.ServerSession;
-import io.atomix.copycat.server.session.SessionListener;
-import io.atomix.copycat.server.storage.snapshot.SnapshotReader;
-import io.atomix.copycat.server.storage.snapshot.SnapshotWriter;
-import io.atomix.resource.ResourceStateMachine;
-import org.onlab.util.CountDownCompleter;
-import org.onlab.util.Match;
-import org.onosproject.store.service.MultimapEvent;
-import org.onosproject.store.service.Versioned;
-import org.slf4j.Logger;
-
-import java.util.Arrays;
-import java.util.Collection;
-import java.util.Comparator;
-import java.util.EnumSet;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Set;
-import java.util.TreeMap;
-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 static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.Clear;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.ContainsEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.ContainsKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.ContainsValue;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.Entries;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.Get;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.IsEmpty;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.KeySet;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.Keys;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.Listen;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.MultiRemove;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.MultimapCommand;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.Put;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.RemoveAll;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.Replace;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.Size;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.Unlisten;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentMultimapCommands.Values;
-import static org.slf4j.LoggerFactory.getLogger;
-
-/**
- * State Machine for {@link AtomixConsistentSetMultimap} resource.
- */
-public class AtomixConsistentSetMultimapState extends ResourceStateMachine
-        implements SessionListener, Snapshottable {
-
-    private final Logger log = getLogger(getClass());
-    private final AtomicLong globalVersion = new AtomicLong(1);
-    private final Map<Long, Commit<? extends Listen>> listeners = new HashMap<>();
-    private final Map<String, MapEntryValue> backingMap = Maps.newHashMap();
-
-    public AtomixConsistentSetMultimapState(Properties properties) {
-        super(properties);
-    }
-
-    @Override
-    public void snapshot(SnapshotWriter writer) {
-    }
-
-    @Override
-    public void install(SnapshotReader reader) {
-    }
-
-    @Override
-    protected void configure(StateMachineExecutor executor) {
-        executor.register(Size.class, this::size);
-        executor.register(IsEmpty.class, this::isEmpty);
-        executor.register(ContainsKey.class, this::containsKey);
-        executor.register(ContainsValue.class, this::containsValue);
-        executor.register(ContainsEntry.class, this::containsEntry);
-        executor.register(Clear.class, this::clear);
-        executor.register(KeySet.class, this::keySet);
-        executor.register(Keys.class, this::keys);
-        executor.register(Values.class, this::values);
-        executor.register(Entries.class, this::entries);
-        executor.register(Get.class, this::get);
-        executor.register(RemoveAll.class, this::removeAll);
-        executor.register(MultiRemove.class, this::multiRemove);
-        executor.register(Put.class, this::put);
-        executor.register(Replace.class, this::replace);
-        executor.register(Listen.class, this::listen);
-        executor.register(Unlisten.class, this::unlisten);
-    }
-
-    /**
-     * Handles a Size commit.
-     *
-     * @param commit Size commit
-     * @return number of unique key value pairs in the multimap
-     */
-    protected int size(Commit<? extends Size> commit) {
-        try {
-            return backingMap.values()
-                    .stream()
-                    .map(valueCollection -> valueCollection.values().size())
-                    .collect(Collectors.summingInt(size -> size));
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles an IsEmpty commit.
-     *
-     * @param commit IsEmpty commit
-     * @return true if the multimap contains no key-value pairs, else false
-     */
-    protected boolean isEmpty(Commit<? extends IsEmpty> commit) {
-        try {
-            return backingMap.isEmpty();
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * 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) {
-        try {
-            return backingMap.containsKey(commit.operation().key());
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * 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) {
-        try {
-            if (backingMap.values().isEmpty()) {
-                return false;
-            }
-            Match<byte[]> match = Match.ifValue(commit.operation().value());
-            return backingMap
-                    .values()
-                    .stream()
-                    .anyMatch(valueList ->
-                                      valueList
-                                              .values()
-                                              .stream()
-                                              .anyMatch(byteValue ->
-                                                    match.matches(byteValue)));
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * 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) {
-        try {
-            MapEntryValue entryValue =
-                    backingMap.get(commit.operation().key());
-            if (entryValue == null) {
-                return false;
-            } else {
-                Match valueMatch = Match.ifValue(commit.operation().value());
-                return entryValue
-                        .values()
-                        .stream()
-                        .anyMatch(byteValue -> valueMatch.matches(byteValue));
-            }
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles a Clear commit.
-     *
-     * @param commit Clear commit
-     */
-    protected void clear(Commit<? extends Clear> commit) {
-        try {
-            backingMap.clear();
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles a KeySet commit.
-     *
-     * @param commit KeySet commit
-     * @return a set of all keys in the multimap
-     */
-    protected Set<String> keySet(Commit<? extends KeySet> commit) {
-        try {
-            return ImmutableSet.copyOf(backingMap.keySet());
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * 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<? extends Keys> commit) {
-        try {
-            Multiset keys = HashMultiset.create();
-            backingMap.forEach((key, mapEntryValue) -> {
-                keys.add(key, mapEntryValue.values().size());
-            });
-            return keys;
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Handles a Values commit.
-     *
-     * @param commit Values commit
-     * @return the set of values in the multimap with duplicates included
-     */
-    protected Multiset<byte[]> values(Commit<? extends Values> commit) {
-        try {
-            return backingMap
-                    .values()
-                    .stream()
-                    .collect(new HashMultisetValueCollector());
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * 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<? extends Entries> commit) {
-        try {
-            return backingMap
-                    .entrySet()
-                    .stream()
-                    .collect(new EntrySetCollector());
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * 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) {
-        try {
-            MapEntryValue mapEntryValue = backingMap.get(commit.operation().key());
-            return toVersioned(backingMap.get(commit.operation().key()));
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * 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.operation().key();
-
-        if (!backingMap.containsKey(key)) {
-            commit.close();
-            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.operation().key();
-
-        if (!backingMap.containsKey(key)) {
-            commit.close();
-            return false;
-        }
-
-        Versioned<Collection<? extends byte[]>> removedValues = backingMap
-                .get(key)
-                .addCommit(commit);
-
-        if (removedValues != null) {
-            publish(removedValues.value().stream()
-                    .map(value -> new MultimapEvent<String, byte[]>(
-                            "", key, null, value))
-                    .collect(Collectors.toList()));
-            return true;
-        }
-
-        return false;
-    }
-
-    /**
-     * 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.operation().key();
-        if (commit.operation().values().isEmpty()) {
-            return false;
-        }
-        if (!backingMap.containsKey(key)) {
-            backingMap.put(key, new NonTransactionalCommit(1));
-        }
-
-        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;
-    }
-
-    protected Versioned<Collection<? extends byte[]>> replace(
-            Commit<? extends Replace> commit) {
-        if (!backingMap.containsKey(commit.operation().key())) {
-            backingMap.put(commit.operation().key(),
-                           new NonTransactionalCommit(1));
-        }
-        return backingMap.get(commit.operation().key()).addCommit(commit);
-    }
-
-    /**
-     * Handles a listen commit.
-     *
-     * @param commit listen commit
-     */
-    protected void listen(Commit<? extends Listen> commit) {
-        Long sessionId = commit.session().id();
-        if (listeners.putIfAbsent(sessionId, commit) != null) {
-            commit.close();
-            return;
-        }
-        commit.session()
-                .onStateChange(
-                        state -> {
-                            if (state == ServerSession.State.CLOSED
-                                    || state == ServerSession.State.EXPIRED) {
-                                Commit<? extends Listen> listener = listeners.remove(sessionId);
-                                if (listener != null) {
-                                    listener.close();
-                                }
-                            }
-                        });
-    }
-
-    /**
-     * Handles an unlisten commit.
-     *
-     * @param commit unlisten commit
-     */
-    protected void unlisten(Commit<? extends Unlisten> commit) {
-        try {
-            Commit<? extends Listen> listener = listeners.remove(commit.session().id());
-            if (listener != null) {
-                listener.close();
-            }
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Publishes events to listeners.
-     *
-     * @param events list of map event to publish
-     */
-    private void publish(List<MultimapEvent<String, byte[]>> events) {
-        listeners.values().forEach(commit ->
-                commit.session().publish(AtomixConsistentSetMultimap.CHANGE_SUBJECT, 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();
-
-        /**
-         * Discards the value by invoke appropriate clean up actions.
-         */
-        void discard();
-
-        /**
-         * 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 MultimapCommand> commit);
-    }
-
-    private class NonTransactionalCommit implements MapEntryValue {
-        private long version;
-        private final TreeMap<byte[], CountDownCompleter<Commit>>
-                valueCountdownMap = Maps.newTreeMap(new ByteArrayComparator());
-        /*This is a mapping of commits that added values to the commits
-        * removing those values, they will not be circular because keys will
-        * be exclusively Put and Replace commits and values will be exclusively
-        * Multiremove commits, each time a Put or replace is removed it should
-        * as part of closing go through and countdown each of the remove
-        * commits depending on it.*/
-        private final HashMultimap<Commit, CountDownCompleter<Commit>>
-                additiveToRemovalCommits = HashMultimap.create();
-
-        public NonTransactionalCommit(
-                long version) {
-            //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(valueCountdownMap.keySet());
-        }
-
-        @Override
-        public long version() {
-            return version;
-        }
-
-        @Override
-        public void discard() {
-            valueCountdownMap.values().forEach(completer ->
-                                                   completer.object().close());
-        }
-
-        @Override
-        public Versioned<Collection<? extends byte[]>> addCommit(
-                Commit<? extends MultimapCommand> commit) {
-            Preconditions.checkNotNull(commit);
-            Preconditions.checkNotNull(commit.operation());
-            Versioned<Collection<? extends byte[]>> retVersion;
-
-            if (commit.operation() instanceof Put) {
-                //Using a treeset here sanitizes the input, removing duplicates
-                Set<byte[]> valuesToAdd =
-                        Sets.newTreeSet(new ByteArrayComparator());
-                ((Put) commit.operation()).values().forEach(value -> {
-                    if (!valueCountdownMap.containsKey(value)) {
-                        valuesToAdd.add(value);
-                    }
-                });
-                if (valuesToAdd.isEmpty()) {
-                    //Do not increment or add the commit if no change resulted
-                    commit.close();
-                    return null;
-                }
-                //When all values from a commit have been removed decrement all
-                //removal commits relying on it and remove itself from the
-                //mapping of additive commits to the commits removing the
-                //values it added. (Only multiremoves will be dependent)
-                CountDownCompleter<Commit> completer =
-                        new CountDownCompleter<>(commit, valuesToAdd.size(),
-                        c -> {
-                            if (additiveToRemovalCommits.containsKey(c)) {
-                                additiveToRemovalCommits.
-                                        get(c).
-                                        forEach(countdown ->
-                                                        countdown.countDown());
-                                additiveToRemovalCommits.removeAll(c);
-                            }
-                            c.close();
-                        });
-                retVersion = new Versioned<>(valuesToAdd, version);
-                valuesToAdd.forEach(value -> valueCountdownMap.put(value,
-                                                                   completer));
-                version++;
-                return retVersion;
-
-            } else if (commit.operation() instanceof Replace) {
-                //Will this work??  Need to check before check-in!
-                Set<byte[]> removedValues = Sets.newHashSet();
-                removedValues.addAll(valueCountdownMap.keySet());
-                retVersion = new Versioned<>(removedValues, version);
-                valueCountdownMap.values().forEach(countdown ->
-                                                   countdown.countDown());
-                valueCountdownMap.clear();
-                Set<byte[]> valuesToAdd =
-                        Sets.newTreeSet(new ByteArrayComparator());
-                ((Replace) commit.operation()).values().forEach(value -> {
-                    valuesToAdd.add(value);
-                });
-                if (valuesToAdd.isEmpty()) {
-                    version = globalVersion.incrementAndGet();
-                    backingMap.remove(((Replace) commit.operation()).key());
-                    //Order is important here, the commit must be closed last
-                    //(or minimally after all uses)
-                    commit.close();
-                    return retVersion;
-                }
-                CountDownCompleter<Commit> completer =
-                        new CountDownCompleter<>(commit, valuesToAdd.size(),
-                                     c -> {
-                                         if (additiveToRemovalCommits
-                                             .containsKey(c)) {
-                                            additiveToRemovalCommits.
-                                                 get(c).
-                                                 forEach(countdown ->
-                                                     countdown.countDown());
-                                             additiveToRemovalCommits.
-                                                     removeAll(c);
-                                         }
-                                         c.close();
-                                     });
-                valuesToAdd.forEach(value ->
-                                    valueCountdownMap.put(value, completer));
-                version = globalVersion.incrementAndGet();
-                return retVersion;
-
-            } else if (commit.operation() 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(valueCountdownMap.keySet());
-                retVersion = new Versioned<>(removed, version);
-                valueCountdownMap.values().forEach(countdown ->
-                                                   countdown.countDown());
-                valueCountdownMap.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.operation()).key();
-                commit.close();
-                version = globalVersion.incrementAndGet();
-                backingMap.remove(key);
-                return retVersion;
-
-            } else if (commit.operation() 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();
-                Set<Commit> commitsRemovedFrom = Sets.newHashSet();
-                ((MultiRemove) commit.operation()).values().forEach(value -> {
-                    if (valueCountdownMap.containsKey(value)) {
-                        removed.add(value);
-                        commitsRemovedFrom
-                                .add(valueCountdownMap.get(value).object());
-                    }
-                });
-                //If there is nothing to be removed no action should be taken.
-                if (removed.isEmpty()) {
-                    //Do not increment or add the commit if no change resulted
-                    commit.close();
-                    return null;
-                }
-                //When all additive commits this depends on are closed this can
-                //be closed as well.
-                CountDownCompleter<Commit> completer =
-                        new CountDownCompleter<>(commit,
-                                                 commitsRemovedFrom.size(),
-                                                 c -> c.close());
-                commitsRemovedFrom.forEach(commitRemovedFrom -> {
-                    additiveToRemovalCommits.put(commitRemovedFrom, completer);
-                });
-                //Save key in case countdown results in closing the commit.
-                String removedKey = ((MultiRemove) commit.operation()).key();
-                removed.forEach(removedValue -> {
-                    valueCountdownMap.remove(removedValue).countDown();
-                });
-                //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 (valueCountdownMap.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 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;
-            }
-        }
-    }
- }
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
index bf462c9..8f91b3d 100644
--- 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
@@ -16,32 +16,12 @@
 
 package org.onosproject.store.primitives.resources.impl;
 
-import com.google.common.collect.Maps;
-import io.atomix.copycat.client.CopycatClient;
-import io.atomix.resource.AbstractResource;
-import io.atomix.resource.ResourceTypeInfo;
-import org.onlab.util.Match;
-import org.onosproject.store.primitives.MapUpdate;
-import org.onosproject.store.primitives.TransactionId;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.FirstKey;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.FloorEntry;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.HigherEntry;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.LastEntry;
-import org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.LowerEntry;
-import org.onosproject.store.service.AsyncConsistentTreeMap;
-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.ConcurrentModificationException;
 import java.util.List;
 import java.util.Map;
 import java.util.NavigableMap;
 import java.util.NavigableSet;
-import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
@@ -49,114 +29,145 @@
 import java.util.function.BiFunction;
 import java.util.function.Predicate;
 
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.CeilingEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.CeilingKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.Clear;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.ContainsKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.ContainsValue;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.EntrySet;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.FirstEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.FloorKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.Get;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.GetOrDefault;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.HigherKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.IsEmpty;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.KeySet;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.LastKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.Listen;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.LowerKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.PollFirstEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.PollLastEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.Size;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.Unlisten;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.UpdateAndGet;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.Values;
+import com.google.common.collect.Maps;
+import io.atomix.protocols.raft.proxy.RaftProxy;
+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.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.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.AtomixConsistentTreeMapEvents.CHANGE;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.ADD_LISTENER;
+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.CLEAR;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.CONTAINS_KEY;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.CONTAINS_VALUE;
+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.ContainsKey;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.ContainsValue;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.ENTRY_SET;
+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.GET;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.GET_OR_DEFAULT;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.Get;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.GetOrDefault;
+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.IS_EMPTY;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.KEY_SET;
+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;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.REMOVE_LISTENER;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.SIZE;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.UPDATE_AND_GET;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.UpdateAndGet;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.VALUES;
 
 /**
  * Implementation of {@link AsyncConsistentTreeMap}.
  */
-@ResourceTypeInfo(id = -155, factory = AtomixConsistentTreeMapFactory.class)
-public class AtomixConsistentTreeMap extends AbstractResource<AtomixConsistentTreeMap>
-        implements AsyncConsistentTreeMap<byte[]> {
+public class AtomixConsistentTreeMap extends AbstractRaftPrimitive implements AsyncConsistentTreeMap<byte[]> {
+    private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
+            .register(KryoNamespaces.BASIC)
+            .register(AtomixConsistentTreeMapOperations.NAMESPACE)
+            .register(AtomixConsistentTreeMapEvents.NAMESPACE)
+            .build());
 
     private final Map<MapEventListener<String, byte[]>, Executor>
             mapEventListeners = Maps.newConcurrentMap();
 
-    public static final String CHANGE_SUBJECT = "changeEvents";
-
-    public AtomixConsistentTreeMap(CopycatClient client, Properties options) {
-        super(client, options);
-    }
-
-    @Override
-    public String name() {
-        return null;
-    }
-
-    @Override
-    public CompletableFuture<AtomixConsistentTreeMap> open() {
-        return super.open().thenApply(result -> {
-            client.onEvent(CHANGE_SUBJECT, this::handleEvent);
-            return result;
-        });
+    public AtomixConsistentTreeMap(RaftProxy proxy) {
+        super(proxy);
+        proxy.addEventListener(CHANGE, SERIALIZER::decode, this::handleEvent);
     }
 
     private void handleEvent(List<MapEvent<String, byte[]>> events) {
         events.forEach(event -> mapEventListeners.
                 forEach((listener, executor) ->
-                                executor.execute(() ->
-                                                     listener.event(event))));
+                        executor.execute(() ->
+                                listener.event(event))));
     }
 
     @Override
     public CompletableFuture<Boolean> isEmpty() {
-        return client.submit(new IsEmpty());
+        return proxy.invoke(IS_EMPTY, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Integer> size() {
-        return client.submit(new Size());
+        return proxy.invoke(SIZE, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Boolean> containsKey(String key) {
-        return client.submit(new ContainsKey(key));
+        return proxy.invoke(CONTAINS_KEY, SERIALIZER::encode, new ContainsKey(key), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Boolean> containsValue(byte[] value) {
-        return client.submit(new ContainsValue(value));
+        return proxy.invoke(CONTAINS_VALUE, SERIALIZER::encode, new ContainsValue(value), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Versioned<byte[]>> get(String key) {
-        return client.submit(new Get(key));
+        return proxy.invoke(GET, SERIALIZER::encode, new Get(key), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Versioned<byte[]>> getOrDefault(String key, byte[] defaultValue) {
-        return client.submit(new GetOrDefault(key, defaultValue));
+        return proxy.invoke(
+                GET_OR_DEFAULT,
+                SERIALIZER::encode,
+                new GetOrDefault(key, defaultValue),
+                SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Set<String>> keySet() {
-        return client.submit(new KeySet());
+        return proxy.invoke(KEY_SET, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Collection<Versioned<byte[]>>> values() {
-        return client.submit(new Values());
+        return proxy.invoke(VALUES, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Set<Map.Entry<String, Versioned<byte[]>>>> entrySet() {
-        return client.submit(new EntrySet());
+        return proxy.invoke(ENTRY_SET, SERIALIZER::decode);
     }
 
     @Override
     @SuppressWarnings("unchecked")
     public CompletableFuture<Versioned<byte[]>> put(String key, byte[] value) {
-        return client.submit(new UpdateAndGet(key, value, Match.ANY, Match.ANY))
+        return proxy.<UpdateAndGet, MapEntryUpdateResult<String, byte[]>>invoke(
+                UPDATE_AND_GET,
+                SERIALIZER::encode,
+                new UpdateAndGet(key, value, Match.ANY, Match.ANY),
+                SERIALIZER::decode)
                 .whenComplete((r, e) -> throwIfLocked(r.status()))
                 .thenApply(v -> v.oldValue());
     }
@@ -164,7 +175,11 @@
     @Override
     @SuppressWarnings("unchecked")
     public CompletableFuture<Versioned<byte[]>> putAndGet(String key, byte[] value) {
-        return client.submit(new UpdateAndGet(key, value, Match.ANY, Match.ANY))
+        return proxy.<UpdateAndGet, MapEntryUpdateResult<String, byte[]>>invoke(
+                UPDATE_AND_GET,
+                SERIALIZER::encode,
+                new UpdateAndGet(key, value, Match.ANY, Match.ANY),
+                SERIALIZER::decode)
                 .whenComplete((r, e) -> throwIfLocked(r.status()))
                 .thenApply(v -> v.newValue());
     }
@@ -172,7 +187,11 @@
     @Override
     @SuppressWarnings("unchecked")
     public CompletableFuture<Versioned<byte[]>> putIfAbsent(String key, byte[] value) {
-        return client.submit(new UpdateAndGet(key, value, Match.NULL, Match.ANY))
+        return proxy.<UpdateAndGet, MapEntryUpdateResult<String, byte[]>>invoke(
+                UPDATE_AND_GET,
+                SERIALIZER::encode,
+                new UpdateAndGet(key, value, Match.NULL, Match.ANY),
+                SERIALIZER::decode)
                 .whenComplete((r, e) -> throwIfLocked(r.status()))
                 .thenApply(v -> v.oldValue());
     }
@@ -180,7 +199,11 @@
     @Override
     @SuppressWarnings("unchecked")
     public CompletableFuture<Versioned<byte[]>> remove(String key) {
-        return client.submit(new UpdateAndGet(key, null, Match.ANY, Match.ANY))
+        return proxy.<UpdateAndGet, MapEntryUpdateResult<String, byte[]>>invoke(
+                UPDATE_AND_GET,
+                SERIALIZER::encode,
+                new UpdateAndGet(key, null, Match.ANY, Match.ANY),
+                SERIALIZER::decode)
                 .whenComplete((r, e) -> throwIfLocked(r.status()))
                 .thenApply(v -> v.oldValue());
     }
@@ -188,7 +211,11 @@
     @Override
     @SuppressWarnings("unchecked")
     public CompletableFuture<Boolean> remove(String key, byte[] value) {
-        return client.submit(new UpdateAndGet(key, null, Match.ifValue(value), Match.ANY))
+        return proxy.<UpdateAndGet, MapEntryUpdateResult<String, byte[]>>invoke(
+                UPDATE_AND_GET,
+                SERIALIZER::encode,
+                new UpdateAndGet(key, null, Match.ifValue(value), Match.ANY),
+                SERIALIZER::decode)
                 .whenComplete((r, e) -> throwIfLocked(r.status()))
                 .thenApply(v -> v.updated());
     }
@@ -196,7 +223,11 @@
     @Override
     @SuppressWarnings("unchecked")
     public CompletableFuture<Boolean> remove(String key, long version) {
-        return client.submit(new UpdateAndGet(key, null, Match.ANY, Match.ifValue(version)))
+        return proxy.<UpdateAndGet, MapEntryUpdateResult<String, byte[]>>invoke(
+                UPDATE_AND_GET,
+                SERIALIZER::encode,
+                new UpdateAndGet(key, null, Match.ANY, Match.ifValue(version)),
+                SERIALIZER::decode)
                 .whenComplete((r, e) -> throwIfLocked(r.status()))
                 .thenApply(v -> v.updated());
     }
@@ -204,7 +235,11 @@
     @Override
     @SuppressWarnings("unchecked")
     public CompletableFuture<Versioned<byte[]>> replace(String key, byte[] value) {
-        return client.submit(new UpdateAndGet(key, value, Match.NOT_NULL, Match.ANY))
+        return proxy.<UpdateAndGet, MapEntryUpdateResult<String, byte[]>>invoke(
+                UPDATE_AND_GET,
+                SERIALIZER::encode,
+                new UpdateAndGet(key, value, Match.NOT_NULL, Match.ANY),
+                SERIALIZER::decode)
                 .whenComplete((r, e) -> throwIfLocked(r.status()))
                 .thenApply(v -> v.oldValue());
     }
@@ -212,7 +247,11 @@
     @Override
     @SuppressWarnings("unchecked")
     public CompletableFuture<Boolean> replace(String key, byte[] oldValue, byte[] newValue) {
-        return client.submit(new UpdateAndGet(key, newValue, Match.ifValue(oldValue), Match.ANY))
+        return proxy.<UpdateAndGet, MapEntryUpdateResult<String, byte[]>>invoke(
+                UPDATE_AND_GET,
+                SERIALIZER::encode,
+                new UpdateAndGet(key, newValue, Match.ifValue(oldValue), Match.ANY),
+                SERIALIZER::decode)
                 .whenComplete((r, e) -> throwIfLocked(r.status()))
                 .thenApply(v -> v.updated());
     }
@@ -220,14 +259,18 @@
     @Override
     @SuppressWarnings("unchecked")
     public CompletableFuture<Boolean> replace(String key, long oldVersion, byte[] newValue) {
-        return client.submit(new UpdateAndGet(key, newValue, Match.ANY, Match.ifValue(oldVersion)))
+        return proxy.<UpdateAndGet, MapEntryUpdateResult<String, byte[]>>invoke(
+                UPDATE_AND_GET,
+                SERIALIZER::encode,
+                new UpdateAndGet(key, newValue, Match.ANY, Match.ifValue(oldVersion)),
+                SERIALIZER::decode)
                 .whenComplete((r, e) -> throwIfLocked(r.status()))
                 .thenApply(v -> v.updated());
     }
 
     @Override
     public CompletableFuture<Void> clear() {
-        return client.submit(new Clear())
+        return proxy.<MapEntryUpdateResult.Status>invoke(CLEAR, SERIALIZER::decode)
                 .whenComplete((r, e) -> throwIfLocked(r))
                 .thenApply(v -> null);
     }
@@ -235,10 +278,10 @@
     @Override
     @SuppressWarnings("unchecked")
     public CompletableFuture<Versioned<byte[]>> computeIf(String key,
-                                                          Predicate<? super byte[]> condition,
-                                                          BiFunction<? super String,
-                                                                  ? super byte[],
-                                                                  ? extends byte[]> remappingFunction) {
+            Predicate<? super byte[]> condition,
+            BiFunction<? super String,
+                    ? super byte[],
+                    ? extends byte[]> remappingFunction) {
         return get(key).thenCompose(r1 -> {
             byte[] existingValue = r1 == null ? null : r1.value();
 
@@ -259,8 +302,11 @@
             }
             Match<byte[]> valueMatch = r1 == null ? Match.NULL : Match.ANY;
             Match<Long> versionMatch = r1 == null ? Match.ANY : Match.ifValue(r1.version());
-            return client.submit(new UpdateAndGet(key, computedValue.get(),
-                                                                      valueMatch, versionMatch))
+            return proxy.<UpdateAndGet, MapEntryUpdateResult<String, byte[]>>invoke(
+                    UPDATE_AND_GET,
+                    SERIALIZER::encode,
+                    new UpdateAndGet(key, computedValue.get(), valueMatch, versionMatch),
+                    SERIALIZER::decode)
                     .whenComplete((r, e) -> throwIfLocked(r.status()))
                     .thenApply(v -> v.newValue());
         });
@@ -270,9 +316,9 @@
     public CompletableFuture<Void> addListener(
             MapEventListener<String, byte[]> listener, Executor executor) {
         if (mapEventListeners.isEmpty()) {
-            return client.submit(new Listen()).thenRun(() ->
-                                   mapEventListeners.put(listener,
-                                           executor));
+            return proxy.invoke(ADD_LISTENER).thenRun(() ->
+                    mapEventListeners.put(listener,
+                            executor));
         } else {
             mapEventListeners.put(listener, executor);
             return CompletableFuture.completedFuture(null);
@@ -283,7 +329,7 @@
     public synchronized CompletableFuture<Void> removeListener(MapEventListener<String, byte[]> listener) {
         if (mapEventListeners.remove(listener) != null &&
                 mapEventListeners.isEmpty()) {
-            return client.submit(new Unlisten())
+            return proxy.invoke(REMOVE_LISTENER)
                     .thenApply(v -> null);
         }
         return CompletableFuture.completedFuture(null);
@@ -298,74 +344,74 @@
 
     @Override
     public CompletableFuture<String> firstKey() {
-        return client.submit(new FirstKey<String>());
+        return proxy.invoke(FIRST_KEY, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<String> lastKey() {
-        return client.submit(new LastKey<String>());
+        return proxy.invoke(LAST_KEY, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> ceilingEntry(String key) {
-        return client.submit(new CeilingEntry(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 client.submit(new FloorEntry<String, Versioned<byte[]>>(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 client.submit(new HigherEntry<String, Versioned<byte[]>>(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 client.submit(new LowerEntry<>(key));
+        return proxy.invoke(LOWER_ENTRY, SERIALIZER::encode, new LowerEntry(key), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> firstEntry() {
-        return client.submit(new FirstEntry());
+        return proxy.invoke(FIRST_ENTRY, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> lastEntry() {
-        return client.submit(new LastEntry<String, Versioned<byte[]>>());
+        return proxy.invoke(LAST_ENTRY, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> pollFirstEntry() {
-        return client.submit(new PollFirstEntry());
+        return proxy.invoke(POLL_FIRST_ENTRY, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Map.Entry<String, Versioned<byte[]>>> pollLastEntry() {
-        return client.submit(new PollLastEntry());
+        return proxy.invoke(POLL_LAST_ENTRY, SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<String> lowerKey(String key) {
-        return client.submit(new LowerKey(key));
+        return proxy.invoke(LOWER_KEY, SERIALIZER::encode, new LowerKey(key), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<String> floorKey(String key) {
-        return client.submit(new FloorKey(key));
+        return proxy.invoke(FLOOR_KEY, SERIALIZER::encode, new FloorKey(key), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<String> ceilingKey(String key) {
-        return client.submit(new CeilingKey(key));
+        return proxy.invoke(CEILING_KEY, SERIALIZER::encode, new CeilingKey(key), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<String> higherKey(String key) {
-        return client.submit(new HigherKey(key));
+        return proxy.invoke(HIGHER_KEY, SERIALIZER::encode, new HigherKey(key), SERIALIZER::decode);
     }
 
     @Override
@@ -404,4 +450,4 @@
     public CompletableFuture<Void> rollback(TransactionId transactionId) {
         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/AtomixConsistentTreeMapCommands.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapCommands.java
deleted file mode 100644
index 59cf1a5..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapCommands.java
+++ /dev/null
@@ -1,700 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Laboratory
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.onosproject.store.primitives.resources.impl;
-
-import com.google.common.base.MoreObjects;
-import io.atomix.catalyst.buffer.BufferInput;
-import io.atomix.catalyst.buffer.BufferOutput;
-import io.atomix.catalyst.serializer.CatalystSerializable;
-import io.atomix.catalyst.serializer.SerializableTypeResolver;
-import io.atomix.catalyst.serializer.Serializer;
-import io.atomix.catalyst.serializer.SerializerRegistry;
-import io.atomix.catalyst.util.Assert;
-import io.atomix.copycat.Command;
-import io.atomix.copycat.Query;
-import org.onlab.util.Match;
-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;
-
-/**
- * {@link org.onosproject.store.service.AsyncConsistentTreeMap} Resource
- * state machine operations.
- */
-public final class AtomixConsistentTreeMapCommands {
-
-    private AtomixConsistentTreeMapCommands() {
-    }
-
-    /**
-     * Abstract treeMap command.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class TreeCommand<V>
-            implements Command<V>, CatalystSerializable {
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .toString();
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer,
-                                Serializer serializer) {
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-        }
-    }
-
-    /**
-     * Abstract treeMap query.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class TreeQuery<V>
-            implements Query<V>, CatalystSerializable {
-        @Override
-        public ConsistencyLevel consistency() {
-            return ConsistencyLevel.LINEARIZABLE_LEASE;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .toString();
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> bufferOutput,
-                                Serializer serializer) {
-
-        }
-
-        @Override
-        public void readObject(BufferInput<?> bufferInput,
-                               Serializer serializer) {
-
-        }
-    }
-    /**
-     * Abstract key-based query.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class KeyQuery<K> extends TreeQuery<K> {
-        protected String key;
-
-        public KeyQuery(String key) {
-            this.key = Assert.notNull(key, "key");
-        }
-
-        public KeyQuery() {
-        }
-
-        public String key() {
-            return key;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("key", key)
-                    .toString();
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> bufferOutput,
-                                Serializer serializer) {
-            super.writeObject(bufferOutput, serializer);
-            serializer.writeObject(key, bufferOutput);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> bufferInput,
-                               Serializer serializer) {
-            super.readObject(bufferInput, serializer);
-            key = serializer.readObject(bufferInput);
-        }
-    }
-
-    /**
-     * Abstract value-based query.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class ValueQuery<V> extends TreeQuery<V> {
-        protected byte[] value;
-
-        public ValueQuery() {}
-
-        public ValueQuery(byte[] value) {
-            this.value = Assert.notNull(value, "value");
-        }
-
-        public byte[] value() {
-            return value;
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> bufferOutput,
-                                Serializer serializer) {
-            super.writeObject(bufferOutput, serializer);
-            serializer.writeObject(value, bufferOutput);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> bufferInput,
-                               Serializer serializer) {
-            super.readObject(bufferInput, serializer);
-            value = serializer.readObject(bufferInput);
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("value", value)
-                    .toString();
-        }
-    }
-
-    /**
-     * Contains key command.
-     */
-    @SuppressWarnings("serial")
-    public static class ContainsKey extends KeyQuery<Boolean> {
-
-        public ContainsKey(String key) {
-            super(key);
-        }
-
-        public ContainsKey() {
-        }
-    }
-    /**
-     * Contains value command.
-     */
-    @SuppressWarnings("serial")
-    public static class ContainsValue extends ValueQuery<Boolean> {
-        public ContainsValue() {
-        }
-
-        public ContainsValue(byte[] value) {
-            super(value);
-        }
-
-    }
-
-    /**
-     * AsyncConsistentTreeMap update command.
-     */
-    @SuppressWarnings("serial")
-    public static class UpdateAndGet
-            extends TreeCommand<MapEntryUpdateResult<String, byte[]>> {
-        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 CompactionMode compaction() {
-            return value == null ? CompactionMode.TOMBSTONE : CompactionMode.QUORUM;
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer,
-                                Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            serializer.writeObject(key, buffer);
-            serializer.writeObject(value, buffer);
-            serializer.writeObject(valueMatch, buffer);
-            serializer.writeObject(versionMatch, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            key = serializer.readObject(buffer);
-            value = serializer.readObject(buffer);
-            valueMatch = serializer.readObject(buffer);
-            versionMatch = serializer.readObject(buffer);
-        }
-
-        @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 KeyQuery<Versioned<byte[]>> {
-        public Get() {
-        }
-
-        public Get(String key) {
-            super(key);
-        }
-    }
-
-    /**
-     * Get or default query.
-     */
-    @SuppressWarnings("serial")
-    public static class GetOrDefault extends KeyQuery<Versioned<byte[]>> {
-        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 void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            serializer.writeObject(defaultValue, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            defaultValue = serializer.readObject(buffer);
-        }
-    }
-
-    /**
-     * Is empty query.
-     */
-    @SuppressWarnings("serial")
-    public static class IsEmpty extends TreeQuery<Boolean> {
-
-    }
-
-    /**
-     * Key set query.
-     */
-    @SuppressWarnings("serial")
-    public static class KeySet extends TreeQuery<Set<String>> {
-    }
-
-    /**
-     * Value set query.
-     */
-    @SuppressWarnings("serial")
-    public static class Values
-            extends TreeQuery<Collection<Versioned<byte[]>>> {
-    }
-
-    /**
-     * Entry set query.
-     */
-    @SuppressWarnings("serial")
-    public static class EntrySet
-            extends TreeQuery<Set<Map.Entry<String, Versioned<byte[]>>>> {
-    }
-
-    /**
-     * Size query.
-     */
-    @SuppressWarnings("serial")
-    public static class Size extends TreeQuery<Integer> {
-    }
-
-    /**
-     * Clear command.
-     */
-    @SuppressWarnings("serial")
-    public static class Clear
-            extends TreeCommand<MapEntryUpdateResult.Status> {
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.TOMBSTONE;
-        }
-    }
-
-    /**
-     * Change listen.
-     */
-    @SuppressWarnings("serial")
-    public static class Listen implements Command<Void>, CatalystSerializable {
-        @Override
-        public void writeObject(BufferOutput<?> buffer,
-                                Serializer serializer) {
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.QUORUM;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .toString();
-        }
-    }
-
-    /**
-     * Change unlisten.
-     */
-    @SuppressWarnings("serial")
-    public static class Unlisten implements Command<Void>,
-            CatalystSerializable {
-        @Override
-        public void writeObject(BufferOutput<?> buffer,
-                                Serializer serializer) {
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.TOMBSTONE;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .toString();
-        }
-    }
-
-        /* Tree map specific commands below */
-
-    /**
-     * First key query.
-     */
-    @SuppressWarnings("serial")
-    public static class FirstKey<K> extends TreeQuery<K> {
-    }
-
-    /**
-     * Last key query.
-     */
-    @SuppressWarnings("serial")
-    public static class LastKey<K> extends TreeQuery<K> {
-    }
-
-    /**
-     * First entry query.
-     */
-    @SuppressWarnings("serial")
-    public static class FirstEntry<K, V> extends TreeQuery<Map.Entry<K, V>> {
-    }
-
-    /**
-     * Last entry query.
-     */
-    @SuppressWarnings("serial")
-    public static class LastEntry<K, V> extends TreeQuery<Map.Entry<K, V>> {
-    }
-
-    /**
-     * First entry query, if none exists returns null.
-     */
-    @SuppressWarnings("serial")
-    public static class PollFirstEntry<K, V>
-            extends TreeQuery<Map.Entry<K, V>> {
-    }
-
-    /**
-     * Last entry query, if none exists returns null.
-     */
-    @SuppressWarnings("serial")
-    public static class PollLastEntry<K, V>
-            extends TreeQuery<Map.Entry<K, V>> {
-    }
-
-    /**
-     * Query returns the entry associated with the largest key less than the
-     * passed in key.
-     */
-    @SuppressWarnings("serial")
-    public static class LowerEntry<K, V> extends KeyQuery<K> {
-        public LowerEntry() {
-        }
-
-        public LowerEntry(String key) {
-            super(key);
-        }
-    }
-
-    /**
-     * Query returns the largest key less than the specified key.
-     */
-    @SuppressWarnings("serial")
-    public static class LowerKey<K> extends KeyQuery<K> {
-        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<K, V> extends KeyQuery<Map.Entry<K, V>> {
-        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<K> extends KeyQuery<K> {
-        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<K, V> extends KeyQuery<Map.Entry<K, V>> {
-        public CeilingEntry() {
-        }
-
-        public CeilingEntry(String key) {
-            super(key);
-        }
-    }
-
-    /**
-     * Returns the smallest key larger than or equal to the specified key.
-     *
-     * @param <K> key type
-     */
-    @SuppressWarnings("serial")
-    public static class CeilingKey<K> extends KeyQuery<K> {
-        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<K, V> extends KeyQuery<Map.Entry<K, V>> {
-        public HigherEntry() {
-        }
-
-        public HigherEntry(String key) {
-            super(key);
-        }
-    }
-
-    /**
-     * Returns the smallest key larger than the specified key.
-     */
-    @SuppressWarnings("serial")
-    public static class HigherKey<K> extends KeyQuery<K> {
-        public HigherKey() {
-        }
-
-        public HigherKey(String key) {
-            super(key);
-        }
-    }
-
-    @SuppressWarnings("serial")
-    public static class NavigableKeySet<K, V>
-            extends TreeQuery<NavigableSet<K>> {
-    }
-
-    @SuppressWarnings("serial")
-    public static class SubMap<K, V> extends TreeQuery<NavigableMap<K, V>> {
-        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("inclusiveFrotBound", inclusiveFrom)
-                    .add("inclusiveToBound", inclusiveTo)
-                    .toString();
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> bufferOutput,
-                                Serializer serializer) {
-            super.writeObject(bufferOutput, serializer);
-            serializer.writeObject(fromKey, bufferOutput);
-            serializer.writeObject(toKey, bufferOutput);
-            serializer.writeObject(inclusiveFrom, bufferOutput);
-            serializer.writeObject(inclusiveTo, bufferOutput);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> bufferInput,
-                               Serializer serializer) {
-            super.readObject(bufferInput, serializer);
-            fromKey = serializer.readObject(bufferInput);
-            toKey = serializer.readObject(bufferInput);
-            inclusiveFrom = serializer.readObject(bufferInput);
-            inclusiveTo = serializer.readObject(bufferInput);
-        }
-
-        public K fromKey() {
-            return fromKey;
-        }
-
-        public K toKey() {
-            return toKey;
-        }
-
-        public boolean isInclusiveFrom() {
-            return inclusiveFrom;
-        }
-
-        public boolean isInclusiveTo() {
-            return inclusiveTo;
-        }
-    }
-
-    /**
-     * Tree map command type resolver.
-     */
-    public static class TypeResolver implements SerializableTypeResolver {
-        @Override
-        public void resolve(SerializerRegistry registry) {
-            //NOTE the registration values must be unique throughout the
-            // project.
-            registry.register(ContainsKey.class, -1161);
-            registry.register(ContainsValue.class, -1162);
-            registry.register(Get.class, -1163);
-            registry.register(GetOrDefault.class, -1192);
-            registry.register(EntrySet.class, -1164);
-            registry.register(Values.class, -1165);
-            registry.register(KeySet.class, -1166);
-            registry.register(Clear.class, -1167);
-            registry.register(IsEmpty.class, -1168);
-            registry.register(Size.class, -1169);
-            registry.register(Listen.class, -1170);
-            registry.register(Unlisten.class, -1171);
-            //Transaction related commands will be added here with numbers
-            // -1172 to -1174
-            registry.register(UpdateAndGet.class, -1175);
-            registry.register(FirstKey.class, -1176);
-            registry.register(LastKey.class, -1177);
-            registry.register(FirstEntry.class, -1178);
-            registry.register(LastEntry.class, -1179);
-            registry.register(PollFirstEntry.class, -1180);
-            registry.register(PollLastEntry.class, -1181);
-            registry.register(LowerEntry.class, -1182);
-            registry.register(LowerKey.class, -1183);
-            registry.register(FloorEntry.class, -1184);
-            registry.register(FloorKey.class, -1185);
-            registry.register(CeilingEntry.class, -1186);
-            registry.register(CeilingKey.class, -1187);
-            registry.register(HigherEntry.class, -1188);
-            registry.register(HigherKey.class, -1189);
-            registry.register(SubMap.class, -1190);
-            registry.register(NavigableKeySet.class, -1191);
-        }
-    }
-}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapEvents.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapEvents.java
new file mode 100644
index 0000000..ab3b972
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapEvents.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2017-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.store.primitives.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 tree map events.
+ */
+public enum AtomixConsistentTreeMapEvents implements EventType {
+    CHANGE("change");
+
+    private final String id;
+
+    AtomixConsistentTreeMapEvents(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(MapEvent.class)
+            .register(MapEvent.Type.class)
+            .build("AtomixConsistentTreeMapEvents");
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapFactory.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapFactory.java
deleted file mode 100644
index 8ce8557..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapFactory.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Laboratory
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.onosproject.store.primitives.resources.impl;
-
-import io.atomix.catalyst.serializer.SerializableTypeResolver;
-import io.atomix.copycat.client.CopycatClient;
-import io.atomix.resource.ResourceFactory;
-import io.atomix.resource.ResourceStateMachine;
-
-import java.util.Properties;
-
-/**
- * Factory for {@link AtomixConsistentTreeMap}.
- */
-public class AtomixConsistentTreeMapFactory implements ResourceFactory<AtomixConsistentTreeMap> {
-    @Override
-    public SerializableTypeResolver createSerializableTypeResolver() {
-        return new AtomixConsistentTreeMapCommands.TypeResolver();
-    }
-
-    @Override
-    public ResourceStateMachine createStateMachine(Properties config) {
-        return new AtomixConsistentTreeMapState(config);
-    }
-
-    @Override
-    public AtomixConsistentTreeMap createInstance(CopycatClient client, Properties options) {
-        return new AtomixConsistentTreeMap(client, options);
-    }
-}
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
new file mode 100644
index 0000000..c6b8d88
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapOperations.java
@@ -0,0 +1,438 @@
+/*
+ * Copyright 2016-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.onosproject.store.primitives.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 {
+    CONTAINS_KEY("containsKey", OperationType.QUERY),
+    CONTAINS_VALUE("containsValue", OperationType.QUERY),
+    ENTRY_SET("entrySet", OperationType.QUERY),
+    GET("get", OperationType.QUERY),
+    GET_OR_DEFAULT("getOrDefault", OperationType.QUERY),
+    IS_EMPTY("isEmpty", OperationType.QUERY),
+    KEY_SET("keySet", OperationType.QUERY),
+    SIZE("size", OperationType.QUERY),
+    VALUES("values", OperationType.QUERY),
+    SUB_MAP("subMap", OperationType.QUERY),
+    FIRST_KEY("firstKey", OperationType.QUERY),
+    LAST_KEY("lastKey", OperationType.QUERY),
+    FIRST_ENTRY("firstEntry", OperationType.QUERY),
+    LAST_ENTRY("lastEntry", OperationType.QUERY),
+    POLL_FIRST_ENTRY("pollFirstEntry", OperationType.QUERY),
+    POLL_LAST_ENTRY("pollLastEntry", OperationType.QUERY),
+    LOWER_ENTRY("lowerEntry", OperationType.QUERY),
+    LOWER_KEY("lowerKey", OperationType.QUERY),
+    FLOOR_ENTRY("floorEntry", OperationType.QUERY),
+    FLOOR_KEY("floorKey", OperationType.QUERY),
+    CEILING_ENTRY("ceilingEntry", OperationType.QUERY),
+    CEILING_KEY("ceilingKey", OperationType.QUERY),
+    HIGHER_ENTRY("higherEntry", OperationType.QUERY),
+    HIGHER_KEY("higherKey", OperationType.QUERY),
+    UPDATE_AND_GET("updateAndGet", OperationType.COMMAND),
+    CLEAR("clear", OperationType.COMMAND),
+    ADD_LISTENER("addListener", OperationType.COMMAND),
+    REMOVE_LISTENER("removeListener", OperationType.COMMAND);
+
+    private final String id;
+    private final OperationType type;
+
+    AtomixConsistentTreeMapOperations(String id, OperationType type) {
+        this.id = id;
+        this.type = type;
+    }
+
+    @Override
+    public String id() {
+        return id;
+    }
+
+    @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(LowerKey.class)
+            .register(LowerEntry.class)
+            .register(HigherKey.class)
+            .register(HigherEntry.class)
+            .register(FloorKey.class)
+            .register(FloorEntry.class)
+            .register(CeilingKey.class)
+            .register(CeilingEntry.class)
+            .register(UpdateAndGet.class)
+            .register(Match.class)
+            .register(Versioned.class)
+            .register(MapEntryUpdateResult.class)
+            .register(MapEntryUpdateResult.Status.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("inclusiveFrotBound", 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
new file mode 100644
index 0000000..b9e7135
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapService.java
@@ -0,0 +1,416 @@
+/*
+ * Copyright 2016-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.onosproject.store.primitives.resources.impl;
+
+import java.util.AbstractMap.SimpleImmutableEntry;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.Set;
+import java.util.TreeMap;
+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.serializers.KryoNamespaces;
+import org.onosproject.store.service.MapEvent;
+import org.onosproject.store.service.Serializer;
+import org.onosproject.store.service.Versioned;
+
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapEvents.CHANGE;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.ADD_LISTENER;
+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.CLEAR;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.CONTAINS_KEY;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.CONTAINS_VALUE;
+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.ContainsKey;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.ContainsValue;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.ENTRY_SET;
+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.GET;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.GET_OR_DEFAULT;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.Get;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.GetOrDefault;
+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.IS_EMPTY;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.KEY_SET;
+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.REMOVE_LISTENER;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.SIZE;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.SUB_MAP;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.SubMap;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.UPDATE_AND_GET;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.UpdateAndGet;
+import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapOperations.VALUES;
+import static org.onosproject.store.primitives.resources.impl.MapEntryUpdateResult.Status;
+
+/**
+ * State machine corresponding to {@link AtomixConsistentTreeMap} backed by a
+ * {@link TreeMap}.
+ */
+public class AtomixConsistentTreeMapService extends AbstractRaftService {
+
+    private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
+            .register(KryoNamespaces.BASIC)
+            .register(AtomixConsistentTreeMapOperations.NAMESPACE)
+            .register(AtomixConsistentTreeMapEvents.NAMESPACE)
+            .register(TreeMapEntryValue.class)
+            .register(new HashMap<>().keySet().getClass())
+            .register(TreeMap.class)
+            .build());
+
+    private final Map<Long, RaftSession> listeners = Maps.newHashMap();
+    private TreeMap<String, TreeMapEntryValue> tree = Maps.newTreeMap();
+    private final Set<String> preparedKeys = Sets.newHashSet();
+
+    @Override
+    public void snapshot(SnapshotWriter writer) {
+        writer.writeObject(Sets.newHashSet(listeners.keySet()), SERIALIZER::encode);
+        writer.writeObject(preparedKeys, SERIALIZER::encode);
+        writer.writeObject(tree, SERIALIZER::encode);
+    }
+
+    @Override
+    public void install(SnapshotReader reader) {
+        listeners.clear();
+        for (long sessionId : reader.<Set<Long>>readObject(SERIALIZER::decode)) {
+            listeners.put(sessionId, getSessions().getSession(sessionId));
+        }
+
+        preparedKeys.clear();
+        preparedKeys.addAll(reader.readObject(SERIALIZER::decode));
+
+        tree.clear();
+        tree.putAll(reader.readObject(SERIALIZER::decode));
+    }
+
+    @Override
+    public void configure(RaftServiceExecutor executor) {
+        // Listeners
+        executor.register(ADD_LISTENER, this::listen);
+        executor.register(REMOVE_LISTENER, this::unlisten);
+        // 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, this::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, this::isEmpty, SERIALIZER::encode);
+        executor.register(KEY_SET, this::keySet, SERIALIZER::encode);
+        executor.register(SIZE, this::size, SERIALIZER::encode);
+        executor.register(VALUES, this::values, SERIALIZER::encode);
+        executor.register(SUB_MAP, SERIALIZER::decode, this::subMap, SERIALIZER::encode);
+        executor.register(FIRST_KEY, this::firstKey, SERIALIZER::encode);
+        executor.register(LAST_KEY, this::lastKey, SERIALIZER::encode);
+        executor.register(FIRST_ENTRY, this::firstEntry, SERIALIZER::encode);
+        executor.register(LAST_ENTRY, this::lastEntry, SERIALIZER::encode);
+        executor.register(POLL_FIRST_ENTRY, this::pollFirstEntry, SERIALIZER::encode);
+        executor.register(POLL_LAST_ENTRY, this::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);
+
+        // Commands
+        executor.register(UPDATE_AND_GET, SERIALIZER::decode, this::updateAndGet, SERIALIZER::encode);
+        executor.register(CLEAR, this::clear, SERIALIZER::encode);
+    }
+
+    protected boolean containsKey(Commit<? extends ContainsKey> commit) {
+        return toVersioned(tree.get((commit.value().key()))) != null;
+    }
+
+    protected boolean containsValue(Commit<? extends ContainsValue> commit) {
+        Match<byte[]> valueMatch = Match
+                .ifValue(commit.value().value());
+        return tree.values().stream().anyMatch(
+                value -> valueMatch.matches(value.value()));
+    }
+
+    protected Versioned<byte[]> get(Commit<? extends Get> commit) {
+        return toVersioned(tree.get(commit.value().key()));
+    }
+
+    protected Versioned<byte[]> getOrDefault(Commit<? extends GetOrDefault> commit) {
+        Versioned<byte[]> value = toVersioned(tree.get(commit.value().key()));
+        return value != null ? value : new Versioned<>(commit.value().defaultValue(), 0);
+    }
+
+    protected int size(Commit<Void> commit) {
+        return tree.size();
+    }
+
+    protected boolean isEmpty(Commit<Void> commit) {
+        return tree.isEmpty();
+    }
+
+    protected Set<String> keySet(Commit<Void> commit) {
+        return tree.keySet().stream().collect(Collectors.toSet());
+    }
+
+    protected Collection<Versioned<byte[]>> values(Commit<Void> commit) {
+        return tree.values().stream().map(this::toVersioned)
+                .collect(Collectors.toList());
+    }
+
+    protected Set<Map.Entry<String, Versioned<byte[]>>> entrySet(Commit<Void> commit) {
+        return tree
+                .entrySet()
+                .stream()
+                .map(e -> Maps.immutableEntry(e.getKey(),
+                        toVersioned(e.getValue())))
+                .collect(Collectors.toSet());
+    }
+
+    protected MapEntryUpdateResult<String, byte[]> updateAndGet(Commit<? extends UpdateAndGet> commit) {
+        Status updateStatus = validate(commit.value());
+        String key = commit.value().key();
+        TreeMapEntryValue oldCommitValue = tree.get(commit.value().key());
+        Versioned<byte[]> oldTreeValue = toVersioned(oldCommitValue);
+
+        if (updateStatus != Status.OK) {
+            return new MapEntryUpdateResult<>(updateStatus, "", key,
+                    oldTreeValue, oldTreeValue);
+        }
+
+        byte[] newValue = commit.value().value();
+        long newVersion = commit.index();
+        Versioned<byte[]> newTreeValue = newValue == null ? null
+                : new Versioned<byte[]>(newValue, newVersion);
+
+        MapEvent.Type updateType = newValue == null ? MapEvent.Type.REMOVE
+                : oldCommitValue == null ? MapEvent.Type.INSERT :
+                MapEvent.Type.UPDATE;
+        if (updateType == MapEvent.Type.REMOVE ||
+                updateType == MapEvent.Type.UPDATE) {
+            tree.remove(key);
+        }
+        if (updateType == MapEvent.Type.INSERT ||
+                updateType == MapEvent.Type.UPDATE) {
+            tree.put(key, new TreeMapEntryValue(newVersion, commit.value().value()));
+        }
+        publish(Lists.newArrayList(new MapEvent<>("", key, newTreeValue,
+                oldTreeValue)));
+        return new MapEntryUpdateResult<>(updateStatus, "", key, oldTreeValue,
+                newTreeValue);
+    }
+
+    protected Status clear(Commit<Void> commit) {
+        Iterator<Map.Entry<String, TreeMapEntryValue>> iterator = tree
+                .entrySet()
+                .iterator();
+        while (iterator.hasNext()) {
+            Map.Entry<String, TreeMapEntryValue> entry = iterator.next();
+            String key = entry.getKey();
+            TreeMapEntryValue value = entry.getValue();
+            Versioned<byte[]> removedValue =
+                    new Versioned<byte[]>(value.value(),
+                            value.version());
+            publish(Lists.newArrayList(new MapEvent<>("", key, null,
+                    removedValue)));
+            iterator.remove();
+        }
+        return Status.OK;
+    }
+
+    protected void listen(Commit<Void> commit) {
+        listeners.put(commit.session().sessionId().id(), commit.session());
+    }
+
+    protected void unlisten(Commit<Void> commit) {
+        closeListener(commit.session().sessionId().id());
+    }
+
+    private Status validate(UpdateAndGet update) {
+        TreeMapEntryValue existingValue = tree.get(update.key());
+        if (existingValue == null && update.value() == null) {
+            return Status.NOOP;
+        }
+        if (preparedKeys.contains(update.key())) {
+            return Status.WRITE_LOCK;
+        }
+        byte[] existingRawValue = existingValue == null ? null :
+                existingValue.value();
+        Long existingVersion = existingValue == null ? null :
+                existingValue.version();
+        return update.valueMatch().matches(existingRawValue)
+                && update.versionMatch().matches(existingVersion) ?
+                Status.OK
+                : Status.PRECONDITION_FAILED;
+    }
+
+    protected NavigableMap<String, TreeMapEntryValue> 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, TreeMapEntryValue> subMap = commit.value();
+        return tree.subMap(subMap.fromKey(), subMap.isInclusiveFrom(),
+                subMap.toKey(), subMap.isInclusiveTo());
+    }
+
+    protected String firstKey(Commit<Void> commit) {
+        if (tree.isEmpty()) {
+            return null;
+        }
+        return tree.firstKey();
+    }
+
+    protected String lastKey(Commit<Void> commit) {
+        return tree.isEmpty() ? null : tree.lastKey();
+    }
+
+    protected Map.Entry<String, Versioned<byte[]>> higherEntry(Commit<? extends HigherEntry> commit) {
+        if (tree.isEmpty()) {
+            return null;
+        }
+        return toVersionedEntry(
+                tree.higherEntry(commit.value().key()));
+    }
+
+    protected Map.Entry<String, Versioned<byte[]>> firstEntry(Commit<Void> commit) {
+        if (tree.isEmpty()) {
+            return null;
+        }
+        return toVersionedEntry(tree.firstEntry());
+    }
+
+    protected Map.Entry<String, Versioned<byte[]>> lastEntry(Commit<Void> commit) {
+        if (tree.isEmpty()) {
+            return null;
+        }
+        return toVersionedEntry(tree.lastEntry());
+    }
+
+    protected Map.Entry<String, Versioned<byte[]>> pollFirstEntry(Commit<Void> commit) {
+        return toVersionedEntry(tree.pollFirstEntry());
+    }
+
+    protected Map.Entry<String, Versioned<byte[]>> pollLastEntry(Commit<Void> commit) {
+        return toVersionedEntry(tree.pollLastEntry());
+    }
+
+    protected Map.Entry<String, Versioned<byte[]>> lowerEntry(Commit<? extends LowerEntry> commit) {
+        return toVersionedEntry(tree.lowerEntry(commit.value().key()));
+    }
+
+    protected String lowerKey(Commit<? extends LowerKey> commit) {
+        return tree.lowerKey(commit.value().key());
+    }
+
+    protected Map.Entry<String, Versioned<byte[]>> floorEntry(Commit<? extends FloorEntry> commit) {
+        return toVersionedEntry(tree.floorEntry(commit.value().key()));
+    }
+
+    protected String floorKey(Commit<? extends FloorKey> commit) {
+        return tree.floorKey(commit.value().key());
+    }
+
+    protected Map.Entry<String, Versioned<byte[]>> ceilingEntry(Commit<CeilingEntry> commit) {
+        return toVersionedEntry(
+                tree.ceilingEntry(commit.value().key()));
+    }
+
+    protected String ceilingKey(Commit<CeilingKey> commit) {
+        return tree.ceilingKey(commit.value().key());
+    }
+
+    protected String higherKey(Commit<HigherKey> commit) {
+        return tree.higherKey(commit.value().key());
+    }
+
+    private Versioned<byte[]> toVersioned(TreeMapEntryValue value) {
+        return value == null ? null :
+                new Versioned<byte[]>(value.value(), value.version());
+    }
+
+    private Map.Entry<String, Versioned<byte[]>> toVersionedEntry(
+            Map.Entry<String, TreeMapEntryValue> entry) {
+        //FIXME is this the best type of entry to return?
+        return entry == null ? null : new SimpleImmutableEntry<>(
+                entry.getKey(), toVersioned(entry.getValue()));
+    }
+
+    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);
+    }
+
+    private static class TreeMapEntryValue {
+        private final long version;
+        private final byte[] value;
+
+        public TreeMapEntryValue(long version, byte[] value) {
+            this.version = version;
+            this.value = value;
+        }
+
+        public byte[] value() {
+            return value;
+        }
+
+        public long version() {
+            return version;
+        }
+    }
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapState.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapState.java
deleted file mode 100644
index 576bfea..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixConsistentTreeMapState.java
+++ /dev/null
@@ -1,575 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Laboratory
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.onosproject.store.primitives.resources.impl;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import io.atomix.copycat.server.Commit;
-import io.atomix.copycat.server.StateMachineExecutor;
-import io.atomix.copycat.server.session.ServerSession;
-import io.atomix.copycat.server.session.SessionListener;
-import io.atomix.resource.ResourceStateMachine;
-import org.onlab.util.Match;
-import org.onosproject.store.service.MapEvent;
-import org.onosproject.store.service.Versioned;
-
-import java.util.AbstractMap.SimpleImmutableEntry;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.NavigableMap;
-import java.util.Properties;
-import java.util.Set;
-import java.util.TreeMap;
-import java.util.function.Function;
-import java.util.stream.Collectors;
-
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.CeilingEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.CeilingKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.Clear;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.ContainsKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.ContainsValue;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.EntrySet;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.FirstEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.FirstKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.FloorEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.FloorKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.Get;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.GetOrDefault;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.HigherEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.HigherKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.IsEmpty;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.KeySet;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.LastEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.LastKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.Listen;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.LowerEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.LowerKey;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.PollFirstEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.PollLastEntry;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.Size;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.SubMap;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.Unlisten;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.UpdateAndGet;
-import static org.onosproject.store.primitives.resources.impl.AtomixConsistentTreeMapCommands.Values;
-import static org.onosproject.store.primitives.resources.impl.MapEntryUpdateResult.*;
-
-/**
- * State machine corresponding to {@link AtomixConsistentTreeMap} backed by a
- * {@link TreeMap}.
- */
-public class AtomixConsistentTreeMapState extends ResourceStateMachine implements SessionListener {
-
-    private final Map<Long, Commit<? extends Listen>> listeners =
-            Maps.newHashMap();
-    private TreeMap<String, TreeMapEntryValue> tree = Maps.newTreeMap();
-    private final Set<String> preparedKeys = Sets.newHashSet();
-
-    private Function<Commit<SubMap>, NavigableMap<String, TreeMapEntryValue>> subMapFunction = this::subMap;
-    private Function<Commit<FirstKey>, String> firstKeyFunction = this::firstKey;
-    private Function<Commit<LastKey>, String> lastKeyFunction = this::lastKey;
-    private Function<Commit<HigherEntry>, Map.Entry<String, Versioned<byte[]>>> higherEntryFunction =
-            this::higherEntry;
-    private Function<Commit<FirstEntry>, Map.Entry<String, Versioned<byte[]>>> firstEntryFunction =
-            this::firstEntry;
-    private Function<Commit<LastEntry>, Map.Entry<String, Versioned<byte[]>>> lastEntryFunction =
-            this::lastEntry;
-    private Function<Commit<PollFirstEntry>, Map.Entry<String, Versioned<byte[]>>> pollFirstEntryFunction =
-            this::pollFirstEntry;
-    private Function<Commit<PollLastEntry>, Map.Entry<String, Versioned<byte[]>>> pollLastEntryFunction =
-            this::pollLastEntry;
-    private Function<Commit<LowerEntry>, Map.Entry<String, Versioned<byte[]>>> lowerEntryFunction =
-            this::lowerEntry;
-    private Function<Commit<LowerKey>, String> lowerKeyFunction = this::lowerKey;
-    private Function<Commit<FloorEntry>, Map.Entry<String, Versioned<byte[]>>> floorEntryFunction =
-            this::floorEntry;
-    private Function<Commit<CeilingEntry>, Map.Entry<String, Versioned<byte[]>>> ceilingEntryFunction =
-            this::ceilingEntry;
-    private Function<Commit<FloorKey>, String> floorKeyFunction = this::floorKey;
-    private Function<Commit<CeilingKey>, String> ceilingKeyFunction = this::ceilingKey;
-    private Function<Commit<HigherKey>, String> higherKeyFunction = this::higherKey;
-
-    public AtomixConsistentTreeMapState(Properties properties) {
-        super(properties);
-    }
-
-    @Override
-    public void configure(StateMachineExecutor executor) {
-        // Listeners
-        executor.register(Listen.class, this::listen);
-        executor.register(Unlisten.class, this::unlisten);
-        // Queries
-        executor.register(ContainsKey.class, this::containsKey);
-        executor.register(ContainsValue.class, this::containsValue);
-        executor.register(EntrySet.class, this::entrySet);
-        executor.register(Get.class, this::get);
-        executor.register(GetOrDefault.class, this::getOrDefault);
-        executor.register(IsEmpty.class, this::isEmpty);
-        executor.register(KeySet.class, this::keySet);
-        executor.register(Size.class, this::size);
-        executor.register(Values.class, this::values);
-        executor.register(SubMap.class, subMapFunction);
-        executor.register(FirstKey.class, firstKeyFunction);
-        executor.register(LastKey.class, lastKeyFunction);
-        executor.register(FirstEntry.class, firstEntryFunction);
-        executor.register(LastEntry.class, lastEntryFunction);
-        executor.register(PollFirstEntry.class, pollFirstEntryFunction);
-        executor.register(PollLastEntry.class, pollLastEntryFunction);
-        executor.register(LowerEntry.class, lowerEntryFunction);
-        executor.register(LowerKey.class, lowerKeyFunction);
-        executor.register(FloorEntry.class, floorEntryFunction);
-        executor.register(FloorKey.class, floorKeyFunction);
-        executor.register(CeilingEntry.class, ceilingEntryFunction);
-        executor.register(CeilingKey.class, ceilingKeyFunction);
-        executor.register(HigherEntry.class, higherEntryFunction);
-        executor.register(HigherKey.class, higherKeyFunction);
-
-        // Commands
-        executor.register(UpdateAndGet.class, this::updateAndGet);
-        executor.register(Clear.class, this::clear);
-    }
-
-    @Override
-    public void delete() {
-        listeners.values().forEach(Commit::close);
-        listeners.clear();
-        tree.values().forEach(TreeMapEntryValue::discard);
-        tree.clear();
-    }
-
-    protected boolean containsKey(Commit<? extends ContainsKey> commit) {
-        try {
-            return toVersioned(tree.get((commit.operation().key()))) != null;
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected boolean containsValue(Commit<? extends ContainsValue> commit) {
-        try {
-            Match<byte[]> valueMatch = Match
-                    .ifValue(commit.operation().value());
-            return tree.values().stream().anyMatch(
-                    value -> valueMatch.matches(value.value()));
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected Versioned<byte[]> get(Commit<? extends Get> commit) {
-        try {
-            return toVersioned(tree.get(commit.operation().key()));
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected Versioned<byte[]> getOrDefault(Commit<? extends GetOrDefault> commit) {
-        try {
-            Versioned<byte[]> value = toVersioned(tree.get(commit.operation().key()));
-            return value != null ? value : new Versioned<>(commit.operation().defaultValue(), 0);
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected int size(Commit<? extends Size> commit) {
-        try {
-            return tree.size();
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected boolean isEmpty(Commit<? extends IsEmpty> commit) {
-        try {
-            return tree.isEmpty();
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected Set<String> keySet(Commit<? extends KeySet> commit) {
-        try {
-            return tree.keySet().stream().collect(Collectors.toSet());
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected Collection<Versioned<byte[]>> values(
-            Commit<? extends Values> commit) {
-        try {
-            return tree.values().stream().map(this::toVersioned)
-                    .collect(Collectors.toList());
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected Set<Map.Entry<String, Versioned<byte[]>>> entrySet(
-            Commit<? extends EntrySet> commit) {
-        try {
-            return tree
-                    .entrySet()
-                    .stream()
-                    .map(e -> Maps.immutableEntry(e.getKey(),
-                                                  toVersioned(e.getValue())))
-                    .collect(Collectors.toSet());
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected MapEntryUpdateResult<String, byte[]> updateAndGet(
-            Commit<? extends UpdateAndGet> commit) {
-        Status updateStatus = validate(commit.operation());
-        String key = commit.operation().key();
-        TreeMapEntryValue oldCommitValue = tree.get(commit.operation().key());
-        Versioned<byte[]> oldTreeValue = toVersioned(oldCommitValue);
-
-        if (updateStatus != Status.OK) {
-            commit.close();
-            return new MapEntryUpdateResult<>(updateStatus, "", key,
-                                                  oldTreeValue, oldTreeValue);
-        }
-
-        byte[] newValue = commit.operation().value();
-        long newVersion = commit.index();
-        Versioned<byte[]> newTreeValue = newValue == null ? null
-                : new Versioned<byte[]>(newValue, newVersion);
-
-        MapEvent.Type updateType = newValue == null ? MapEvent.Type.REMOVE
-                : oldCommitValue == null ? MapEvent.Type.INSERT :
-                MapEvent.Type.UPDATE;
-        if (updateType == MapEvent.Type.REMOVE ||
-                updateType == MapEvent.Type.UPDATE) {
-            tree.remove(key);
-            oldCommitValue.discard();
-        }
-        if (updateType == MapEvent.Type.INSERT ||
-                updateType == MapEvent.Type.UPDATE) {
-            tree.put(key, new NonTransactionalCommit(newVersion, commit));
-        } else {
-            commit.close();
-        }
-        publish(Lists.newArrayList(new MapEvent<>("", key, newTreeValue,
-                                                  oldTreeValue)));
-        return new MapEntryUpdateResult<>(updateStatus, "", key, oldTreeValue,
-                                          newTreeValue);
-    }
-
-    protected Status clear(
-            Commit<? extends Clear> commit) {
-        try {
-            Iterator<Map.Entry<String, TreeMapEntryValue>> iterator = tree
-                    .entrySet()
-                    .iterator();
-            while (iterator.hasNext()) {
-                Map.Entry<String, TreeMapEntryValue> entry = iterator.next();
-                String key = entry.getKey();
-                TreeMapEntryValue value = entry.getValue();
-                Versioned<byte[]> removedValue =
-                        new Versioned<byte[]>(value.value(),
-                                              value.version());
-                publish(Lists.newArrayList(new MapEvent<>("", key, null,
-                                                          removedValue)));
-                value.discard();
-                iterator.remove();
-            }
-            return Status.OK;
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected void listen(
-            Commit<? extends Listen> commit) {
-        Long sessionId = commit.session().id();
-        listeners.put(sessionId, commit);
-        commit.session()
-                .onStateChange(
-                        state -> {
-                            if (state == ServerSession.State.CLOSED
-                                    || state == ServerSession.State.EXPIRED) {
-                                Commit<? extends Listen> listener =
-                                        listeners.remove(sessionId);
-                                if (listener != null) {
-                                    listener.close();
-                                }
-                            }
-                        });
-    }
-
-    protected void unlisten(
-            Commit<? extends Unlisten> commit) {
-        try {
-            Commit<? extends AtomixConsistentTreeMapCommands.Listen> listener =
-                    listeners.remove(commit.session().id());
-            if (listener != null) {
-                listener.close();
-            }
-        } finally {
-            commit.close();
-        }
-    }
-
-    private Status validate(UpdateAndGet update) {
-        TreeMapEntryValue existingValue = tree.get(update.key());
-        if (existingValue == null && update.value() == null) {
-            return Status.NOOP;
-        }
-        if (preparedKeys.contains(update.key())) {
-            return Status.WRITE_LOCK;
-        }
-        byte[] existingRawValue = existingValue == null ? null :
-                existingValue.value();
-        Long existingVersion = existingValue == null ? null :
-                existingValue.version();
-        return update.valueMatch().matches(existingRawValue)
-                && update.versionMatch().matches(existingVersion) ?
-                Status.OK
-                : Status.PRECONDITION_FAILED;
-    }
-
-    protected NavigableMap<String, TreeMapEntryValue> subMap(
-            Commit<? extends SubMap> commit) {
-        //Do not support this until lazy communication is possible.  At present
-        // it transmits up to the entire map.
-        try {
-            SubMap<String, TreeMapEntryValue> subMap = commit.operation();
-            return tree.subMap(subMap.fromKey(), subMap.isInclusiveFrom(),
-                               subMap.toKey(), subMap.isInclusiveTo());
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected String firstKey(Commit<? extends FirstKey> commit) {
-        try {
-            if (tree.isEmpty()) {
-                return null;
-            }
-            return tree.firstKey();
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected String lastKey(Commit<? extends LastKey> commit) {
-        try {
-            return tree.isEmpty() ? null : tree.lastKey();
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected Map.Entry<String, Versioned<byte[]>> higherEntry(
-            Commit<? extends HigherEntry> commit) {
-        try {
-            if (tree.isEmpty()) {
-                return null;
-            }
-            return toVersionedEntry(
-                    tree.higherEntry(commit.operation().key()));
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected Map.Entry<String, Versioned<byte[]>> firstEntry(
-            Commit<? extends FirstEntry> commit) {
-        try {
-            if (tree.isEmpty()) {
-                return null;
-            }
-            return toVersionedEntry(tree.firstEntry());
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected Map.Entry<String, Versioned<byte[]>> lastEntry(
-            Commit<? extends LastEntry> commit) {
-        try {
-            if (tree.isEmpty()) {
-                return null;
-            }
-            return toVersionedEntry(tree.lastEntry());
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected Map.Entry<String, Versioned<byte[]>> pollFirstEntry(
-            Commit<? extends PollFirstEntry> commit) {
-        try {
-            return toVersionedEntry(tree.pollFirstEntry());
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected Map.Entry<String, Versioned<byte[]>> pollLastEntry(
-            Commit<? extends PollLastEntry> commit) {
-        try {
-            return toVersionedEntry(tree.pollLastEntry());
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected Map.Entry<String, Versioned<byte[]>> lowerEntry(
-            Commit<? extends LowerEntry> commit) {
-        try {
-            return toVersionedEntry(tree.lowerEntry(commit.operation().key()));
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected String lowerKey(Commit<? extends LowerKey> commit) {
-        try {
-            return tree.lowerKey(commit.operation().key());
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected Map.Entry<String, Versioned<byte[]>> floorEntry(
-            Commit<? extends FloorEntry> commit) {
-        try {
-            return toVersionedEntry(tree.floorEntry(commit.operation().key()));
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected String floorKey(Commit<? extends FloorKey> commit) {
-        try {
-            return tree.floorKey(commit.operation().key());
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected Map.Entry<String, Versioned<byte[]>> ceilingEntry(
-            Commit<CeilingEntry> commit) {
-        try {
-            return toVersionedEntry(
-                    tree.ceilingEntry(commit.operation().key()));
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected String ceilingKey(Commit<CeilingKey> commit) {
-        try {
-            return tree.ceilingKey(commit.operation().key());
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected String higherKey(Commit<HigherKey> commit) {
-        try {
-            return tree.higherKey(commit.operation().key());
-        } finally {
-            commit.close();
-        }
-    }
-
-    private Versioned<byte[]> toVersioned(TreeMapEntryValue value) {
-        return value == null ? null :
-                new Versioned<byte[]>(value.value(), value.version());
-    }
-
-    private Map.Entry<String, Versioned<byte[]>> toVersionedEntry(
-            Map.Entry<String, TreeMapEntryValue> entry) {
-        //FIXME is this the best type of entry to return?
-        return entry == null ? null : new SimpleImmutableEntry<>(
-                entry.getKey(), toVersioned(entry.getValue()));
-    }
-
-    private void publish(List<MapEvent<String, byte[]>> events) {
-        listeners.values().forEach(commit -> commit.session()
-                .publish(AtomixConsistentTreeMap.CHANGE_SUBJECT, events));
-    }
-
-    @Override
-    public void register(ServerSession session) {
-    }
-
-    @Override
-    public void unregister(ServerSession session) {
-        closeListener(session.id());
-    }
-
-    @Override
-    public void expire(ServerSession session) {
-        closeListener(session.id());
-    }
-
-    @Override
-    public void close(ServerSession session) {
-        closeListener(session.id());
-    }
-
-    private void closeListener(Long sessionId) {
-        Commit<? extends Listen> commit = listeners.remove(sessionId);
-        if (commit != null) {
-            commit.close();
-        }
-    }
-
-    private interface TreeMapEntryValue {
-
-        byte[] value();
-
-        long version();
-
-        void discard();
-    }
-
-    private class NonTransactionalCommit implements TreeMapEntryValue {
-        private final long version;
-        private final Commit<? extends UpdateAndGet> commit;
-
-        public NonTransactionalCommit(long version,
-                                      Commit<? extends UpdateAndGet> commit) {
-            this.version = version;
-            this.commit = commit;
-        }
-
-        @Override
-        public byte[] value() {
-            return commit.operation().value();
-        }
-
-        @Override
-        public long version() {
-            return version;
-        }
-
-        @Override
-        public void discard() {
-            commit.close();
-        }
-    }
-}
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
index e670d40..954008c 100644
--- 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
@@ -15,63 +15,76 @@
  */
 package org.onosproject.store.primitives.resources.impl;
 
-import io.atomix.variables.DistributedLong;
-
 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;
 
 /**
- * {@code AsyncAtomicCounter} implementation backed by Atomix
- * {@link DistributedLong}.
+ * Atomix counter implementation.
  */
-public class AtomixCounter implements AsyncAtomicCounter {
+public class AtomixCounter extends AbstractRaftPrimitive implements AsyncAtomicCounter {
+    private static final Serializer SERIALIZER = Serializer.using(KryoNamespace.newBuilder()
+            .register(KryoNamespaces.BASIC)
+            .register(AtomixCounterOperations.NAMESPACE)
+            .build());
 
-    private final String name;
-    private final DistributedLong distLong;
-
-    public AtomixCounter(String name, DistributedLong distLong) {
-        this.name = name;
-        this.distLong = distLong;
+    public AtomixCounter(RaftProxy proxy) {
+        super(proxy);
     }
 
-    @Override
-    public String name() {
-        return name;
-    }
-
-    @Override
-    public CompletableFuture<Long> incrementAndGet() {
-        return distLong.incrementAndGet();
-    }
-
-    @Override
-    public CompletableFuture<Long> getAndIncrement() {
-        return distLong.getAndIncrement();
-    }
-
-    @Override
-    public CompletableFuture<Long> getAndAdd(long delta) {
-        return distLong.getAndAdd(delta);
-    }
-
-    @Override
-    public CompletableFuture<Long> addAndGet(long delta) {
-        return distLong.addAndGet(delta);
+    private long nullOrZero(Long value) {
+        return value != null ? value : 0;
     }
 
     @Override
     public CompletableFuture<Long> get() {
-        return distLong.get();
+        return proxy.<Long>invoke(GET, SERIALIZER::decode).thenApply(this::nullOrZero);
     }
 
     @Override
     public CompletableFuture<Void> set(long value) {
-        return distLong.set(value);
+        return proxy.invoke(SET, SERIALIZER::encode, new Set(value));
     }
 
     @Override
     public CompletableFuture<Boolean> compareAndSet(long expectedValue, long updateValue) {
-        return distLong.compareAndSet(expectedValue, 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
new file mode 100644
index 0000000..0ebe7e4
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixCounterOperations.java
@@ -0,0 +1,188 @@
+/*
+ * Copyright 2017-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.store.primitives.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("set", OperationType.COMMAND),
+    COMPARE_AND_SET("compareAndSet", OperationType.COMMAND),
+    INCREMENT_AND_GET("incrementAndGet", OperationType.COMMAND),
+    GET_AND_INCREMENT("getAndIncrement", OperationType.COMMAND),
+    ADD_AND_GET("addAndGet", OperationType.COMMAND),
+    GET_AND_ADD("getAndAdd", OperationType.COMMAND),
+    GET("get", OperationType.QUERY);
+
+    private final String id;
+    private final OperationType type;
+
+    AtomixCounterOperations(String id, OperationType type) {
+        this.id = id;
+        this.type = type;
+    }
+
+    @Override
+    public String id() {
+        return id;
+    }
+
+    @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
new file mode 100644
index 0000000..a802393
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixCounterService.java
@@ -0,0 +1,153 @@
+/*
+ * Copyright 2017-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.store.primitives.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/AtomixDocumentTree.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTree.java
index 3875e55..1730fe8 100644
--- 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
@@ -16,70 +16,65 @@
 
 package org.onosproject.store.primitives.resources.impl;
 
-import static com.google.common.base.Preconditions.checkNotNull;
-import static org.onosproject.store.primitives.resources.impl.DocumentTreeUpdateResult.Status.ILLEGAL_MODIFICATION;
-import static org.onosproject.store.primitives.resources.impl.DocumentTreeUpdateResult.Status.INVALID_PATH;
-import static org.onosproject.store.primitives.resources.impl.DocumentTreeUpdateResult.Status.OK;
-import io.atomix.copycat.client.CopycatClient;
-import io.atomix.resource.AbstractResource;
-import io.atomix.resource.ResourceTypeInfo;
-
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
-import java.util.Properties;
 import java.util.concurrent.CompletableFuture;
 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.resources.impl.AtomixDocumentTreeCommands.Clear;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeCommands.Get;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeCommands.GetChildren;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeCommands.Listen;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeCommands.Unlisten;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeCommands.Update;
+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.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.Versioned;
 
-import com.google.common.util.concurrent.MoreExecutors;
+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.ADD_LISTENER;
+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.REMOVE_LISTENER;
+import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.UPDATE;
+import static org.onosproject.store.primitives.resources.impl.DocumentTreeUpdateResult.Status.ILLEGAL_MODIFICATION;
+import static org.onosproject.store.primitives.resources.impl.DocumentTreeUpdateResult.Status.INVALID_PATH;
+import static org.onosproject.store.primitives.resources.impl.DocumentTreeUpdateResult.Status.OK;
 
 /**
  * Distributed resource providing the {@link AsyncDocumentTree} primitive.
  */
-@ResourceTypeInfo(id = -156, factory = AtomixDocumentTreeFactory.class)
-public class AtomixDocumentTree extends AbstractResource<AtomixDocumentTree>
-    implements AsyncDocumentTree<byte[]> {
+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 HashMap<>();
-    public static final String CHANGE_SUBJECT = "changeEvents";
 
-    protected AtomixDocumentTree(CopycatClient client, Properties options) {
-        super(client, options);
-    }
-
-    @Override
-    public CompletableFuture<AtomixDocumentTree> open() {
-        return super.open().thenApply(result -> {
-            client.onStateChange(state -> {
-                if (state == CopycatClient.State.CONNECTED && isListening()) {
-                    client.submit(new Listen());
-                }
-            });
-            client.onEvent(CHANGE_SUBJECT, this::processTreeUpdates);
-            return result;
+    public AtomixDocumentTree(RaftProxy proxy) {
+        super(proxy);
+        proxy.addStateChangeListener(state -> {
+            if (state == RaftProxy.State.CONNECTED && isListening()) {
+                proxy.invoke(ADD_LISTENER, SERIALIZER::encode, new Listen());
+            }
         });
-    }
-
-    @Override
-    public String name() {
-        return null;
+        proxy.addEventListener(CHANGE, SERIALIZER::decode, this::processTreeUpdates);
     }
 
     @Override
@@ -89,7 +84,7 @@
 
     @Override
     public CompletableFuture<Void> destroy() {
-        return client.submit(new Clear());
+        return proxy.invoke(CLEAR);
     }
 
     @Override
@@ -99,17 +94,20 @@
 
     @Override
     public CompletableFuture<Map<String, Versioned<byte[]>>> getChildren(DocumentPath path) {
-        return client.submit(new GetChildren(checkNotNull(path)));
+        return proxy.invoke(GET_CHILDREN, SERIALIZER::encode, new GetChildren(checkNotNull(path)), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Versioned<byte[]>> get(DocumentPath path) {
-        return client.submit(new Get(checkNotNull(path)));
+        return proxy.invoke(GET, SERIALIZER::encode, new Get(checkNotNull(path)), SERIALIZER::decode);
     }
 
     @Override
     public CompletableFuture<Versioned<byte[]>> set(DocumentPath path, byte[] value) {
-        return client.submit(new Update(checkNotNull(path), Optional.ofNullable(value), Match.any(), Match.any()))
+        return proxy.<Update, DocumentTreeUpdateResult<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());
@@ -138,7 +136,7 @@
                 .thenCompose(status -> {
                     if (status == ILLEGAL_MODIFICATION) {
                         return createRecursive(path.parent(), null)
-                                    .thenCompose(r -> createInternal(path, value).thenApply(v -> true));
+                                .thenCompose(r -> createInternal(path, value).thenApply(v -> true));
                     }
                     return CompletableFuture.completedFuture(status == OK);
                 });
@@ -146,19 +144,24 @@
 
     @Override
     public CompletableFuture<Boolean> replace(DocumentPath path, byte[] newValue, long version) {
-        return client.submit(new Update(checkNotNull(path),
-                                        Optional.ofNullable(newValue),
-                                        Match.any(),
-                                        Match.ifValue(version)))
+        return proxy.<Update, DocumentTreeUpdateResult<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 client.submit(new Update(checkNotNull(path),
-                                        Optional.ofNullable(newValue),
-                                        Match.ifValue(currentValue),
-                                        Match.any()))
+        return proxy.<Update, DocumentTreeUpdateResult<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());
@@ -175,7 +178,10 @@
         if (path.equals(DocumentPath.from("root"))) {
             return Tools.exceptionalFuture(new IllegalDocumentModificationException());
         }
-        return client.submit(new Update(checkNotNull(path), null, Match.any(), Match.ifNotNull()))
+        return proxy.<Update, DocumentTreeUpdateResult<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());
@@ -194,8 +200,8 @@
         InternalListener internalListener = new InternalListener(path, listener, MoreExecutors.directExecutor());
         // TODO: Support API that takes an executor
         if (!eventListeners.containsKey(listener)) {
-            return client.submit(new Listen(path))
-                         .thenRun(() -> eventListeners.put(listener, internalListener));
+            return proxy.invoke(ADD_LISTENER, SERIALIZER::encode, new Listen(path))
+                    .thenRun(() -> eventListeners.put(listener, internalListener));
         }
         return CompletableFuture.completedFuture(null);
     }
@@ -205,14 +211,18 @@
         checkNotNull(listener);
         InternalListener internalListener = eventListeners.remove(listener);
         if  (internalListener != null && eventListeners.isEmpty()) {
-            return client.submit(new Unlisten(internalListener.path)).thenApply(v -> null);
+            return proxy.invoke(REMOVE_LISTENER, SERIALIZER::encode, new Unlisten(internalListener.path))
+                    .thenApply(v -> null);
         }
         return CompletableFuture.completedFuture(null);
     }
 
     private CompletableFuture<DocumentTreeUpdateResult.Status> createInternal(DocumentPath path, byte[] value) {
-        return client.submit(new Update(checkNotNull(path), Optional.ofNullable(value), Match.any(), Match.ifNull()))
-                     .thenApply(result -> result.status());
+        return proxy.<Update, DocumentTreeUpdateResult<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() {
@@ -242,4 +252,4 @@
             }
         }
     }
-}
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeCommands.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeCommands.java
deleted file mode 100644
index 153dbcb..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeCommands.java
+++ /dev/null
@@ -1,292 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Laboratory
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.onosproject.store.primitives.resources.impl;
-
-import io.atomix.catalyst.buffer.BufferInput;
-import io.atomix.catalyst.buffer.BufferOutput;
-import io.atomix.catalyst.serializer.CatalystSerializable;
-import io.atomix.catalyst.serializer.SerializableTypeResolver;
-import io.atomix.catalyst.serializer.Serializer;
-import io.atomix.catalyst.serializer.SerializerRegistry;
-import io.atomix.copycat.Command;
-import io.atomix.copycat.Query;
-
-import java.util.Map;
-import java.util.Optional;
-
-import org.onlab.util.Match;
-import org.onosproject.store.service.DocumentPath;
-import org.onosproject.store.service.Versioned;
-
-import com.google.common.base.MoreObjects;
-
-/**
- * {@link AtomixDocumentTree} resource state machine operations.
- */
-public class AtomixDocumentTreeCommands {
-
-    /**
-     * Abstract DocumentTree operation.
-     */
-    public abstract static class DocumentTreeOperation<V> implements CatalystSerializable {
-
-        private DocumentPath path;
-
-        DocumentTreeOperation(DocumentPath path) {
-            this.path = path;
-        }
-
-        public DocumentPath path() {
-            return path;
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            serializer.writeObject(path, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            path = serializer.readObject(buffer);
-        }
-    }
-
-    /**
-     * Abstract DocumentTree query.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class DocumentTreeQuery<V> extends DocumentTreeOperation<V> implements Query<V> {
-
-         DocumentTreeQuery(DocumentPath path) {
-             super(path);
-        }
-
-         @Override
-         public ConsistencyLevel consistency() {
-           return ConsistencyLevel.SEQUENTIAL;
-         }
-    }
-
-    /**
-     * Abstract DocumentTree command.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class DocumentTreeCommand<V> extends DocumentTreeOperation<V> implements Command<V> {
-
-        DocumentTreeCommand(DocumentPath path) {
-             super(path);
-        }
-    }
-
-    /**
-     * DocumentTree#get query.
-     */
-    @SuppressWarnings("serial")
-    public static class Get extends DocumentTreeQuery<Versioned<byte[]>> {
-        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 DocumentTreeQuery<Map<String, Versioned<byte[]>>> {
-        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 DocumentTreeCommand<DocumentTreeUpdateResult<byte[]>> {
-
-        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 void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            serializer.writeObject(value, buffer);
-            serializer.writeObject(valueMatch, buffer);
-            serializer.writeObject(versionMatch, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            value = serializer.readObject(buffer);
-            valueMatch = serializer.readObject(buffer);
-            versionMatch = serializer.readObject(buffer);
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return value == null ? CompactionMode.TOMBSTONE : CompactionMode.QUORUM;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("path", path())
-                    .add("value", value)
-                    .add("valueMatch", valueMatch)
-                    .add("versionMatch", versionMatch)
-                    .toString();
-        }
-    }
-
-    /**
-     * Clear command.
-     */
-    @SuppressWarnings("serial")
-    public static class Clear implements Command<Void>, CatalystSerializable {
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.TOMBSTONE;
-        }
-    }
-
-    /**
-     * Change listen.
-     */
-    @SuppressWarnings("serial")
-    public static class Listen extends DocumentTreeCommand<Void> {
-
-        public Listen() {
-            this(DocumentPath.from("root"));
-        }
-
-        public Listen(DocumentPath path) {
-            super(path);
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.QUORUM;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("path", path())
-                    .toString();
-        }
-    }
-
-    /**
-     * Change unlisten.
-     */
-    @SuppressWarnings("serial")
-    public static class Unlisten extends DocumentTreeCommand<Void> {
-
-        public Unlisten() {
-            this(DocumentPath.from("root"));
-        }
-
-        public Unlisten(DocumentPath path) {
-            super(path);
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.TOMBSTONE;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("path", path())
-                    .toString();
-        }
-    }
-
-    /**
-     * DocumentTree command type resolver.
-     */
-    public static class TypeResolver implements SerializableTypeResolver {
-        @Override
-        public void resolve(SerializerRegistry registry) {
-            registry.register(Get.class, -911);
-            registry.register(GetChildren.class, -912);
-            registry.register(Update.class, -913);
-            registry.register(Listen.class, -914);
-            registry.register(Unlisten.class, -915);
-            registry.register(Clear.class, -916);
-        }
-    }
-}
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
new file mode 100644
index 0000000..8aa23ce
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeEvents.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2017-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.store.primitives.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("change");
+
+    private final String id;
+
+    AtomixDocumentTreeEvents(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(DocumentTreeEvent.class)
+            .register(DocumentTreeEvent.Type.class)
+            .build("AtomixDocumentTreeEvents");
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeFactory.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeFactory.java
deleted file mode 100644
index 4282566..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeFactory.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Laboratory
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.onosproject.store.primitives.resources.impl;
-
-import io.atomix.catalyst.serializer.SerializableTypeResolver;
-import io.atomix.copycat.client.CopycatClient;
-import io.atomix.resource.ResourceFactory;
-import io.atomix.resource.ResourceStateMachine;
-
-import java.util.Properties;
-
-/**
- * {@link AtomixDocumentTree} resource factory.
- *
- */
-public class AtomixDocumentTreeFactory implements ResourceFactory<AtomixDocumentTree> {
-
-    @Override
-    public SerializableTypeResolver createSerializableTypeResolver() {
-      return new AtomixDocumentTreeCommands.TypeResolver();
-    }
-
-    @Override
-    public ResourceStateMachine createStateMachine(Properties config) {
-      return new AtomixDocumentTreeState(config);
-    }
-
-    @Override
-    public AtomixDocumentTree createInstance(CopycatClient client, Properties options) {
-      return new AtomixDocumentTree(client, options);
-    }
-  }
\ No newline at end of file
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
new file mode 100644
index 0000000..f4213e2
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeOperations.java
@@ -0,0 +1,219 @@
+/*
+ * Copyright 2016-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.onosproject.store.primitives.resources.impl;
+
+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.serializers.KryoNamespaces;
+import org.onosproject.store.service.DocumentPath;
+import org.onosproject.store.service.Versioned;
+
+/**
+ * {@link AtomixDocumentTree} resource state machine operations.
+ */
+public enum AtomixDocumentTreeOperations implements OperationId {
+    ADD_LISTENER("set", OperationType.COMMAND),
+    REMOVE_LISTENER("compareAndSet", OperationType.COMMAND),
+    GET("incrementAndGet", OperationType.QUERY),
+    GET_CHILDREN("getAndIncrement", OperationType.QUERY),
+    UPDATE("addAndGet", OperationType.COMMAND),
+    CLEAR("getAndAdd", OperationType.COMMAND);
+
+    private final String id;
+    private final OperationType type;
+
+    AtomixDocumentTreeOperations(String id, OperationType type) {
+        this.id = id;
+        this.type = type;
+    }
+
+    @Override
+    public String id() {
+        return id;
+    }
+
+    @Override
+    public OperationType type() {
+        return type;
+    }
+
+    public static final KryoNamespace NAMESPACE = KryoNamespace.newBuilder()
+            .register(KryoNamespaces.BASIC)
+            .nextId(KryoNamespaces.BEGIN_USER_CUSTOM_ID)
+            .register(Listen.class)
+            .register(Unlisten.class)
+            .register(Get.class)
+            .register(GetChildren.class)
+            .register(Update.class)
+            .register(DocumentPath.class)
+            .register(Match.class)
+            .register(Versioned.class)
+            .register(DocumentTreeUpdateResult.class)
+            .register(DocumentTreeUpdateResult.Status.class)
+            .build("AtomixDocumentTreeOperations");
+
+    /**
+     * Abstract DocumentTree command.
+     */
+    @SuppressWarnings("serial")
+    public abstract static class DocumentTreeOperation {
+        private DocumentPath path;
+
+        DocumentTreeOperation(DocumentPath path) {
+            this.path = path;
+        }
+
+        public DocumentPath path() {
+            return path;
+        }
+    }
+
+    /**
+     * DocumentTree#get query.
+     */
+    @SuppressWarnings("serial")
+    public static class Get extends DocumentTreeOperation {
+        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 DocumentTreeOperation {
+        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 DocumentTreeOperation {
+        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 DocumentTreeOperation {
+        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 DocumentTreeOperation {
+        public Unlisten() {
+            this(DocumentPath.from("root"));
+        }
+
+        public Unlisten(DocumentPath path) {
+            super(path);
+        }
+
+        @Override
+        public String toString() {
+            return MoreObjects.toStringHelper(getClass())
+                    .add("path", path())
+                    .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
new file mode 100644
index 0000000..6b7c550
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeService.java
@@ -0,0 +1,306 @@
+/*
+ * Copyright 2016-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.onosproject.store.primitives.resources.impl;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+import java.util.Queue;
+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.base.Throwables;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Queues;
+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.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.DocumentTreeUpdateResult.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.Serializer;
+import org.onosproject.store.service.Versioned;
+
+import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeEvents.CHANGE;
+import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.ADD_LISTENER;
+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.REMOVE_LISTENER;
+import static org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeOperations.UPDATE;
+
+/**
+ * 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(getSessions().getSession(input.readLong()),
+                            kryo.readObjectOrNull(input, DocumentPath.class));
+                }
+            }, Listener.class)
+            .register(Versioned.class)
+            .register(DocumentPath.class)
+            .register(new HashMap().keySet().getClass())
+            .register(TreeMap.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 = new DefaultDocumentTree<>(versionCounter::incrementAndGet);
+
+    @Override
+    public void snapshot(SnapshotWriter writer) {
+        writer.writeLong(versionCounter.get());
+        writer.writeObject(listeners, serializer::encode);
+        writer.writeObject(docTree, serializer::encode);
+    }
+
+    @Override
+    public void install(SnapshotReader reader) {
+        versionCounter = new AtomicLong(reader.readLong());
+        listeners = reader.readObject(serializer::decode);
+        docTree = 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);
+    }
+
+    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 == null ? null : node);
+        } catch (IllegalStateException e) {
+            return null;
+        }
+    }
+
+    protected Map<String, Versioned<byte[]>> getChildren(Commit<? extends GetChildren> commit) {
+        return docTree.getChildren(commit.value().path());
+    }
+
+    protected DocumentTreeUpdateResult<byte[]> update(Commit<? extends Update> commit) {
+        DocumentTreeUpdateResult<byte[]> result = null;
+        DocumentPath path = commit.value().path();
+        boolean updated = false;
+        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) {
+                    docTree.removeNode(path);
+                } else {
+                    docTree.set(path, commit.value().value().orElse(null));
+                }
+                updated = true;
+            }
+            Versioned<byte[]> newValue = updated ? docTree.get(path) : currentValue;
+            Status updateStatus = updated
+                    ? Status.OK : commit.value().value() == null ? Status.INVALID_PATH : Status.NOOP;
+            result = new DocumentTreeUpdateResult<>(path, updateStatus, newValue, currentValue);
+        } catch (IllegalDocumentModificationException e) {
+            result = DocumentTreeUpdateResult.illegalModification(path);
+        } catch (NoSuchDocumentPathException e) {
+            result = DocumentTreeUpdateResult.invalidPath(path);
+        } catch (Exception e) {
+            getLogger().error("Failed to apply {} to state machine", commit.value(), e);
+            throw Throwables.propagate(e);
+        }
+        notifyListeners(path, result);
+        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);
+            }
+        }
+    }
+
+    private void notifyListeners(DocumentPath path, DocumentTreeUpdateResult<byte[]> result) {
+        if (result.status() != Status.OK) {
+            return;
+        }
+        DocumentTreeEvent<byte[]> event =
+                new DocumentTreeEvent<>(path,
+                        result.created() ? Type.CREATED : result.newValue() == null ? Type.DELETED : Type.UPDATED,
+                        Optional.ofNullable(result.newValue()),
+                        Optional.ofNullable(result.oldValue()));
+
+        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/AtomixDocumentTreeState.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeState.java
deleted file mode 100644
index 8a8a23c..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixDocumentTreeState.java
+++ /dev/null
@@ -1,342 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Laboratory
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.onosproject.store.primitives.resources.impl;
-
-import static org.slf4j.LoggerFactory.getLogger;
-import io.atomix.copycat.server.Commit;
-import io.atomix.copycat.server.Snapshottable;
-import io.atomix.copycat.server.StateMachineExecutor;
-import io.atomix.copycat.server.session.ServerSession;
-import io.atomix.copycat.server.session.SessionListener;
-import io.atomix.copycat.server.storage.snapshot.SnapshotReader;
-import io.atomix.copycat.server.storage.snapshot.SnapshotWriter;
-import io.atomix.resource.ResourceStateMachine;
-
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Optional;
-import java.util.Properties;
-import java.util.Queue;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.stream.Collectors;
-
-import org.onlab.util.Match;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeCommands.Clear;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeCommands.Get;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeCommands.GetChildren;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeCommands.Listen;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeCommands.Unlisten;
-import org.onosproject.store.primitives.resources.impl.AtomixDocumentTreeCommands.Update;
-import org.onosproject.store.primitives.resources.impl.DocumentTreeUpdateResult.Status;
-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.Versioned;
-import org.slf4j.Logger;
-
-import com.google.common.base.Throwables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Queues;
-
-/**
- * State Machine for {@link AtomixDocumentTree} resource.
- */
-public class AtomixDocumentTreeState
-    extends ResourceStateMachine
-    implements SessionListener, Snapshottable {
-
-    private final Logger log = getLogger(getClass());
-    private final Map<Long, SessionListenCommits> listeners = new HashMap<>();
-    private AtomicLong versionCounter = new AtomicLong(0);
-    private final DocumentTree<TreeNodeValue> docTree = new DefaultDocumentTree<>(versionCounter::incrementAndGet);
-
-    public AtomixDocumentTreeState(Properties properties) {
-        super(properties);
-    }
-
-    @Override
-    public void snapshot(SnapshotWriter writer) {
-        writer.writeLong(versionCounter.get());
-    }
-
-    @Override
-    public void install(SnapshotReader reader) {
-        versionCounter = new AtomicLong(reader.readLong());
-    }
-
-    @Override
-    protected void configure(StateMachineExecutor executor) {
-        // Listeners
-        executor.register(Listen.class, this::listen);
-        executor.register(Unlisten.class, this::unlisten);
-        // queries
-        executor.register(Get.class, this::get);
-        executor.register(GetChildren.class, this::getChildren);
-        // commands
-        executor.register(Update.class, this::update);
-        executor.register(Clear.class, this::clear);
-    }
-
-    protected void listen(Commit<? extends Listen> commit) {
-        Long sessionId = commit.session().id();
-        listeners.computeIfAbsent(sessionId, k -> new SessionListenCommits()).add(commit);
-        commit.session().onStateChange(
-                        state -> {
-                            if (state == ServerSession.State.CLOSED
-                                    || state == ServerSession.State.EXPIRED) {
-                                closeListener(commit.session().id());
-                            }
-                        });
-    }
-
-    protected void unlisten(Commit<? extends Unlisten> commit) {
-        Long sessionId = commit.session().id();
-        try {
-            SessionListenCommits listenCommits = listeners.get(sessionId);
-            if (listenCommits != null) {
-                listenCommits.remove(commit);
-            }
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected Versioned<byte[]> get(Commit<? extends Get> commit) {
-        try {
-            Versioned<TreeNodeValue> value = docTree.get(commit.operation().path());
-            return value == null ? null : value.map(node -> node == null ? null : node.value());
-        } catch (IllegalStateException e) {
-            return null;
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected Map<String, Versioned<byte[]>> getChildren(Commit<? extends GetChildren> commit) {
-        try {
-            Map<String, Versioned<TreeNodeValue>> children = docTree.getChildren(commit.operation().path());
-            return children == null
-                    ? null : Maps.newHashMap(Maps.transformValues(children,
-                                                                  value -> value.map(TreeNodeValue::value)));
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected DocumentTreeUpdateResult<byte[]> update(Commit<? extends Update> commit) {
-        DocumentTreeUpdateResult<byte[]> result = null;
-        DocumentPath path = commit.operation().path();
-        boolean updated = false;
-        Versioned<TreeNodeValue> currentValue = docTree.get(path);
-        try {
-            Match<Long> versionMatch = commit.operation().versionMatch();
-            Match<byte[]> valueMatch = commit.operation().valueMatch();
-
-            if (versionMatch.matches(currentValue == null ? null : currentValue.version())
-                    && valueMatch.matches(currentValue == null ? null : currentValue.value().value())) {
-                if (commit.operation().value() == null) {
-                    docTree.removeNode(path);
-                } else {
-                    docTree.set(path, new NonTransactionalCommit(commit));
-                }
-                updated = true;
-            }
-            Versioned<TreeNodeValue> newValue = updated ? docTree.get(path) : currentValue;
-            Status updateStatus = updated
-                    ? Status.OK : commit.operation().value() == null ? Status.INVALID_PATH : Status.NOOP;
-            result = new DocumentTreeUpdateResult<>(path,
-                    updateStatus,
-                    newValue == null
-                        ? null : newValue.map(TreeNodeValue::value),
-                    currentValue == null
-                        ? null : currentValue.map(TreeNodeValue::value));
-        } catch (IllegalDocumentModificationException e) {
-            result = DocumentTreeUpdateResult.illegalModification(path);
-        } catch (NoSuchDocumentPathException e) {
-            result = DocumentTreeUpdateResult.invalidPath(path);
-        } catch (Exception e) {
-            log.error("Failed to apply {} to state machine", commit.operation(), e);
-            throw Throwables.propagate(e);
-        } finally {
-            if (updated) {
-                if (currentValue != null) {
-                    currentValue.value().discard();
-                }
-            } else {
-                commit.close();
-            }
-        }
-        notifyListeners(path, result);
-        return result;
-    }
-
-    protected void clear(Commit<? extends Clear> commit) {
-        try {
-            Queue<DocumentPath> toClearQueue = Queues.newArrayDeque();
-            Map<String, Versioned<TreeNodeValue>> 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<TreeNodeValue>> children = docTree.getChildren(path);
-                if (children.size() == 0) {
-                    docTree.removeNode(path).value().discard();
-                } else {
-                    children.keySet()
-                            .stream()
-                            .forEach(name -> toClearQueue.add(new DocumentPath(name, path)));
-                    toClearQueue.add(path);
-                }
-            }
-        } finally {
-            commit.close();
-        }
-    }
-
-    /**
-     * Interface implemented by tree node values.
-     */
-    private interface TreeNodeValue {
-        /**
-         * Returns the raw {@code byte[]}.
-         *
-         * @return raw value
-         */
-        byte[] value();
-
-        /**
-         * Discards the value by invoke appropriate clean up actions.
-         */
-        void discard();
-    }
-
-    /**
-     * A {@code TreeNodeValue} that is derived from a non-transactional update
-     * i.e. via any standard tree update operation.
-     */
-    private class NonTransactionalCommit implements TreeNodeValue {
-        private final Commit<? extends Update> commit;
-
-        public NonTransactionalCommit(Commit<? extends Update> commit) {
-            this.commit = commit;
-        }
-
-        @Override
-        public byte[] value() {
-            return commit.operation().value().orElse(null);
-        }
-
-        @Override
-        public void discard() {
-            commit.close();
-        }
-    }
-
-    private void notifyListeners(DocumentPath path, DocumentTreeUpdateResult<byte[]> result) {
-        if (result.status() != Status.OK) {
-            return;
-        }
-        DocumentTreeEvent<byte[]> event =
-                new DocumentTreeEvent<>(path,
-                        result.created() ? Type.CREATED : result.newValue() == null ? Type.DELETED : Type.UPDATED,
-                        Optional.ofNullable(result.newValue()),
-                        Optional.ofNullable(result.oldValue()));
-
-        listeners.values()
-                 .stream()
-                 .filter(l -> event.path().isDescendentOf(l.leastCommonAncestorPath()))
-                 .forEach(listener -> listener.publish(AtomixDocumentTree.CHANGE_SUBJECT, Arrays.asList(event)));
-    }
-
-    @Override
-    public void register(ServerSession session) {
-    }
-
-    @Override
-    public void unregister(ServerSession session) {
-        closeListener(session.id());
-    }
-
-    @Override
-    public void expire(ServerSession session) {
-        closeListener(session.id());
-    }
-
-    @Override
-    public void close(ServerSession session) {
-        closeListener(session.id());
-    }
-
-    private void closeListener(Long sessionId) {
-        SessionListenCommits listenCommits = listeners.remove(sessionId);
-        if (listenCommits != null) {
-            listenCommits.close();
-        }
-    }
-
-    private class SessionListenCommits {
-        private final List<Commit<? extends Listen>> commits = Lists.newArrayList();
-        private DocumentPath leastCommonAncestorPath;
-
-        public void add(Commit<? extends Listen> commit) {
-            commits.add(commit);
-            recomputeLeastCommonAncestor();
-        }
-
-        public void remove(Commit<? extends Unlisten> commit) {
-            // Remove the first listen commit with path matching path in unlisten commit
-            Iterator<Commit<? extends Listen>> iterator = commits.iterator();
-            while (iterator.hasNext()) {
-                Commit<? extends Listen> listenCommit = iterator.next();
-                if (listenCommit.operation().path().equals(commit.operation().path())) {
-                    iterator.remove();
-                    listenCommit.close();
-                }
-            }
-            recomputeLeastCommonAncestor();
-        }
-
-        public DocumentPath leastCommonAncestorPath() {
-            return leastCommonAncestorPath;
-        }
-
-        public <M> void publish(String topic, M message) {
-            commits.stream().findAny().ifPresent(commit -> commit.session().publish(topic, message));
-        }
-
-        public void close() {
-            commits.forEach(Commit::close);
-            commits.clear();
-            leastCommonAncestorPath = null;
-        }
-
-        private void recomputeLeastCommonAncestor() {
-            this.leastCommonAncestorPath = DocumentPath.leastCommonAncestor(commits.stream()
-                    .map(c -> c.operation().path())
-                    .collect(Collectors.toList()));
-        }
-    }
-}
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
index 971b60c..7d487f4 100644
--- 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
@@ -18,36 +18,34 @@
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.atomic.AtomicLong;
 
-import io.atomix.variables.DistributedLong;
+import org.onosproject.store.service.AsyncAtomicCounter;
 import org.onosproject.store.service.AsyncAtomicIdGenerator;
 
 /**
  * {@code AsyncAtomicIdGenerator} implementation backed by Atomix
- * {@link DistributedLong}.
+ * {@link AsyncAtomicCounter}.
  */
 public class AtomixIdGenerator implements AsyncAtomicIdGenerator {
 
     private static final long DEFAULT_BATCH_SIZE = 1000;
-    private final String name;
-    private final DistributedLong distLong;
+    private final AsyncAtomicCounter counter;
     private final long batchSize;
     private CompletableFuture<Long> reserveFuture;
     private long base;
     private final AtomicLong delta = new AtomicLong();
 
-    public AtomixIdGenerator(String name, DistributedLong distLong) {
-        this(name, distLong, DEFAULT_BATCH_SIZE);
+    public AtomixIdGenerator(AsyncAtomicCounter counter) {
+        this(counter, DEFAULT_BATCH_SIZE);
     }
 
-    AtomixIdGenerator(String name, DistributedLong distLong, long batchSize) {
-        this.name = name;
-        this.distLong = distLong;
+    AtomixIdGenerator(AsyncAtomicCounter counter, long batchSize) {
+        this.counter = counter;
         this.batchSize = batchSize;
     }
 
     @Override
     public String name() {
-        return name;
+        return counter.name();
     }
 
     @Override
@@ -64,9 +62,9 @@
 
     private CompletableFuture<Long> reserve() {
         if (reserveFuture == null || reserveFuture.isDone()) {
-            reserveFuture = distLong.getAndAdd(batchSize);
+            reserveFuture = counter.getAndAdd(batchSize);
         } else {
-            reserveFuture = reserveFuture.thenCompose(v -> distLong.getAndAdd(batchSize));
+            reserveFuture = reserveFuture.thenCompose(v -> counter.getAndAdd(batchSize));
         }
         reserveFuture = reserveFuture.thenApply(base -> {
             this.base = base;
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
index 3e45091..825fa98 100644
--- 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
@@ -15,73 +15,65 @@
  */
 package org.onosproject.store.primitives.resources.impl;
 
-import io.atomix.copycat.client.CopycatClient;
-import io.atomix.resource.AbstractResource;
-import io.atomix.resource.ResourceTypeInfo;
-
-import java.util.Collection;
 import java.util.List;
 import java.util.Map;
-import java.util.Properties;
 import java.util.Set;
 import java.util.concurrent.CompletableFuture;
 import java.util.function.Consumer;
-import java.util.function.Function;
 
-import org.onosproject.cluster.Leadership;
-import org.onosproject.cluster.NodeId;
-import org.onosproject.event.Change;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.Anoint;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.GetAllLeaderships;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.GetElectedTopics;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.GetLeadership;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.Listen;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.Promote;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.Run;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.Unlisten;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.Withdraw;
-import org.onosproject.store.service.AsyncLeaderElector;
-
-import com.google.common.collect.ImmutableSet;
 import com.google.common.cache.CacheBuilder;
 import com.google.common.cache.CacheLoader;
 import com.google.common.cache.LoadingCache;
 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.
  */
-@ResourceTypeInfo(id = -152, factory = AtomixLeaderElectorFactory.class)
-public class AtomixLeaderElector extends AbstractResource<AtomixLeaderElector>
-    implements AsyncLeaderElector {
-    private final Set<Consumer<Status>> statusChangeListeners =
-            Sets.newCopyOnWriteArraySet();
-    private final Set<Consumer<Change<Leadership>>> leadershipChangeListeners =
-            Sets.newCopyOnWriteArraySet();
+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();
     private final Consumer<Change<Leadership>> cacheUpdater;
     private final Consumer<Status> statusListener;
 
-    public static final String CHANGE_SUBJECT = "leadershipChangeEvents";
     private final LoadingCache<String, CompletableFuture<Leadership>> cache;
 
-    Function<CopycatClient.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);
-        }
-    };
-
-    public AtomixLeaderElector(CopycatClient client, Properties properties) {
-        super(client, properties);
+    public AtomixLeaderElector(RaftProxy proxy) {
+        super(proxy);
         cache = CacheBuilder.newBuilder()
                 .maximumSize(1000)
-                .build(CacheLoader.from(topic -> this.client.submit(new GetLeadership(topic))));
+                .build(CacheLoader.from(topic -> proxy.invoke(
+                        GET_LEADERSHIP, SERIALIZER::encode, new GetLeadership(topic), SERIALIZER::decode)));
 
         cacheUpdater = change -> {
             Leadership leadership = change.newValue();
@@ -93,7 +85,13 @@
             }
         };
         addStatusChangeListener(statusListener);
-        client.onStateChange(this::handleStateChange);
+
+        proxy.addStateChangeListener(state -> {
+            if (state == RaftProxy.State.CONNECTED && isListening()) {
+                proxy.invoke(ADD_LISTENER);
+            }
+        });
+        proxy.addEventListener(CHANGE, SERIALIZER::decode, this::handleEvent);
     }
 
     @Override
@@ -102,24 +100,6 @@
         return removeChangeListener(cacheUpdater);
     }
 
-    @Override
-    public String name() {
-        return null;
-    }
-
-    @Override
-    public CompletableFuture<AtomixLeaderElector> open() {
-        return super.open().thenApply(result -> {
-            client.onStateChange(state -> {
-                if (state == CopycatClient.State.CONNECTED && isListening()) {
-                    client.submit(new Listen());
-                }
-            });
-            client.onEvent(CHANGE_SUBJECT, this::handleEvent);
-            return result;
-        });
-    }
-
     public CompletableFuture<AtomixLeaderElector> setupCache() {
         return addChangeListener(cacheUpdater).thenApply(v -> this);
     }
@@ -130,27 +110,32 @@
 
     @Override
     public CompletableFuture<Leadership> run(String topic, NodeId nodeId) {
-        return client.submit(new Run(topic, nodeId)).whenComplete((r, e) -> cache.invalidate(topic));
+        return proxy.<Run, Leadership>invoke(RUN, SERIALIZER::encode, new Run(topic, nodeId), SERIALIZER::decode)
+                .whenComplete((r, e) -> cache.invalidate(topic));
     }
 
     @Override
     public CompletableFuture<Void> withdraw(String topic) {
-        return client.submit(new Withdraw(topic)).whenComplete((r, e) -> cache.invalidate(topic));
+        return proxy.invoke(WITHDRAW, SERIALIZER::encode, new Withdraw(topic))
+                .whenComplete((r, e) -> cache.invalidate(topic));
     }
 
     @Override
     public CompletableFuture<Boolean> anoint(String topic, NodeId nodeId) {
-        return client.submit(new Anoint(topic, nodeId)).whenComplete((r, e) -> cache.invalidate(topic));
+        return proxy.<Anoint, Boolean>invoke(ANOINT, SERIALIZER::encode, new Anoint(topic, nodeId), SERIALIZER::decode)
+                .whenComplete((r, e) -> cache.invalidate(topic));
     }
 
     @Override
     public CompletableFuture<Boolean> promote(String topic, NodeId nodeId) {
-        return client.submit(new Promote(topic, nodeId)).whenComplete((r, e) -> cache.invalidate(topic));
+        return proxy.<Promote, Boolean>invoke(
+                PROMOTE, SERIALIZER::encode, new Promote(topic, nodeId), SERIALIZER::decode)
+                .whenComplete((r, e) -> cache.invalidate(topic));
     }
 
     @Override
     public CompletableFuture<Void> evict(NodeId nodeId) {
-        return client.submit(new AtomixLeaderElectorCommands.Evict(nodeId));
+        return proxy.invoke(EVICT, SERIALIZER::encode, new AtomixLeaderElectorOperations.Evict(nodeId));
     }
 
     @Override
@@ -165,17 +150,17 @@
 
     @Override
     public CompletableFuture<Map<String, Leadership>> getLeaderships() {
-        return client.submit(new GetAllLeaderships());
+        return proxy.invoke(GET_ALL_LEADERSHIPS, SERIALIZER::decode);
     }
 
     public CompletableFuture<Set<String>> getElectedTopics(NodeId nodeId) {
-        return client.submit(new GetElectedTopics(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 client.submit(new Listen()).thenRun(() -> leadershipChangeListeners.add(consumer));
+            return proxy.invoke(ADD_LISTENER).thenRun(() -> leadershipChangeListeners.add(consumer));
         } else {
             leadershipChangeListeners.add(consumer);
             return CompletableFuture.completedFuture(null);
@@ -185,31 +170,12 @@
     @Override
     public synchronized CompletableFuture<Void> removeChangeListener(Consumer<Change<Leadership>> consumer) {
         if (leadershipChangeListeners.remove(consumer) && leadershipChangeListeners.isEmpty()) {
-            return client.submit(new Unlisten()).thenApply(v -> null);
+            return proxy.invoke(REMOVE_LISTENER).thenApply(v -> null);
         }
         return CompletableFuture.completedFuture(null);
     }
 
-    @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);
-    }
-
     private boolean isListening() {
         return !leadershipChangeListeners.isEmpty();
     }
-
-    private void handleStateChange(CopycatClient.State state) {
-        statusChangeListeners().forEach(listener -> listener.accept(mapper.apply(state)));
-    }
-}
+}
\ No newline at end of file
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorCommands.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorCommands.java
deleted file mode 100644
index 87e6bcc..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorCommands.java
+++ /dev/null
@@ -1,461 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Laboratory
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.onosproject.store.primitives.resources.impl;
-
-import java.util.Map;
-import java.util.Set;
-
-import org.onosproject.cluster.Leadership;
-import org.onosproject.cluster.NodeId;
-
-
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Strings;
-
-import io.atomix.catalyst.buffer.BufferInput;
-import io.atomix.catalyst.buffer.BufferOutput;
-import io.atomix.catalyst.serializer.CatalystSerializable;
-import io.atomix.catalyst.serializer.SerializableTypeResolver;
-import io.atomix.catalyst.serializer.Serializer;
-import io.atomix.catalyst.serializer.SerializerRegistry;
-import io.atomix.catalyst.util.Assert;
-import io.atomix.copycat.Command;
-import io.atomix.copycat.Query;
-
-/**
- * {@link AtomixLeaderElector} resource state machine operations.
- */
-public final class AtomixLeaderElectorCommands {
-
-    private AtomixLeaderElectorCommands() {
-    }
-
-    /**
-     * Abstract election query.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class ElectionQuery<V> implements Query<V>, CatalystSerializable {
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-        }
-    }
-
-    /**
-     * Abstract election topic query.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class TopicQuery<V> extends ElectionQuery<V> implements CatalystSerializable {
-        String topic;
-
-        public TopicQuery() {
-        }
-
-        public TopicQuery(String topic) {
-          this.topic = Assert.notNull(topic, "topic");
-        }
-
-        /**
-         * Returns the topic.
-         * @return topic
-         */
-        public String topic() {
-          return topic;
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            serializer.writeObject(topic, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-          topic = serializer.readObject(buffer);
-        }
-    }
-
-    /**
-     * Abstract election command.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class ElectionCommand<V> implements Command<V>, CatalystSerializable {
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-        }
-    }
-
-    /**
-     * Listen command.
-     */
-    @SuppressWarnings("serial")
-    public static class Listen extends ElectionCommand<Void> {
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.QUORUM;
-        }
-    }
-
-    /**
-     * Unlisten command.
-     */
-    @SuppressWarnings("serial")
-    public static class Unlisten extends ElectionCommand<Void> {
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.TOMBSTONE;
-        }
-    }
-
-    /**
-     * GetLeader query.
-     */
-    @SuppressWarnings("serial")
-    public static class GetLeadership extends TopicQuery<Leadership> {
-
-        public GetLeadership() {
-        }
-
-        public GetLeadership(String topic) {
-            super(topic);
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("topic", topic)
-                    .toString();
-        }
-    }
-
-    /**
-     * GetAllLeaders query.
-     */
-    @SuppressWarnings("serial")
-    public static class GetAllLeaderships extends ElectionQuery<Map<String, Leadership>> {
-    }
-
-    /**
-     * GetElectedTopics query.
-     */
-    @SuppressWarnings("serial")
-    public static class GetElectedTopics extends ElectionQuery<Set<String>> {
-        private NodeId nodeId;
-
-        public GetElectedTopics() {
-        }
-
-        public GetElectedTopics(NodeId nodeId) {
-            this.nodeId = Assert.argNot(nodeId, nodeId == null, "nodeId cannot be null");
-        }
-
-        /**
-         * 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();
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            super.writeObject(buffer, serializer);
-            serializer.writeObject(nodeId, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            super.readObject(buffer, serializer);
-            nodeId = serializer.readObject(buffer);
-        }
-    }
-
-    /**
-     * Enter and run for leadership.
-     */
-    @SuppressWarnings("serial")
-    public static class Run extends ElectionCommand<Leadership> {
-        private String topic;
-        private NodeId nodeId;
-
-        public Run() {
-        }
-
-        public Run(String topic, NodeId nodeId) {
-            this.topic = Assert.argNot(topic, Strings.isNullOrEmpty(topic), "topic cannot be null or empty");
-            this.nodeId = Assert.argNot(nodeId, nodeId == null, "nodeId cannot be null");
-        }
-
-        /**
-         * Returns the topic.
-         *
-         * @return topic
-         */
-        public String topic() {
-            return topic;
-        }
-
-        /**
-         * Returns the nodeId.
-         *
-         * @return the nodeId
-         */
-        public NodeId nodeId() {
-            return nodeId;
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.SNAPSHOT;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("topic", topic)
-                    .add("nodeId", nodeId)
-                    .toString();
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            buffer.writeString(topic);
-            buffer.writeString(nodeId.toString());
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            topic = buffer.readString();
-            nodeId = new NodeId(buffer.readString());
-        }
-    }
-
-    /**
-     * Withdraw from a leadership contest.
-     */
-    @SuppressWarnings("serial")
-    public static class Withdraw extends ElectionCommand<Void> {
-        private String topic;
-
-        public Withdraw() {
-        }
-
-        public Withdraw(String topic) {
-            this.topic = Assert.argNot(topic, Strings.isNullOrEmpty(topic), "topic cannot be null or empty");
-        }
-
-        /**
-         * Returns the topic.
-         *
-         * @return The topic
-         */
-        public String topic() {
-            return topic;
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.SNAPSHOT;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("topic", topic)
-                    .toString();
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            buffer.writeString(topic);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            topic = buffer.readString();
-        }
-    }
-
-    /**
-     * Command for administratively changing the leadership state for a node.
-     */
-    @SuppressWarnings("serial")
-    public abstract static class ElectionChangeCommand<V> extends ElectionCommand<V>  {
-        private String topic;
-        private NodeId nodeId;
-
-        ElectionChangeCommand() {
-            topic = null;
-            nodeId = null;
-        }
-
-        public ElectionChangeCommand(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 CompactionMode compaction() {
-            return CompactionMode.SNAPSHOT;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("topic", topic)
-                    .add("nodeId", nodeId)
-                    .toString();
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            buffer.writeString(topic);
-            buffer.writeString(nodeId.toString());
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            topic = buffer.readString();
-            nodeId = new NodeId(buffer.readString());
-        }
-    }
-
-    /**
-     * Command for administratively anoint a node as leader.
-     */
-    @SuppressWarnings("serial")
-    public static class Anoint extends ElectionChangeCommand<Boolean> {
-
-        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 ElectionChangeCommand<Boolean> {
-
-        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 ElectionCommand<Void> {
-        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 CompactionMode compaction() {
-            return CompactionMode.SNAPSHOT;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("nodeId", nodeId)
-                    .toString();
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            buffer.writeString(nodeId.toString());
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            nodeId = new NodeId(buffer.readString());
-        }
-    }
-
-    /**
-     * Map command type resolver.
-     */
-    public static class TypeResolver implements SerializableTypeResolver {
-        @Override
-        public void resolve(SerializerRegistry registry) {
-            registry.register(Run.class, -861);
-            registry.register(Withdraw.class, -862);
-            registry.register(Anoint.class, -863);
-            registry.register(GetAllLeaderships.class, -864);
-            registry.register(GetElectedTopics.class, -865);
-            registry.register(GetLeadership.class, -866);
-            registry.register(Listen.class, -867);
-            registry.register(Unlisten.class, -868);
-            registry.register(Promote.class, -869);
-            registry.register(Evict.class, -870);
-        }
-    }
-}
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
new file mode 100644
index 0000000..b6ba8a5
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorEvents.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2017-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.store.primitives.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("change");
+
+    private final String id;
+
+    AtomixLeaderElectorEvents(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)
+            .build("AtomixLeaderElectorEvents");
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorFactory.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorFactory.java
deleted file mode 100644
index 22f23b5..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorFactory.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Laboratory
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.onosproject.store.primitives.resources.impl;
-
-import io.atomix.catalyst.serializer.SerializableTypeResolver;
-import io.atomix.copycat.client.CopycatClient;
-import io.atomix.resource.ResourceFactory;
-import io.atomix.resource.ResourceStateMachine;
-
-import java.util.Properties;
-
-/**
- * {@link AtomixLeaderElector} resource factory.
- *
- */
-public class AtomixLeaderElectorFactory implements ResourceFactory<AtomixLeaderElector> {
-
-    @Override
-    public SerializableTypeResolver createSerializableTypeResolver() {
-        return new AtomixLeaderElectorCommands.TypeResolver();
-    }
-
-    @Override
-    public ResourceStateMachine createStateMachine(Properties config) {
-        return new AtomixLeaderElectorState(config);
-    }
-
-    @Override
-    public AtomixLeaderElector createInstance(CopycatClient client, Properties options) {
-        return new AtomixLeaderElector(client, options);
-    }
-}
\ No newline at end of file
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
new file mode 100644
index 0000000..17b8c23
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorOperations.java
@@ -0,0 +1,333 @@
+/*
+ * Copyright 2016-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.store.primitives.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("addListener", OperationType.COMMAND),
+    REMOVE_LISTENER("removeListener", OperationType.COMMAND),
+    RUN("run", OperationType.COMMAND),
+    WITHDRAW("withdraw", OperationType.COMMAND),
+    ANOINT("anoint", OperationType.COMMAND),
+    PROMOTE("promote", OperationType.COMMAND),
+    EVICT("evict", OperationType.COMMAND),
+    GET_LEADERSHIP("getLeadership", OperationType.QUERY),
+    GET_ALL_LEADERSHIPS("getAllLeaderships", OperationType.QUERY),
+    GET_ELECTED_TOPICS("getElectedTopics", OperationType.QUERY);
+
+    private final String id;
+    private final OperationType type;
+
+    AtomixLeaderElectorOperations(String id, OperationType type) {
+        this.id = id;
+        this.type = type;
+    }
+
+    @Override
+    public String id() {
+        return id;
+    }
+
+    @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/AtomixLeaderElectorState.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorService.java
similarity index 63%
rename from core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorState.java
rename to core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorService.java
index 369c191..c65d920 100644
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorState.java
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixLeaderElectorService.java
@@ -15,18 +15,6 @@
  */
 package org.onosproject.store.primitives.resources.impl;
 
-import static org.slf4j.LoggerFactory.getLogger;
-
-import com.google.common.collect.ImmutableSet;
-import io.atomix.copycat.server.session.ServerSession;
-import io.atomix.copycat.server.Commit;
-import io.atomix.copycat.server.Snapshottable;
-import io.atomix.copycat.server.StateMachineExecutor;
-import io.atomix.copycat.server.session.SessionListener;
-import io.atomix.copycat.server.storage.snapshot.SnapshotReader;
-import io.atomix.copycat.server.storage.snapshot.SnapshotWriter;
-import io.atomix.resource.ResourceStateMachine;
-
 import java.util.Arrays;
 import java.util.HashMap;
 import java.util.LinkedHashMap;
@@ -34,69 +22,101 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Optional;
-import java.util.Properties;
 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.base.Throwables;
+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.AtomixLeaderElectorCommands.Anoint;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.Evict;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.GetAllLeaderships;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.GetElectedTopics;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.GetLeadership;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.Listen;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.Promote;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.Run;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.Unlisten;
-import org.onosproject.store.primitives.resources.impl.AtomixLeaderElectorCommands.Withdraw;
-import org.onosproject.store.serializers.KryoNamespaces;
+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 org.slf4j.Logger;
 
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Objects;
-import com.google.common.base.Throwables;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
+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 AtomixLeaderElectorState extends ResourceStateMachine
-    implements SessionListener, Snapshottable {
+public class AtomixLeaderElectorService extends AbstractRaftService {
 
-    private final Logger log = getLogger(getClass());
+    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 final Map<Long, Commit<? extends Listen>> listeners = new LinkedHashMap<>();
-    private final Serializer serializer = Serializer.using(Arrays.asList(KryoNamespaces.API),
-                                                           ElectionState.class,
-                                                           Registration.class);
+    private Map<Long, RaftSession> listeners = new LinkedHashMap<>();
 
-    public AtomixLeaderElectorState(Properties properties) {
-        super(properties);
+    @Override
+    public void snapshot(SnapshotWriter writer) {
+        writer.writeObject(Sets.newHashSet(listeners.keySet()), SERIALIZER::encode);
+        writer.writeObject(termCounters, SERIALIZER::encode);
+        writer.writeObject(elections, SERIALIZER::encode);
+        getLogger().debug("Took state machine snapshot");
     }
 
     @Override
-    protected void configure(StateMachineExecutor executor) {
+    public void install(SnapshotReader reader) {
+        listeners = new LinkedHashMap<>();
+        for (Long sessionId : reader.<Set<Long>>readObject(SERIALIZER::decode)) {
+            listeners.put(sessionId, getSessions().getSession(sessionId));
+        }
+        termCounters = reader.readObject(SERIALIZER::decode);
+        elections = reader.readObject(SERIALIZER::decode);
+        getLogger().debug("Reinstated state machine from snapshot");
+    }
+
+    @Override
+    protected void configure(RaftServiceExecutor executor) {
         // Notification
-        executor.register(Listen.class, this::listen);
-        executor.register(Unlisten.class, this::unlisten);
+        executor.register(ADD_LISTENER, this::listen);
+        executor.register(REMOVE_LISTENER, this::unlisten);
         // Commands
-        executor.register(Run.class, this::run);
-        executor.register(Withdraw.class, this::withdraw);
-        executor.register(Anoint.class, this::anoint);
-        executor.register(Promote.class, this::promote);
-        executor.register(Evict.class, this::evict);
+        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(GetLeadership.class, this::leadership);
-        executor.register(GetAllLeaderships.class, this::allLeaderships);
-        executor.register(GetElectedTopics.class, this::electedTopics);
+        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) {
@@ -107,16 +127,7 @@
         if (changes.isEmpty()) {
             return;
         }
-        listeners.values()
-                 .forEach(listener -> listener.session()
-                                              .publish(AtomixLeaderElector.CHANGE_SUBJECT, changes));
-    }
-
-    @Override
-    public void delete() {
-      // Close and clear Listeners
-      listeners.values().forEach(Commit::close);
-      listeners.clear();
+        listeners.values().forEach(session -> session.publish(CHANGE, SERIALIZER::encode, changes));
     }
 
     /**
@@ -124,22 +135,8 @@
      *
      * @param commit listen commit
      */
-    public void listen(Commit<? extends Listen> commit) {
-        Long sessionId = commit.session().id();
-        if (listeners.putIfAbsent(commit.session().id(), commit) != null) {
-            commit.close();
-        }
-        commit.session()
-                .onStateChange(
-                        state -> {
-                            if (state == ServerSession.State.CLOSED
-                                    || state == ServerSession.State.EXPIRED) {
-                                Commit<? extends Listen> listener = listeners.remove(sessionId);
-                                if (listener != null) {
-                                    listener.close();
-                                }
-                            }
-                        });
+    public void listen(Commit<Void> commit) {
+        listeners.put(commit.session().sessionId().id(), commit.session());
     }
 
     /**
@@ -147,27 +144,20 @@
      *
      * @param commit unlisten commit
      */
-    public void unlisten(Commit<? extends Unlisten> commit) {
-        try {
-            Commit<? extends Listen> listener = listeners.remove(commit.session().id());
-            if (listener != null) {
-                listener.close();
-            }
-        } finally {
-            commit.close();
-        }
+    public void unlisten(Commit<Void> commit) {
+        listeners.remove(commit.session().sessionId().id());
     }
 
     /**
-     * Applies an {@link AtomixLeaderElectorCommands.Run} commit.
+     * 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.operation().topic();
+            String topic = commit.value().topic();
             Leadership oldLeadership = leadership(topic);
-            Registration registration = new Registration(commit.operation().nodeId(), commit.session().id());
+            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);
@@ -186,44 +176,40 @@
             }
             return newLeadership;
         } catch (Exception e) {
-            log.error("State machine operation failed", e);
+            getLogger().error("State machine operation failed", e);
             throw Throwables.propagate(e);
-        } finally {
-            commit.close();
         }
     }
 
     /**
-     * Applies an {@link AtomixLeaderElectorCommands.Withdraw} commit.
+     * Applies an {@link AtomixLeaderElectorOperations.Withdraw} commit.
      * @param commit withdraw commit
      */
     public void withdraw(Commit<? extends Withdraw> commit) {
         try {
-            String topic = commit.operation().topic();
+            String topic = commit.value().topic();
             Leadership oldLeadership = leadership(topic);
             elections.computeIfPresent(topic, (k, v) -> v.cleanup(commit.session(),
-                                        termCounter(topic)::incrementAndGet));
+                    termCounter(topic)::incrementAndGet));
             Leadership newLeadership = leadership(topic);
             if (!Objects.equal(oldLeadership, newLeadership)) {
                 notifyLeadershipChange(oldLeadership, newLeadership);
             }
         } catch (Exception e) {
-            log.error("State machine operation failed", e);
+            getLogger().error("State machine operation failed", e);
             throw Throwables.propagate(e);
-        } finally {
-            commit.close();
         }
     }
 
     /**
-     * Applies an {@link AtomixLeaderElectorCommands.Anoint} commit.
+     * 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.operation().topic();
-            NodeId nodeId = commit.operation().nodeId();
+            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)));
@@ -233,24 +219,22 @@
             }
             return (electionState != null &&
                     electionState.leader() != null &&
-                    commit.operation().nodeId().equals(electionState.leader().nodeId()));
+                    commit.value().nodeId().equals(electionState.leader().nodeId()));
         } catch (Exception e) {
-            log.error("State machine operation failed", e);
+            getLogger().error("State machine operation failed", e);
             throw Throwables.propagate(e);
-        } finally {
-            commit.close();
         }
     }
 
     /**
-     * Applies an {@link AtomixLeaderElectorCommands.Promote} commit.
+     * 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.operation().topic();
-            NodeId nodeId = commit.operation().nodeId();
+            String topic = commit.value().topic();
+            NodeId nodeId = commit.value().nodeId();
             Leadership oldLeadership = leadership(topic);
             if (oldLeadership == null || !oldLeadership.candidates().contains(nodeId)) {
                 return false;
@@ -262,21 +246,19 @@
             }
             return true;
         } catch (Exception e) {
-            log.error("State machine operation failed", e);
+            getLogger().error("State machine operation failed", e);
             throw Throwables.propagate(e);
-        } finally {
-            commit.close();
         }
     }
 
     /**
-     * Applies an {@link AtomixLeaderElectorCommands.Evict} commit.
+     * 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.operation().nodeId();
+            NodeId nodeId = commit.value().nodeId();
             Set<String> topics = Maps.filterValues(elections, e -> e.candidates().contains(nodeId)).keySet();
             topics.forEach(topic -> {
                 Leadership oldLeadership = leadership(topic);
@@ -288,65 +270,57 @@
             });
             notifyLeadershipChanges(changes);
         } catch (Exception e) {
-            log.error("State machine operation failed", e);
+            getLogger().error("State machine operation failed", e);
             throw Throwables.propagate(e);
-        } finally {
-            commit.close();
         }
     }
 
     /**
-     * Applies an {@link AtomixLeaderElectorCommands.GetLeadership} commit.
+     * Applies an {@link AtomixLeaderElectorOperations.GetLeadership} commit.
      * @param commit GetLeadership commit
      * @return leader
      */
-    public Leadership leadership(Commit<? extends GetLeadership> commit) {
-        String topic = commit.operation().topic();
+    public Leadership getLeadership(Commit<? extends GetLeadership> commit) {
+        String topic = commit.value().topic();
         try {
             return leadership(topic);
         } catch (Exception e) {
-            log.error("State machine operation failed", e);
+            getLogger().error("State machine operation failed", e);
             throw Throwables.propagate(e);
-        } finally {
-            commit.close();
         }
     }
 
     /**
-     * Applies an {@link AtomixLeaderElectorCommands.GetElectedTopics} commit.
+     * 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.operation().nodeId();
+            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) {
-            log.error("State machine operation failed", e);
+            getLogger().error("State machine operation failed", e);
             throw Throwables.propagate(e);
-        } finally {
-            commit.close();
         }
     }
 
     /**
-     * Applies an {@link AtomixLeaderElectorCommands.GetAllLeaderships} commit.
+     * Applies an {@link AtomixLeaderElectorOperations#GET_ALL_LEADERSHIPS} commit.
      * @param commit GetAllLeaderships commit
      * @return topic to leader mapping
      */
-    public Map<String, Leadership> allLeaderships(Commit<? extends GetAllLeaderships> commit) {
+    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) {
-            log.error("State machine operation failed", e);
+            getLogger().error("State machine operation failed", e);
             throw Throwables.propagate(e);
-        } finally {
-            commit.close();
         }
     }
 
@@ -366,11 +340,8 @@
         return electionState == null ? new LinkedList<>() : electionState.candidates();
     }
 
-    private void onSessionEnd(ServerSession session) {
-        Commit<? extends AtomixLeaderElectorCommands.Listen> listener = listeners.remove(session.id());
-        if (listener != null) {
-            listener.close();
-        }
+    private void onSessionEnd(RaftSession session) {
+        listeners.remove(session.sessionId().id());
         Set<String> topics = elections.keySet();
         List<Change<Leadership>> changes = Lists.newArrayList();
         topics.forEach(topic -> {
@@ -440,15 +411,15 @@
             this.termStartTime = termStartTime;
         }
 
-        public ElectionState cleanup(ServerSession session, Supplier<Long> termCounter) {
+        public ElectionState cleanup(RaftSession session, Supplier<Long> termCounter) {
             Optional<Registration> registration =
-                    registrations.stream().filter(r -> r.sessionId() == session.id()).findFirst();
+                    registrations.stream().filter(r -> r.sessionId() == session.sessionId().id()).findFirst();
             if (registration.isPresent()) {
                 List<Registration> updatedRegistrations =
                         registrations.stream()
-                        .filter(r -> r.sessionId() != session.id())
-                        .collect(Collectors.toList());
-                if (leader.sessionId() == session.id()) {
+                                .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),
@@ -471,8 +442,8 @@
             if (registration.isPresent()) {
                 List<Registration> updatedRegistrations =
                         registrations.stream()
-                        .filter(r -> !r.nodeId().equals(nodeId))
-                        .collect(Collectors.toList());
+                                .filter(r -> !r.nodeId().equals(nodeId))
+                                .collect(Collectors.toList());
                 if (leader.nodeId().equals(nodeId)) {
                     if (!updatedRegistrations.isEmpty()) {
                         return new ElectionState(updatedRegistrations,
@@ -522,14 +493,14 @@
 
         public ElectionState transferLeadership(NodeId nodeId, AtomicLong termCounter) {
             Registration newLeader = registrations.stream()
-                                                  .filter(r -> r.nodeId().equals(nodeId))
-                                                  .findFirst()
-                                                  .orElse(null);
+                    .filter(r -> r.nodeId().equals(nodeId))
+                    .findFirst()
+                    .orElse(null);
             if (newLeader != null) {
                 return new ElectionState(registrations,
-                                         newLeader,
-                                         termCounter.incrementAndGet(),
-                                         System.currentTimeMillis());
+                        newLeader,
+                        termCounter.incrementAndGet(),
+                        System.currentTimeMillis());
             } else {
                 return this;
             }
@@ -537,66 +508,33 @@
 
         public ElectionState promote(NodeId nodeId) {
             Registration registration = registrations.stream()
-                                                  .filter(r -> r.nodeId().equals(nodeId))
-                                                  .findFirst()
-                                                  .orElse(null);
+                    .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);
+                    .filter(r -> !r.nodeId().equals(nodeId))
+                    .forEach(updatedRegistrations::add);
             return new ElectionState(updatedRegistrations,
-                                    leader,
-                                    term,
-                                    termStartTime);
+                    leader,
+                    term,
+                    termStartTime);
 
         }
     }
 
     @Override
-    public void register(ServerSession session) {
-    }
-
-    @Override
-    public void unregister(ServerSession session) {
+    public void onExpire(RaftSession session) {
         onSessionEnd(session);
     }
 
     @Override
-    public void expire(ServerSession session) {
+    public void onClose(RaftSession session) {
         onSessionEnd(session);
     }
 
-    @Override
-    public void close(ServerSession session) {
-        onSessionEnd(session);
-    }
-
-    @Override
-    public void snapshot(SnapshotWriter writer) {
-        byte[] encodedTermCounters = serializer.encode(termCounters);
-        writer.writeInt(encodedTermCounters.length);
-        writer.write(encodedTermCounters);
-        byte[] encodedElections  = serializer.encode(elections);
-        writer.writeInt(encodedElections.length);
-        writer.write(encodedElections);
-        log.debug("Took state machine snapshot");
-    }
-
-    @Override
-    public void install(SnapshotReader reader) {
-        int encodedTermCountersSize = reader.readInt();
-        byte[] encodedTermCounters = new byte[encodedTermCountersSize];
-        reader.read(encodedTermCounters);
-        termCounters = serializer.decode(encodedTermCounters);
-        int encodedElectionsSize = reader.readInt();
-        byte[] encodedElections = new byte[encodedElectionsSize];
-        reader.read(encodedElections);
-        elections = serializer.decode(encodedElections);
-        log.debug("Reinstated state machine from snapshot");
-    }
-
     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/AtomixSerializerAdapter.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixSerializerAdapter.java
new file mode 100644
index 0000000..7eafdb6
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixSerializerAdapter.java
@@ -0,0 +1,39 @@
+/*
+ * Copyright 2017-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.store.primitives.resources.impl;
+
+import org.onosproject.store.service.Serializer;
+
+/**
+ * ONOS to Atomix serializer adapter.
+ */
+public class AtomixSerializerAdapter implements io.atomix.serializer.Serializer {
+    private final Serializer serializer;
+
+    public AtomixSerializerAdapter(Serializer serializer) {
+        this.serializer = serializer;
+    }
+
+    @Override
+    public <T> byte[] encode(T object) {
+        return serializer.encode(object);
+    }
+
+    @Override
+    public <T> T decode(byte[] bytes) {
+        return serializer.decode(bytes);
+    }
+}
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
index 569a597..e21ec8c 100644
--- 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
@@ -15,16 +15,8 @@
  */
 package org.onosproject.store.primitives.resources.impl;
 
-import static java.util.concurrent.Executors.newSingleThreadExecutor;
-import static org.onlab.util.Tools.groupedThreads;
-import static org.slf4j.LoggerFactory.getLogger;
-import io.atomix.copycat.client.CopycatClient;
-import io.atomix.resource.AbstractResource;
-import io.atomix.resource.ResourceTypeInfo;
-
 import java.util.Collection;
 import java.util.List;
-import java.util.Properties;
 import java.util.Timer;
 import java.util.concurrent.CompletableFuture;
 import java.util.concurrent.Executor;
@@ -34,63 +26,64 @@
 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.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Add;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Clear;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Complete;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Register;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Stats;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Take;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Unregister;
+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 com.google.common.collect.ImmutableList;
+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.
  */
-@ResourceTypeInfo(id = -154, factory = AtomixWorkQueueFactory.class)
-public class AtomixWorkQueue extends AbstractResource<AtomixWorkQueue>
-    implements WorkQueue<byte[]> {
+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());
-    public static final String TASK_AVAILABLE = "task-available";
     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);
 
-    protected AtomixWorkQueue(CopycatClient client, Properties options) {
-        super(client, options);
-    }
-
-    @Override
-    public String name() {
-        return null;
+    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 client.submit(new Clear());
-    }
-
-    @Override
-    public CompletableFuture<AtomixWorkQueue> open() {
-        return super.open().thenApply(result -> {
-            client.onStateChange(state -> {
-                if (state == CopycatClient.State.CONNECTED && isRegistered.get()) {
-                    client.submit(new Register());
-                }
-            });
-            client.onEvent(TASK_AVAILABLE, this::resumeWork);
-            return result;
-        });
+        return proxy.invoke(CLEAR);
     }
 
     @Override
@@ -98,7 +91,7 @@
         if (items.isEmpty()) {
             return CompletableFuture.completedFuture(null);
         }
-        return client.submit(new Add(items));
+        return proxy.invoke(ADD, SERIALIZER::encode, new Add(items));
     }
 
     @Override
@@ -106,7 +99,7 @@
         if (maxTasks <= 0) {
             return CompletableFuture.completedFuture(ImmutableList.of());
         }
-        return client.submit(new Take(maxTasks));
+        return proxy.invoke(TAKE, SERIALIZER::encode, new Take(maxTasks), SERIALIZER::decode);
     }
 
     @Override
@@ -114,21 +107,21 @@
         if (taskIds.isEmpty()) {
             return CompletableFuture.completedFuture(null);
         }
-        return client.submit(new Complete(taskIds));
+        return proxy.invoke(COMPLETE, SERIALIZER::encode, new Complete(taskIds));
     }
 
     @Override
     public CompletableFuture<Void> registerTaskProcessor(Consumer<byte[]> callback,
-                                          int parallelism,
-                                          Executor executor) {
+            int parallelism,
+            Executor executor) {
         Accumulator<String> completedTaskAccumulator =
                 new CompletedTaskAccumulator(timer, 50, 50); // TODO: make configurable
         taskProcessor.set(new TaskProcessor(callback,
-                                            parallelism,
-                                            executor,
-                                            completedTaskAccumulator));
+                parallelism,
+                executor,
+                completedTaskAccumulator));
         return register().thenCompose(v -> take(parallelism))
-                         .thenAccept(taskProcessor.get());
+                .thenAccept(taskProcessor.get());
     }
 
     @Override
@@ -138,7 +131,7 @@
 
     @Override
     public CompletableFuture<WorkQueueStats> stats() {
-        return client.submit(new Stats());
+        return proxy.invoke(STATS, SERIALIZER::decode);
     }
 
     private void resumeWork() {
@@ -147,15 +140,15 @@
             return;
         }
         this.take(activeProcessor.headRoom())
-            .whenCompleteAsync((tasks, e) -> activeProcessor.accept(tasks), executor);
+                .whenCompleteAsync((tasks, e) -> activeProcessor.accept(tasks), executor);
     }
 
     private CompletableFuture<Void> register() {
-        return client.submit(new Register()).thenRun(() -> isRegistered.set(true));
+        return proxy.invoke(REGISTER).thenRun(() -> isRegistered.set(true));
     }
 
     private CompletableFuture<Void> unregister() {
-        return client.submit(new Unregister()).thenRun(() -> isRegistered.set(false));
+        return proxy.invoke(UNREGISTER).thenRun(() -> isRegistered.set(false));
     }
 
     // TaskId accumulator for paced triggering of task completion calls.
@@ -178,9 +171,9 @@
         private final Accumulator<String> taskCompleter;
 
         public TaskProcessor(Consumer<byte[]> backingConsumer,
-                             int parallelism,
-                             Executor executor,
-                             Accumulator<String> taskCompleter) {
+                int parallelism,
+                Executor executor,
+                Accumulator<String> taskCompleter) {
             this.backingConsumer = backingConsumer;
             this.headRoom = new AtomicInteger(parallelism);
             this.executor = executor;
@@ -198,17 +191,17 @@
             }
             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();
-                    }
-                }));
+                    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/AtomixWorkQueueCommands.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueCommands.java
deleted file mode 100644
index 977470d..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueCommands.java
+++ /dev/null
@@ -1,273 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Laboratory
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.onosproject.store.primitives.resources.impl;
-
-import io.atomix.catalyst.buffer.BufferInput;
-import io.atomix.catalyst.buffer.BufferOutput;
-import io.atomix.catalyst.serializer.CatalystSerializable;
-import io.atomix.catalyst.serializer.SerializableTypeResolver;
-import io.atomix.catalyst.serializer.Serializer;
-import io.atomix.catalyst.serializer.SerializerRegistry;
-import io.atomix.copycat.Command;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-
-import org.onosproject.store.service.Task;
-import org.onosproject.store.service.WorkQueueStats;
-
-import com.google.common.base.MoreObjects;
-
-/**
- * {@link AtomixWorkQueue} resource state machine operations.
- */
-public final class AtomixWorkQueueCommands {
-
-    private AtomixWorkQueueCommands() {
-    }
-
-    /**
-     * Command to add a collection of tasks to the queue.
-     */
-    @SuppressWarnings("serial")
-    public static class Add implements Command<Void>, CatalystSerializable {
-
-        private Collection<byte[]> items;
-
-        private Add() {
-        }
-
-        public Add(Collection<byte[]> items) {
-            this.items = items;
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            buffer.writeInt(items.size());
-            items.forEach(task -> serializer.writeObject(task, buffer));
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            items = IntStream.range(0, buffer.readInt())
-                             .mapToObj(i -> serializer.<byte[]>readObject(buffer))
-                             .collect(Collectors.toCollection(ArrayList::new));
-        }
-
-        public Collection<byte[]> items() {
-            return items;
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.QUORUM;
-        }
-
-        @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 implements Command<Collection<Task<byte[]>>>, CatalystSerializable {
-
-        private int maxTasks;
-
-        private Take() {
-        }
-
-        public Take(int maxTasks) {
-            this.maxTasks = maxTasks;
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            buffer.writeInt(maxTasks);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            maxTasks = buffer.readInt();
-        }
-
-        public int maxTasks() {
-            return maxTasks;
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.QUORUM;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("maxTasks", maxTasks)
-                    .toString();
-        }
-    }
-
-    @SuppressWarnings("serial")
-    public static class Stats implements Command<WorkQueueStats>, CatalystSerializable {
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .toString();
-        }
-    }
-
-
-
-    @SuppressWarnings("serial")
-    public static class Register implements Command<Void>, CatalystSerializable {
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.QUORUM;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .toString();
-        }
-    }
-
-    @SuppressWarnings("serial")
-    public static class Unregister implements Command<Void>, CatalystSerializable {
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.TOMBSTONE;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .toString();
-        }
-    }
-
-    @SuppressWarnings("serial")
-    public static class Complete implements Command<Void>, CatalystSerializable {
-        private Collection<String> taskIds;
-
-        private Complete() {
-        }
-
-        public Complete(Collection<String> taskIds) {
-            this.taskIds = taskIds;
-        }
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-            serializer.writeObject(taskIds, buffer);
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-            taskIds = serializer.readObject(buffer);
-        }
-
-        public Collection<String> taskIds() {
-            return taskIds;
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.TOMBSTONE;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .add("taskIds", taskIds)
-                    .toString();
-        }
-    }
-
-    @SuppressWarnings("serial")
-    public static class Clear implements Command<Void>, CatalystSerializable {
-
-        @Override
-        public void writeObject(BufferOutput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public void readObject(BufferInput<?> buffer, Serializer serializer) {
-        }
-
-        @Override
-        public CompactionMode compaction() {
-            return CompactionMode.TOMBSTONE;
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                    .toString();
-        }
-    }
-
-    /**
-     * Work queue command type resolver.
-     */
-    public static class TypeResolver implements SerializableTypeResolver {
-        @Override
-        public void resolve(SerializerRegistry registry) {
-            registry.register(Register.class, -960);
-            registry.register(Unregister.class, -961);
-            registry.register(Take.class, -962);
-            registry.register(Add.class, -963);
-            registry.register(Complete.class, -964);
-            registry.register(Stats.class, -965);
-            registry.register(Clear.class, -966);
-        }
-    }
-}
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
new file mode 100644
index 0000000..50b7366
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueEvents.java
@@ -0,0 +1,42 @@
+/*
+ * Copyright 2017-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.store.primitives.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("taskAvailable");
+
+    private final String id;
+
+    AtomixWorkQueueEvents(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)
+            .build("AtomixWorkQueueEvents");
+}
diff --git a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueFactory.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueFactory.java
deleted file mode 100644
index 0c61b2e..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueFactory.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Laboratory
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.onosproject.store.primitives.resources.impl;
-
-import io.atomix.catalyst.serializer.SerializableTypeResolver;
-import io.atomix.copycat.client.CopycatClient;
-import io.atomix.resource.ResourceFactory;
-import io.atomix.resource.ResourceStateMachine;
-
-import java.util.Properties;
-
-/**
- * {@link AtomixWorkQueue} resource factory.
- */
-public class AtomixWorkQueueFactory implements ResourceFactory<AtomixWorkQueue> {
-
-    @Override
-    public SerializableTypeResolver createSerializableTypeResolver() {
-        return new AtomixWorkQueueCommands.TypeResolver();
-    }
-
-    @Override
-    public ResourceStateMachine createStateMachine(Properties config) {
-        return new AtomixWorkQueueState(config);
-    }
-
-    @Override
-    public AtomixWorkQueue createInstance(CopycatClient client, Properties properties) {
-        return new AtomixWorkQueue(client, properties);
-    }
-}
\ No newline at end of file
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
new file mode 100644
index 0000000..a7a0df1
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueOperations.java
@@ -0,0 +1,148 @@
+/*
+ * Copyright 2016-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.store.primitives.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("stats", OperationType.QUERY),
+    REGISTER("register", OperationType.COMMAND),
+    UNREGISTER("unregister", OperationType.COMMAND),
+    ADD("add", OperationType.COMMAND),
+    TAKE("take", OperationType.COMMAND),
+    COMPLETE("complete", OperationType.COMMAND),
+    CLEAR("clear", OperationType.COMMAND);
+
+    private final String id;
+    private final OperationType type;
+
+    AtomixWorkQueueOperations(String id, OperationType type) {
+        this.id = id;
+        this.type = type;
+    }
+
+    @Override
+    public String id() {
+        return id;
+    }
+
+    @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
new file mode 100644
index 0000000..6458ec8
--- /dev/null
+++ b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueService.java
@@ -0,0 +1,243 @@
+/*
+ * Copyright 2016-present Open Networking Laboratory
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.onosproject.store.primitives.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.base.Throwables;
+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, getSessions().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) {
+            getLogger().warn("State machine update failed", e);
+            throw Throwables.propagate(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) {
+            getLogger().warn("State machine update failed", e);
+            throw Throwables.propagate(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/AtomixWorkQueueState.java b/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueState.java
deleted file mode 100644
index 82f28e8..0000000
--- a/core/store/primitives/src/main/java/org/onosproject/store/primitives/resources/impl/AtomixWorkQueueState.java
+++ /dev/null
@@ -1,305 +0,0 @@
-/*
- * Copyright 2016-present Open Networking Laboratory
- *
- * Licensed under the Apache License, Version 2.0 (the "License");
- * you may not use this file except in compliance with the License.
- * You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.onosproject.store.primitives.resources.impl;
-
-import static org.slf4j.LoggerFactory.getLogger;
-import io.atomix.copycat.server.Commit;
-import io.atomix.copycat.server.Snapshottable;
-import io.atomix.copycat.server.StateMachineExecutor;
-import io.atomix.copycat.server.session.ServerSession;
-import io.atomix.copycat.server.session.SessionListener;
-import io.atomix.copycat.server.storage.snapshot.SnapshotReader;
-import io.atomix.copycat.server.storage.snapshot.SnapshotWriter;
-import io.atomix.resource.ResourceStateMachine;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Properties;
-import java.util.Queue;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.function.Consumer;
-import java.util.stream.Collectors;
-import java.util.stream.IntStream;
-
-import org.onlab.util.CountDownCompleter;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Add;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Clear;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Complete;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Register;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Stats;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Take;
-import org.onosproject.store.primitives.resources.impl.AtomixWorkQueueCommands.Unregister;
-import org.onosproject.store.service.Task;
-import org.onosproject.store.service.WorkQueueStats;
-import org.slf4j.Logger;
-
-import com.google.common.base.MoreObjects;
-import com.google.common.base.Throwables;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Queues;
-import com.google.common.util.concurrent.AtomicLongMap;
-
-/**
- * State machine for {@link AtomixWorkQueue} resource.
- */
-public class AtomixWorkQueueState  extends ResourceStateMachine implements SessionListener, Snapshottable {
-
-    private final Logger log = getLogger(getClass());
-
-    private final AtomicLong totalCompleted = new AtomicLong(0);
-
-    private final Queue<TaskHolder> unassignedTasks = Queues.newArrayDeque();
-    private final Map<String, TaskAssignment> assignments = Maps.newHashMap();
-    private final Map<Long, Commit<? extends Register>> registeredWorkers = Maps.newHashMap();
-    private final AtomicLongMap<Long> activeTasksPerSession = AtomicLongMap.create();
-
-    protected AtomixWorkQueueState(Properties config) {
-        super(config);
-    }
-
-    @Override
-    protected void configure(StateMachineExecutor executor) {
-        executor.register(Stats.class, this::stats);
-        executor.register(Register.class, (Consumer<Commit<Register>>) this::register);
-        executor.register(Unregister.class, (Consumer<Commit<Unregister>>) this::unregister);
-        executor.register(Add.class, (Consumer<Commit<Add>>) this::add);
-        executor.register(Take.class, this::take);
-        executor.register(Complete.class, (Consumer<Commit<Complete>>) this::complete);
-        executor.register(Clear.class, (Consumer<Commit<Clear>>) this::clear);
-    }
-
-    protected WorkQueueStats stats(Commit<? extends Stats> commit) {
-        try {
-            return WorkQueueStats.builder()
-                    .withTotalCompleted(totalCompleted.get())
-                    .withTotalPending(unassignedTasks.size())
-                    .withTotalInProgress(assignments.size())
-                    .build();
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected void clear(Commit<? extends Clear> commit) {
-        try {
-            unassignedTasks.forEach(TaskHolder::complete);
-            unassignedTasks.clear();
-            assignments.values().forEach(TaskAssignment::markComplete);
-            assignments.clear();
-            registeredWorkers.values().forEach(Commit::close);
-            registeredWorkers.clear();
-            activeTasksPerSession.clear();
-            totalCompleted.set(0);
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected void register(Commit<? extends Register> commit) {
-        long sessionId = commit.session().id();
-        if (registeredWorkers.putIfAbsent(sessionId, commit) != null) {
-            commit.close();
-        }
-    }
-
-    protected void unregister(Commit<? extends Unregister> commit) {
-        try {
-            Commit<? extends Register> registerCommit = registeredWorkers.remove(commit.session().id());
-            if (registerCommit != null) {
-                registerCommit.close();
-            }
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected void add(Commit<? extends Add> commit) {
-        Collection<byte[]> items = commit.operation().items();
-
-        // Create a CountDownCompleter that will close the commit when all tasks
-        // submitted as part of it are completed.
-        CountDownCompleter<Commit<? extends Add>> referenceTracker =
-                new CountDownCompleter<>(commit, items.size(), Commit::close);
-
-        AtomicInteger itemIndex = new AtomicInteger(0);
-        items.forEach(item -> {
-            String taskId = String.format("%d:%d:%d", commit.session().id(),
-                                                      commit.index(),
-                                                      itemIndex.getAndIncrement());
-            unassignedTasks.add(new TaskHolder(new Task<>(taskId, item), referenceTracker));
-        });
-
-        // Send an event to all sessions that have expressed interest in task processing
-        // and are not actively processing a task.
-        registeredWorkers.values()
-                         .stream()
-                         .map(Commit::session)
-                         .forEach(session -> session.publish(AtomixWorkQueue.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().id();
-            int maxTasks = commit.operation().maxTasks();
-            return IntStream.range(0, Math.min(maxTasks, unassignedTasks.size()))
-                            .mapToObj(i -> {
-                                TaskHolder holder = unassignedTasks.poll();
-                                String taskId = holder.task().taskId();
-                                TaskAssignment assignment = new TaskAssignment(sessionId, holder);
-
-                                // bookkeeping
-                                assignments.put(taskId, assignment);
-                                activeTasksPerSession.incrementAndGet(sessionId);
-
-                                return holder.task();
-                            })
-                            .collect(Collectors.toCollection(ArrayList::new));
-        } catch (Exception e) {
-            log.warn("State machine update failed", e);
-            throw Throwables.propagate(e);
-        } finally {
-            commit.close();
-        }
-    }
-
-    protected void complete(Commit<? extends Complete> commit) {
-        long sessionId = commit.session().id();
-        try {
-            commit.operation().taskIds().forEach(taskId -> {
-                TaskAssignment assignment = assignments.get(taskId);
-                if (assignment != null && assignment.sessionId() == sessionId) {
-                    assignments.remove(taskId).markComplete();
-                    // bookkeeping
-                    totalCompleted.incrementAndGet();
-                    activeTasksPerSession.decrementAndGet(sessionId);
-                }
-            });
-        } catch (Exception e) {
-            log.warn("State machine update failed", e);
-            throw Throwables.propagate(e);
-        } finally {
-            commit.close();
-        }
-    }
-
-    @Override
-    public void register(ServerSession session) {
-    }
-
-    @Override
-    public void unregister(ServerSession session) {
-        evictWorker(session.id());
-    }
-
-    @Override
-    public void expire(ServerSession session) {
-        evictWorker(session.id());
-    }
-
-    @Override
-    public void close(ServerSession session) {
-        evictWorker(session.id());
-    }
-
-    @Override
-    public void snapshot(SnapshotWriter writer) {
-        writer.writeLong(totalCompleted.get());
-    }
-
-    @Override
-    public void install(SnapshotReader reader) {
-        totalCompleted.set(reader.readLong());
-    }
-
-    private void evictWorker(long sessionId) {
-        Commit<? extends Register> commit = registeredWorkers.remove(sessionId);
-        if (commit != null) {
-            commit.close();
-        }
-
-        // 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.taskHolder());
-                iter.remove();
-            }
-        }
-
-        // Bookkeeping
-        activeTasksPerSession.remove(sessionId);
-        activeTasksPerSession.removeAllZeros();
-    }
-
-    private class TaskHolder {
-
-        private final Task<byte[]> task;
-        private final CountDownCompleter<Commit<? extends Add>> referenceTracker;
-
-        public TaskHolder(Task<byte[]> delegate, CountDownCompleter<Commit<? extends Add>> referenceTracker) {
-            this.task = delegate;
-            this.referenceTracker = referenceTracker;
-        }
-
-        public Task<byte[]> task() {
-            return task;
-        }
-
-        public void complete() {
-            referenceTracker.countDown();
-        }
-    }
-
-    private class TaskAssignment {
-        private final long sessionId;
-        private final TaskHolder taskHolder;
-
-        public TaskAssignment(long sessionId, TaskHolder taskHolder) {
-            this.sessionId = sessionId;
-            this.taskHolder = taskHolder;
-        }
-
-        public long sessionId() {
-            return sessionId;
-        }
-
-        public TaskHolder taskHolder() {
-            return taskHolder;
-        }
-
-        public void markComplete() {
-            taskHolder.complete();
-        }
-
-        @Override
-        public String toString() {
-            return MoreObjects.toStringHelper(getClass())
-                              .add("sessionId", sessionId)
-                              .add("taskHolder", taskHolder)
-                              .toString();
-        }
-    }
-}
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
index a351ad5..ba248ba 100644
--- 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
@@ -41,7 +41,7 @@
 public class DefaultDocumentTree<V> implements DocumentTree<V> {
 
     private static final DocumentPath ROOT_PATH = DocumentPath.from("root");
-    private final DefaultDocumentTreeNode<V> root;
+    final DefaultDocumentTreeNode<V> root;
     private final Supplier<Long> versionSupplier;
 
     public DefaultDocumentTree() {
@@ -55,6 +55,11 @@
         this.versionSupplier = versionSupplier;
     }
 
+    DefaultDocumentTree(Supplier<Long> versionSupplier, DefaultDocumentTreeNode<V> root) {
+        this.root = root;
+        this.versionSupplier = versionSupplier;
+    }
+
     @Override
     public DocumentPath root() {
         return ROOT_PATH;
@@ -195,4 +200,4 @@
             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
index 4b4852b..6568779 100644
--- 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
@@ -41,9 +41,9 @@
     private final DocumentTreeNode<V> parent;
 
     public DefaultDocumentTreeNode(DocumentPath key,
-                            V value,
-                            long version,
-                            DocumentTreeNode<V> parent) {
+            V value,
+            long version,
+            DocumentTreeNode<V> parent) {
         this.key = checkNotNull(key);
         this.value = new Versioned<>(value, version);
         this.parent = parent;
@@ -137,9 +137,9 @@
     public String toString() {
         MoreObjects.ToStringHelper helper =
                 MoreObjects.toStringHelper(getClass())
-                .add("parent", this.parent)
-                .add("key", this.key)
-                .add("value", this.value);
+                        .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) +
@@ -147,4 +147,4 @@
         }
         return helper.toString();
     }
-}
+}
\ No newline at end of file