New flowrule store that utilizes a simple mechanism to periodically backsup its flowentries.
For efficiency reasons backups are only run for those devices whose flow entries are updated (since last backup) or if the device master/backup has changed.
This backup mechanism will be a short term solution until we get to a more close to real-time backup approach.
This change also disables the flowrule store based on hazelcast.

Change-Id: Iaae08852edee20b999ff97c60ca8bc6576e645f6
diff --git a/core/store/dist/src/main/java/org/onosproject/store/flow/ReplicaInfo.java b/core/store/dist/src/main/java/org/onosproject/store/flow/ReplicaInfo.java
index 66c081e..d33ac203 100644
--- a/core/store/dist/src/main/java/org/onosproject/store/flow/ReplicaInfo.java
+++ b/core/store/dist/src/main/java/org/onosproject/store/flow/ReplicaInfo.java
@@ -17,8 +17,8 @@
 
 import static com.google.common.base.Preconditions.checkNotNull;
 
-import java.util.Collection;
 import java.util.Collections;
+import java.util.List;
 
 import org.onosproject.cluster.NodeId;
 
@@ -30,15 +30,15 @@
 public final class ReplicaInfo {
 
     private final Optional<NodeId> master;
-    private final Collection<NodeId> backups;
+    private final List<NodeId> backups;
 
     /**
      * Creates a ReplicaInfo instance.
      *
      * @param master NodeId of the node where the master copy should be
-     * @param backups collection of NodeId, where backup copies should be placed
+     * @param backups list of NodeId, where backup copies should be placed
      */
-    public ReplicaInfo(NodeId master, Collection<NodeId> backups) {
+    public ReplicaInfo(NodeId master, List<NodeId> backups) {
         this.master = Optional.fromNullable(master);
         this.backups = checkNotNull(backups);
     }
@@ -57,7 +57,7 @@
      *
      * @return collection of NodeId, where backup copies should be placed
      */
-    public Collection<NodeId> backups() {
+    public List<NodeId> backups() {
         return backups;
     }
 
diff --git a/core/store/dist/src/main/java/org/onosproject/store/flow/impl/DistributedFlowRuleStore.java b/core/store/dist/src/main/java/org/onosproject/store/flow/impl/DistributedFlowRuleStore.java
index 7f8769b..f1ac1f8 100644
--- a/core/store/dist/src/main/java/org/onosproject/store/flow/impl/DistributedFlowRuleStore.java
+++ b/core/store/dist/src/main/java/org/onosproject/store/flow/impl/DistributedFlowRuleStore.java
@@ -108,7 +108,7 @@
 /**
  * Manages inventory of flow rules using a distributed state management protocol.
  */
-@Component(immediate = true)
+@Component(immediate = false, enabled = false)
 @Service
 public class DistributedFlowRuleStore
         extends AbstractHazelcastStore<FlowRuleBatchEvent, FlowRuleStoreDelegate>
diff --git a/core/store/dist/src/main/java/org/onosproject/store/flow/impl/FlowStoreMessageSubjects.java b/core/store/dist/src/main/java/org/onosproject/store/flow/impl/FlowStoreMessageSubjects.java
index 46c2d26..041053c 100644
--- a/core/store/dist/src/main/java/org/onosproject/store/flow/impl/FlowStoreMessageSubjects.java
+++ b/core/store/dist/src/main/java/org/onosproject/store/flow/impl/FlowStoreMessageSubjects.java
@@ -36,5 +36,8 @@
         = new MessageSubject("peer-forward-remove-flow-entry");
 
     public static final MessageSubject REMOTE_APPLY_COMPLETED
-            = new MessageSubject("peer-apply-completed");
+        = new MessageSubject("peer-apply-completed");
+
+    public static final MessageSubject FLOW_TABLE_BACKUP
+        = new MessageSubject("peer-flow-table-backup");
 }
diff --git a/core/store/dist/src/main/java/org/onosproject/store/flow/impl/NewDistributedFlowRuleStore.java b/core/store/dist/src/main/java/org/onosproject/store/flow/impl/NewDistributedFlowRuleStore.java
new file mode 100644
index 0000000..cd85a75
--- /dev/null
+++ b/core/store/dist/src/main/java/org/onosproject/store/flow/impl/NewDistributedFlowRuleStore.java
@@ -0,0 +1,684 @@
+ /*
+ * Copyright 2014-2015 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.flow.impl;
+
+import com.google.common.base.Objects;
+import com.google.common.collect.Iterables;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.Futures;
+
+import org.apache.felix.scr.annotations.Activate;
+import org.apache.felix.scr.annotations.Component;
+import org.apache.felix.scr.annotations.Deactivate;
+import org.apache.felix.scr.annotations.Modified;
+import org.apache.felix.scr.annotations.Property;
+import org.apache.felix.scr.annotations.Reference;
+import org.apache.felix.scr.annotations.ReferenceCardinality;
+import org.apache.felix.scr.annotations.Service;
+import org.onlab.util.KryoNamespace;
+import org.onlab.util.NewConcurrentHashMap;
+import org.onlab.util.Tools;
+import org.onosproject.cfg.ComponentConfigService;
+import org.onosproject.cluster.ClusterService;
+import org.onosproject.cluster.NodeId;
+import org.onosproject.core.CoreService;
+import org.onosproject.core.IdGenerator;
+import org.onosproject.mastership.MastershipService;
+import org.onosproject.net.DeviceId;
+import org.onosproject.net.device.DeviceService;
+import org.onosproject.net.flow.CompletedBatchOperation;
+import org.onosproject.net.flow.DefaultFlowEntry;
+import org.onosproject.net.flow.FlowEntry;
+import org.onosproject.net.flow.FlowEntry.FlowEntryState;
+import org.onosproject.net.flow.FlowId;
+import org.onosproject.net.flow.FlowRule;
+import org.onosproject.net.flow.FlowRuleBatchEntry;
+import org.onosproject.net.flow.FlowRuleBatchEntry.FlowRuleOperation;
+import org.onosproject.net.flow.FlowRuleBatchEvent;
+import org.onosproject.net.flow.FlowRuleBatchOperation;
+import org.onosproject.net.flow.FlowRuleBatchRequest;
+import org.onosproject.net.flow.FlowRuleEvent;
+import org.onosproject.net.flow.FlowRuleEvent.Type;
+import org.onosproject.net.flow.FlowRuleService;
+import org.onosproject.net.flow.FlowRuleStore;
+import org.onosproject.net.flow.FlowRuleStoreDelegate;
+import org.onosproject.net.flow.StoredFlowEntry;
+import org.onosproject.store.AbstractStore;
+import org.onosproject.store.cluster.messaging.ClusterCommunicationService;
+import org.onosproject.store.cluster.messaging.ClusterMessage;
+import org.onosproject.store.cluster.messaging.ClusterMessageHandler;
+import org.onosproject.store.flow.ReplicaInfo;
+import org.onosproject.store.flow.ReplicaInfoService;
+import org.onosproject.store.serializers.KryoSerializer;
+import org.onosproject.store.serializers.StoreSerializer;
+import org.onosproject.store.serializers.impl.DistributedStoreSerializers;
+import org.osgi.service.component.ComponentContext;
+import org.slf4j.Logger;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Dictionary;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.stream.Collectors;
+
+import static org.apache.commons.lang3.concurrent.ConcurrentUtils.createIfAbsentUnchecked;
+import static com.google.common.base.Strings.isNullOrEmpty;
+import static org.onlab.util.Tools.get;
+import static org.onlab.util.Tools.groupedThreads;
+import static org.onosproject.net.flow.FlowRuleEvent.Type.RULE_REMOVED;
+import static org.onosproject.store.flow.impl.FlowStoreMessageSubjects.*;
+import static org.slf4j.LoggerFactory.getLogger;
+
+/**
+ * Manages inventory of flow rules using a distributed state management protocol.
+ */
+@Component(immediate = true, enabled = true)
+@Service
+public class NewDistributedFlowRuleStore
+        extends AbstractStore<FlowRuleBatchEvent, FlowRuleStoreDelegate>
+        implements FlowRuleStore {
+
+    private final Logger log = getLogger(getClass());
+
+    private static final int MESSAGE_HANDLER_THREAD_POOL_SIZE = 8;
+    private static final boolean DEFAULT_BACKUP_ENABLED = true;
+    private static final long FLOW_RULE_STORE_TIMEOUT_MILLIS = 5000;
+
+    @Property(name = "msgHandlerPoolSize", intValue = MESSAGE_HANDLER_THREAD_POOL_SIZE,
+            label = "Number of threads in the message handler pool")
+    private int msgHandlerPoolSize = MESSAGE_HANDLER_THREAD_POOL_SIZE;
+
+    @Property(name = "backupEnabled", boolValue = DEFAULT_BACKUP_ENABLED,
+            label = "Indicates whether backups are enabled or not")
+    private boolean backupEnabled = DEFAULT_BACKUP_ENABLED;
+
+    private InternalFlowTable flowTable = new InternalFlowTable();
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected ReplicaInfoService replicaInfoManager;
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected ClusterCommunicationService clusterCommunicator;
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected ClusterService clusterService;
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected DeviceService deviceService;
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected CoreService coreService;
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected ComponentConfigService configService;
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected MastershipService mastershipService;
+
+    private Map<Long, NodeId> pendingResponses = Maps.newConcurrentMap();
+    private ExecutorService messageHandlingExecutor;
+
+    private final ScheduledExecutorService backupSenderExecutor =
+            Executors.newSingleThreadScheduledExecutor(groupedThreads("onos/flow", "backup-sender"));
+
+    protected static final StoreSerializer SERIALIZER = new KryoSerializer() {
+        @Override
+        protected void setupKryoPool() {
+            serializerPool = KryoNamespace.newBuilder()
+                    .register(DistributedStoreSerializers.STORE_COMMON)
+                    .nextId(DistributedStoreSerializers.STORE_CUSTOM_BEGIN)
+                    .register(FlowRuleEvent.class)
+                    .register(FlowRuleEvent.Type.class)
+                    .build();
+        }
+    };
+
+    private IdGenerator idGenerator;
+    private NodeId local;
+
+    @Activate
+    public void activate(ComponentContext context) {
+        configService.registerProperties(getClass());
+
+        idGenerator = coreService.getIdGenerator(FlowRuleService.FLOW_OP_TOPIC);
+
+        local = clusterService.getLocalNode().id();
+
+        messageHandlingExecutor = Executors.newFixedThreadPool(
+                msgHandlerPoolSize, groupedThreads("onos/store/flow", "message-handlers"));
+
+        registerMessageHandlers(messageHandlingExecutor);
+
+        backupSenderExecutor.scheduleWithFixedDelay(() -> flowTable.backup(), 0, 2000, TimeUnit.MILLISECONDS);
+
+        logConfig("Started");
+    }
+
+    @Deactivate
+    public void deactivate(ComponentContext context) {
+        configService.unregisterProperties(getClass(), false);
+        unregisterMessageHandlers();
+        messageHandlingExecutor.shutdownNow();
+        backupSenderExecutor.shutdownNow();
+        log.info("Stopped");
+    }
+
+    @SuppressWarnings("rawtypes")
+    @Modified
+    public void modified(ComponentContext context) {
+        if (context == null) {
+            backupEnabled = DEFAULT_BACKUP_ENABLED;
+            logConfig("Default config");
+            return;
+        }
+
+        Dictionary properties = context.getProperties();
+        int newPoolSize;
+        boolean newBackupEnabled;
+        try {
+            String s = get(properties, "msgHandlerPoolSize");
+            newPoolSize = isNullOrEmpty(s) ? msgHandlerPoolSize : Integer.parseInt(s.trim());
+
+            s = get(properties, "backupEnabled");
+            newBackupEnabled = isNullOrEmpty(s) ? backupEnabled : Boolean.parseBoolean(s.trim());
+
+        } catch (NumberFormatException | ClassCastException e) {
+            newPoolSize = MESSAGE_HANDLER_THREAD_POOL_SIZE;
+            newBackupEnabled = DEFAULT_BACKUP_ENABLED;
+        }
+
+        if (newBackupEnabled != backupEnabled) {
+            backupEnabled = newBackupEnabled;
+        }
+        if (newPoolSize != msgHandlerPoolSize) {
+            msgHandlerPoolSize = newPoolSize;
+            ExecutorService oldMsgHandler = messageHandlingExecutor;
+            messageHandlingExecutor = Executors.newFixedThreadPool(
+                    msgHandlerPoolSize, groupedThreads("onos/store/flow", "message-handlers"));
+
+            // replace previously registered handlers.
+            registerMessageHandlers(messageHandlingExecutor);
+            oldMsgHandler.shutdown();
+        }
+        logConfig("Reconfigured");
+    }
+
+    private void registerMessageHandlers(ExecutorService executor) {
+
+        clusterCommunicator.addSubscriber(APPLY_BATCH_FLOWS, new OnStoreBatch(), executor);
+        clusterCommunicator.<FlowRuleBatchEvent>addSubscriber(
+                REMOTE_APPLY_COMPLETED, SERIALIZER::decode, this::notifyDelegate, executor);
+        clusterCommunicator.addSubscriber(
+                GET_FLOW_ENTRY, SERIALIZER::decode, flowTable::getFlowEntry, SERIALIZER::encode, executor);
+        clusterCommunicator.addSubscriber(
+                GET_DEVICE_FLOW_ENTRIES, SERIALIZER::decode, flowTable::getFlowEntries, SERIALIZER::encode, executor);
+        clusterCommunicator.addSubscriber(
+                REMOVE_FLOW_ENTRY, SERIALIZER::decode, this::removeFlowRuleInternal, SERIALIZER::encode, executor);
+        clusterCommunicator.addSubscriber(
+                REMOVE_FLOW_ENTRY, SERIALIZER::decode, this::removeFlowRuleInternal, SERIALIZER::encode, executor);
+        clusterCommunicator.addSubscriber(
+                FLOW_TABLE_BACKUP, SERIALIZER::decode, flowTable::onBackupReceipt, executor);
+    }
+
+    private void unregisterMessageHandlers() {
+        clusterCommunicator.removeSubscriber(REMOVE_FLOW_ENTRY);
+        clusterCommunicator.removeSubscriber(GET_DEVICE_FLOW_ENTRIES);
+        clusterCommunicator.removeSubscriber(GET_FLOW_ENTRY);
+        clusterCommunicator.removeSubscriber(APPLY_BATCH_FLOWS);
+        clusterCommunicator.removeSubscriber(REMOTE_APPLY_COMPLETED);
+        clusterCommunicator.removeSubscriber(FLOW_TABLE_BACKUP);
+    }
+
+    private void logConfig(String prefix) {
+        log.info("{} with msgHandlerPoolSize = {}; backupEnabled = {}",
+                 prefix, msgHandlerPoolSize, backupEnabled);
+    }
+
+    // This is not a efficient operation on a distributed sharded
+    // flow store. We need to revisit the need for this operation or at least
+    // make it device specific.
+    @Override
+    public int getFlowRuleCount() {
+        AtomicInteger sum = new AtomicInteger(0);
+        deviceService.getDevices().forEach(device -> sum.addAndGet(Iterables.size(getFlowEntries(device.id()))));
+        return sum.get();
+    }
+
+    @Override
+    public FlowEntry getFlowEntry(FlowRule rule) {
+
+        ReplicaInfo replicaInfo = replicaInfoManager.getReplicaInfoFor(rule.deviceId());
+        NodeId master = replicaInfo.master().orNull();
+
+        if (master == null) {
+            log.warn("Failed to getFlowEntry: No master for {}", rule.deviceId());
+            return null;
+        }
+
+        if (Objects.equal(local, master)) {
+            return flowTable.getFlowEntry(rule);
+        }
+
+        log.trace("Forwarding getFlowEntry to {}, which is the primary (master) for device {}",
+                  master, rule.deviceId());
+
+        return Tools.futureGetOrElse(clusterCommunicator.sendAndReceive(rule,
+                                    FlowStoreMessageSubjects.GET_FLOW_ENTRY,
+                                    SERIALIZER::encode,
+                                    SERIALIZER::decode,
+                                    master),
+                               FLOW_RULE_STORE_TIMEOUT_MILLIS,
+                               TimeUnit.MILLISECONDS,
+                               null);
+    }
+
+    @Override
+    public Iterable<FlowEntry> getFlowEntries(DeviceId deviceId) {
+
+        ReplicaInfo replicaInfo = replicaInfoManager.getReplicaInfoFor(deviceId);
+        NodeId master = replicaInfo.master().orNull();
+
+        if (master == null) {
+            log.warn("Failed to getFlowEntries: No master for {}", deviceId);
+            return Collections.emptyList();
+        }
+
+        if (Objects.equal(local, master)) {
+            return flowTable.getFlowEntries(deviceId);
+        }
+
+        log.trace("Forwarding getFlowEntries to {}, which is the primary (master) for device {}",
+                  master, deviceId);
+
+        return Tools.futureGetOrElse(clusterCommunicator.sendAndReceive(deviceId,
+                                    FlowStoreMessageSubjects.GET_DEVICE_FLOW_ENTRIES,
+                                    SERIALIZER::encode,
+                                    SERIALIZER::decode,
+                                    master),
+                               FLOW_RULE_STORE_TIMEOUT_MILLIS,
+                               TimeUnit.MILLISECONDS,
+                               Collections.emptyList());
+    }
+
+    @Override
+    public void storeFlowRule(FlowRule rule) {
+        storeBatch(new FlowRuleBatchOperation(
+                Arrays.asList(new FlowRuleBatchEntry(FlowRuleOperation.ADD, rule)),
+                rule.deviceId(), idGenerator.getNewId()));
+    }
+
+    @Override
+    public void storeBatch(FlowRuleBatchOperation operation) {
+        if (operation.getOperations().isEmpty()) {
+            notifyDelegate(FlowRuleBatchEvent.completed(
+                    new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
+                    new CompletedBatchOperation(true, Collections.emptySet(), operation.deviceId())));
+            return;
+        }
+
+        DeviceId deviceId = operation.deviceId();
+
+        ReplicaInfo replicaInfo = replicaInfoManager.getReplicaInfoFor(deviceId);
+        NodeId master = replicaInfo.master().orNull();
+
+        if (master == null) {
+            log.warn("No master for {} : flows will be marked for removal", deviceId);
+
+            updateStoreInternal(operation);
+
+            notifyDelegate(FlowRuleBatchEvent.completed(
+                    new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
+                    new CompletedBatchOperation(true, Collections.emptySet(), operation.deviceId())));
+            return;
+        }
+
+        if (Objects.equal(local, master)) {
+            storeBatchInternal(operation);
+            return;
+        }
+
+        log.trace("Forwarding storeBatch to {}, which is the primary (master) for device {}",
+                  master, deviceId);
+
+        if (!clusterCommunicator.unicast(operation,
+                                         APPLY_BATCH_FLOWS,
+                                         SERIALIZER::encode,
+                                         master)) {
+            log.warn("Failed to storeBatch: {} to {}", operation, replicaInfo.master());
+
+            Set<FlowRule> allFailures = operation.getOperations().stream()
+                    .map(op -> op.target())
+                    .collect(Collectors.toSet());
+
+            notifyDelegate(FlowRuleBatchEvent.completed(
+                    new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
+                    new CompletedBatchOperation(false, allFailures, deviceId)));
+            return;
+        }
+    }
+
+    private void storeBatchInternal(FlowRuleBatchOperation operation) {
+
+        final DeviceId did = operation.deviceId();
+        //final Collection<FlowEntry> ft = flowTable.getFlowEntries(did);
+        Set<FlowRuleBatchEntry> currentOps = updateStoreInternal(operation);
+        if (currentOps.isEmpty()) {
+            batchOperationComplete(FlowRuleBatchEvent.completed(
+                    new FlowRuleBatchRequest(operation.id(), Collections.emptySet()),
+                    new CompletedBatchOperation(true, Collections.emptySet(), did)));
+            return;
+        }
+
+        notifyDelegate(FlowRuleBatchEvent.requested(new
+                           FlowRuleBatchRequest(operation.id(),
+                                                currentOps), operation.deviceId()));
+    }
+
+    private Set<FlowRuleBatchEntry> updateStoreInternal(FlowRuleBatchOperation operation) {
+        return operation.getOperations().stream().map(
+                op -> {
+                    StoredFlowEntry entry;
+                    switch (op.operator()) {
+                        case ADD:
+                            entry = new DefaultFlowEntry(op.target());
+                            // always add requested FlowRule
+                            // Note: 2 equal FlowEntry may have different treatment
+                            flowTable.remove(entry.deviceId(), entry);
+                            flowTable.add(entry);
+
+                            return op;
+                        case REMOVE:
+                            entry = flowTable.getFlowEntry(op.target());
+                            if (entry != null) {
+                                entry.setState(FlowEntryState.PENDING_REMOVE);
+                                return op;
+                            }
+                            break;
+                        case MODIFY:
+                            //TODO: figure this out at some point
+                            break;
+                        default:
+                            log.warn("Unknown flow operation operator: {}", op.operator());
+                    }
+                    return null;
+                }
+        ).filter(op -> op != null).collect(Collectors.toSet());
+    }
+
+    @Override
+    public void deleteFlowRule(FlowRule rule) {
+        storeBatch(
+                new FlowRuleBatchOperation(
+                        Arrays.asList(
+                                new FlowRuleBatchEntry(
+                                        FlowRuleOperation.REMOVE,
+                                        rule)), rule.deviceId(), idGenerator.getNewId()));
+    }
+
+    @Override
+    public FlowRuleEvent addOrUpdateFlowRule(FlowEntry rule) {
+        ReplicaInfo replicaInfo = replicaInfoManager.getReplicaInfoFor(rule.deviceId());
+        if (Objects.equal(local, replicaInfo.master().orNull())) {
+            return addOrUpdateFlowRuleInternal(rule);
+        }
+
+        log.warn("Tried to update FlowRule {} state,"
+                         + " while the Node was not the master.", rule);
+        return null;
+    }
+
+    private FlowRuleEvent addOrUpdateFlowRuleInternal(FlowEntry rule) {
+        // check if this new rule is an update to an existing entry
+        StoredFlowEntry stored = flowTable.getFlowEntry(rule);
+        if (stored != null) {
+            stored.setBytes(rule.bytes());
+            stored.setLife(rule.life());
+            stored.setPackets(rule.packets());
+            if (stored.state() == FlowEntryState.PENDING_ADD) {
+                stored.setState(FlowEntryState.ADDED);
+                return new FlowRuleEvent(Type.RULE_ADDED, rule);
+            }
+            return new FlowRuleEvent(Type.RULE_UPDATED, rule);
+        }
+
+        // TODO: Confirm if this behavior is correct. See SimpleFlowRuleStore
+        // TODO: also update backup if the behavior is correct.
+        flowTable.add(rule);
+        return null;
+    }
+
+    @Override
+    public FlowRuleEvent removeFlowRule(FlowEntry rule) {
+        final DeviceId deviceId = rule.deviceId();
+        ReplicaInfo replicaInfo = replicaInfoManager.getReplicaInfoFor(deviceId);
+        NodeId master = replicaInfo.master().orNull();
+
+        if (Objects.equal(local, master)) {
+            // bypass and handle it locally
+            return removeFlowRuleInternal(rule);
+        }
+
+        if (master == null) {
+            log.warn("Failed to removeFlowRule: No master for {}", deviceId);
+            // TODO: revisit if this should be null (="no-op") or Exception
+            return null;
+        }
+
+        log.trace("Forwarding removeFlowRule to {}, which is the master for device {}",
+                  master, deviceId);
+
+        return Futures.get(clusterCommunicator.sendAndReceive(
+                               rule,
+                               REMOVE_FLOW_ENTRY,
+                               SERIALIZER::encode,
+                               SERIALIZER::decode,
+                               master),
+                           FLOW_RULE_STORE_TIMEOUT_MILLIS,
+                           TimeUnit.MILLISECONDS,
+                           RuntimeException.class);
+    }
+
+    private FlowRuleEvent removeFlowRuleInternal(FlowEntry rule) {
+        final DeviceId deviceId = rule.deviceId();
+        // This is where one could mark a rule as removed and still keep it in the store.
+        final boolean removed = flowTable.remove(deviceId, rule); //flowEntries.remove(deviceId, rule);
+        return removed ? new FlowRuleEvent(RULE_REMOVED, rule) : null;
+    }
+
+    @Override
+    public void batchOperationComplete(FlowRuleBatchEvent event) {
+        //FIXME: need a per device pending response
+        NodeId nodeId = pendingResponses.remove(event.subject().batchId());
+        if (nodeId == null) {
+            notifyDelegate(event);
+        } else {
+            // TODO check unicast return value
+            clusterCommunicator.unicast(event, REMOTE_APPLY_COMPLETED, SERIALIZER::encode, nodeId);
+            //error log: log.warn("Failed to respond to peer for batch operation result");
+        }
+    }
+
+    private final class OnStoreBatch implements ClusterMessageHandler {
+
+        @Override
+        public void handle(final ClusterMessage message) {
+            FlowRuleBatchOperation operation = SERIALIZER.decode(message.payload());
+            log.debug("received batch request {}", operation);
+
+            final DeviceId deviceId = operation.deviceId();
+            ReplicaInfo replicaInfo = replicaInfoManager.getReplicaInfoFor(deviceId);
+            if (!local.equals(replicaInfo.master().orNull())) {
+
+                Set<FlowRule> failures = new HashSet<>(operation.size());
+                for (FlowRuleBatchEntry op : operation.getOperations()) {
+                    failures.add(op.target());
+                }
+                CompletedBatchOperation allFailed = new CompletedBatchOperation(false, failures, deviceId);
+                // This node is no longer the master, respond as all failed.
+                // TODO: we might want to wrap response in envelope
+                // to distinguish sw programming failure and hand over
+                // it make sense in the latter case to retry immediately.
+                message.respond(SERIALIZER.encode(allFailed));
+                return;
+            }
+
+            pendingResponses.put(operation.id(), message.sender());
+            storeBatchInternal(operation);
+        }
+    }
+
+    private class InternalFlowTable {
+
+        private final ConcurrentMap<DeviceId, ConcurrentMap<FlowId, Set<StoredFlowEntry>>>
+                flowEntries = new ConcurrentHashMap<>();
+
+        private final Map<DeviceId, Long> lastBackupTimes = Maps.newConcurrentMap();
+        private final Map<DeviceId, Long> lastUpdateTimes = Maps.newConcurrentMap();
+        private final Map<DeviceId, NodeId> lastBackupNodes = Maps.newConcurrentMap();
+
+        private NewConcurrentHashMap<FlowId, Set<StoredFlowEntry>> lazyEmptyFlowTable() {
+            return NewConcurrentHashMap.<FlowId, Set<StoredFlowEntry>>ifNeeded();
+        }
+
+        /**
+         * Returns the flow table for specified device.
+         *
+         * @param deviceId identifier of the device
+         * @return Map representing Flow Table of given device.
+         */
+        private ConcurrentMap<FlowId, Set<StoredFlowEntry>> getFlowTable(DeviceId deviceId) {
+            return createIfAbsentUnchecked(flowEntries, deviceId, lazyEmptyFlowTable());
+        }
+
+        private Set<StoredFlowEntry> getFlowEntriesInternal(DeviceId deviceId, FlowId flowId) {
+            return getFlowTable(deviceId).computeIfAbsent(flowId, id -> Sets.newCopyOnWriteArraySet());
+        }
+
+        private StoredFlowEntry getFlowEntryInternal(FlowRule rule) {
+            Set<StoredFlowEntry> flowEntries = getFlowEntriesInternal(rule.deviceId(), rule.id());
+            return flowEntries.stream()
+                              .filter(entry -> Objects.equal(entry, rule))
+                              .findAny()
+                              .orElse(null);
+        }
+
+        private Set<FlowEntry> getFlowEntriesInternal(DeviceId deviceId) {
+            Set<FlowEntry> result = Sets.newHashSet();
+            getFlowTable(deviceId).values().forEach(result::addAll);
+            return result;
+        }
+
+        public StoredFlowEntry getFlowEntry(FlowRule rule) {
+            return getFlowEntryInternal(rule);
+        }
+
+        public Set<FlowEntry> getFlowEntries(DeviceId deviceId) {
+            return getFlowEntriesInternal(deviceId);
+        }
+
+        public void add(FlowEntry rule) {
+            getFlowEntriesInternal(rule.deviceId(), rule.id()).add((StoredFlowEntry) rule);
+            lastUpdateTimes.put(rule.deviceId(), System.currentTimeMillis());
+        }
+
+        public boolean remove(DeviceId deviceId, FlowEntry rule) {
+            try {
+                return getFlowEntriesInternal(deviceId, rule.id()).remove(rule);
+            } finally {
+                lastUpdateTimes.put(deviceId, System.currentTimeMillis());
+            }
+        }
+
+        private NodeId getBackupNode(DeviceId deviceId) {
+            List<NodeId> deviceStandbys = replicaInfoManager.getReplicaInfoFor(deviceId).backups();
+            // pick the standby which is most likely to become next master
+            return deviceStandbys.isEmpty() ? null : deviceStandbys.get(0);
+        }
+
+        private void backup() {
+            //TODO: Force backup when backups change.
+            try {
+                // determine the set of devices that we need to backup during this run.
+                Set<DeviceId> devicesToBackup = mastershipService.getDevicesOf(local)
+                            .stream()
+                            .filter(deviceId -> {
+                                Long lastBackupTime = lastBackupTimes.get(deviceId);
+                                Long lastUpdateTime = lastUpdateTimes.get(deviceId);
+                                NodeId lastBackupNode = lastBackupNodes.get(deviceId);
+                                return lastBackupTime == null
+                                        ||  !Objects.equal(lastBackupNode, getBackupNode(deviceId))
+                                        || (lastUpdateTime != null && lastUpdateTime > lastBackupTime);
+                            })
+                            .collect(Collectors.toSet());
+
+                // compute a mapping from node to the set of devices whose flow entries it should backup
+                Map<NodeId, Set<DeviceId>> devicesToBackupByNode = Maps.newHashMap();
+                devicesToBackup.forEach(deviceId -> {
+                    NodeId backupLocation = getBackupNode(deviceId);
+                    if (backupLocation != null) {
+                        devicesToBackupByNode.computeIfAbsent(backupLocation, nodeId -> Sets.newHashSet())
+                                             .add(deviceId);
+                    }
+                });
+
+                // send the device flow entries to their respective backup nodes
+                devicesToBackupByNode.forEach((nodeId, deviceIds) -> {
+                    Map<DeviceId, ConcurrentMap<FlowId, Set<StoredFlowEntry>>> deviceFlowEntries =
+                            Maps.newConcurrentMap();
+                    flowEntries.forEach((key, value) -> {
+                        if (deviceIds.contains(key)) {
+                            deviceFlowEntries.put(key, value);
+                        }
+                    });
+                    clusterCommunicator.unicast(deviceFlowEntries,
+                            FLOW_TABLE_BACKUP,
+                            SERIALIZER::encode,
+                            nodeId);
+                });
+
+                // update state for use in subsequent run.
+                devicesToBackupByNode.forEach((node, devices) -> {
+                    devices.forEach(id -> {
+                        lastBackupTimes.put(id, System.currentTimeMillis());
+                        lastBackupNodes.put(id, node);
+                    });
+                });
+            } catch (Exception e) {
+                log.error("Backup failed.", e);
+            }
+        }
+
+        private void onBackupReceipt(Map<DeviceId, Map<FlowId, Set<StoredFlowEntry>>> flowTables) {
+            Set<DeviceId> managedDevices = mastershipService.getDevicesOf(local);
+            Maps.filterKeys(flowTables, managedDevices::contains).forEach((deviceId, flowTable) -> {
+                Map<FlowId, Set<StoredFlowEntry>> deviceFlowTable = getFlowTable(deviceId);
+                deviceFlowTable.clear();
+                deviceFlowTable.putAll(flowTable);
+            });
+        }
+    }
+}
diff --git a/core/store/dist/src/main/java/org/onosproject/store/flow/impl/ReplicaInfoManager.java b/core/store/dist/src/main/java/org/onosproject/store/flow/impl/ReplicaInfoManager.java
index e6acfea..f94afe0 100644
--- a/core/store/dist/src/main/java/org/onosproject/store/flow/impl/ReplicaInfoManager.java
+++ b/core/store/dist/src/main/java/org/onosproject/store/flow/impl/ReplicaInfoManager.java
@@ -21,6 +21,8 @@
 import static org.onosproject.store.flow.ReplicaInfoEvent.Type.BACKUPS_CHANGED;
 
 import java.util.Collections;
+import java.util.List;
+
 import org.apache.felix.scr.annotations.Activate;
 import org.apache.felix.scr.annotations.Component;
 import org.apache.felix.scr.annotations.Deactivate;
@@ -28,6 +30,7 @@
 import org.apache.felix.scr.annotations.ReferenceCardinality;
 import org.apache.felix.scr.annotations.Service;
 import org.onosproject.cluster.NodeId;
+import org.onosproject.cluster.RoleInfo;
 import org.onosproject.event.AbstractListenerRegistry;
 import org.onosproject.event.EventDeliveryService;
 import org.onosproject.mastership.MastershipEvent;
@@ -76,9 +79,7 @@
 
     @Override
     public ReplicaInfo getReplicaInfoFor(DeviceId deviceId) {
-        // TODO: populate backup List when we reach the point we need them.
-        return new ReplicaInfo(mastershipService.getMasterFor(deviceId),
-                               Collections.<NodeId>emptyList());
+        return buildFromRoleInfo(mastershipService.getNodesFor(deviceId));
     }
 
     @Override
@@ -91,13 +92,17 @@
         listenerRegistry.removeListener(checkNotNull(listener));
     }
 
+    private static ReplicaInfo buildFromRoleInfo(RoleInfo roles) {
+        List<NodeId> backups = roles.backups() == null ?
+                Collections.emptyList() : roles.backups();
+        return new ReplicaInfo(roles.master(), backups);
+    }
+
     final class InternalMastershipListener implements MastershipListener {
 
         @Override
         public void event(MastershipEvent event) {
-            final ReplicaInfo replicaInfo
-                = new ReplicaInfo(event.roleInfo().master(),
-                                  event.roleInfo().backups());
+            final ReplicaInfo replicaInfo = buildFromRoleInfo(event.roleInfo());
 
             switch (event.type()) {
             case MASTER_CHANGED:
diff --git a/core/store/dist/src/main/java/org/onosproject/store/hz/StoreManager.java b/core/store/dist/src/main/java/org/onosproject/store/hz/StoreManager.java
index 253b424..dc159af 100644
--- a/core/store/dist/src/main/java/org/onosproject/store/hz/StoreManager.java
+++ b/core/store/dist/src/main/java/org/onosproject/store/hz/StoreManager.java
@@ -38,7 +38,7 @@
 /**
  * Auxiliary bootstrap of distributed store.
  */
-@Component(immediate = true)
+@Component(immediate = false, enabled = false)
 @Service
 public class StoreManager implements StoreService {
 
diff --git a/core/store/dist/src/main/java/org/onosproject/store/mastership/impl/ConsistentDeviceMastershipStore.java b/core/store/dist/src/main/java/org/onosproject/store/mastership/impl/ConsistentDeviceMastershipStore.java
index 0d85784..5bc55af 100644
--- a/core/store/dist/src/main/java/org/onosproject/store/mastership/impl/ConsistentDeviceMastershipStore.java
+++ b/core/store/dist/src/main/java/org/onosproject/store/mastership/impl/ConsistentDeviceMastershipStore.java
@@ -218,6 +218,8 @@
         NodeId master = null;
         final List<NodeId> standbys = Lists.newLinkedList();
 
+        List<NodeId> candidates = leadershipService.getCandidates(createDeviceMastershipTopic(deviceId));
+
         for (Map.Entry<NodeId, MastershipRole> entry : roles.entrySet()) {
             if (entry.getValue() == MastershipRole.MASTER) {
                 master = entry.getKey();
@@ -226,7 +228,9 @@
             }
         }
 
-        return new RoleInfo(master, standbys);
+        List<NodeId> sortedStandbyList = candidates.stream().filter(standbys::contains).collect(Collectors.toList());
+
+        return new RoleInfo(master, sortedStandbyList);
     }
 
     @Override
diff --git a/core/store/dist/src/test/java/org/onosproject/store/flow/impl/ReplicaInfoManagerTest.java b/core/store/dist/src/test/java/org/onosproject/store/flow/impl/ReplicaInfoManagerTest.java
index b3d9f3f..9bc1153 100644
--- a/core/store/dist/src/test/java/org/onosproject/store/flow/impl/ReplicaInfoManagerTest.java
+++ b/core/store/dist/src/test/java/org/onosproject/store/flow/impl/ReplicaInfoManagerTest.java
@@ -153,6 +153,11 @@
         }
 
         @Override
+        public RoleInfo getNodesFor(DeviceId deviceId) {
+            return new RoleInfo(masters.get(deviceId), Collections.emptyList());
+        }
+
+        @Override
         public void addListener(MastershipListener listener) {
             mastershipListenerRegistry.addListener(listener);
         }
diff --git a/core/store/serializers/src/main/java/org/onosproject/store/serializers/KryoNamespaces.java b/core/store/serializers/src/main/java/org/onosproject/store/serializers/KryoNamespaces.java
index 3f2f669..debdde2 100644
--- a/core/store/serializers/src/main/java/org/onosproject/store/serializers/KryoNamespaces.java
+++ b/core/store/serializers/src/main/java/org/onosproject/store/serializers/KryoNamespaces.java
@@ -18,6 +18,7 @@
 import com.google.common.collect.ImmutableList;
 import com.google.common.collect.ImmutableMap;
 import com.google.common.collect.ImmutableSet;
+
 import org.onlab.packet.ChassisId;
 import org.onlab.packet.Ip4Address;
 import org.onlab.packet.Ip4Prefix;
@@ -170,6 +171,8 @@
 import java.util.HashSet;
 import java.util.LinkedList;
 import java.util.Optional;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CopyOnWriteArraySet;
 
 public final class KryoNamespaces {
 
@@ -191,6 +194,8 @@
                       ImmutableMap.of("a", 1).getClass(),
                       ImmutableMap.of("R", 2, "D", 2).getClass())
             .register(HashMap.class)
+            .register(ConcurrentHashMap.class)
+            .register(CopyOnWriteArraySet.class)
             .register(ArrayList.class,
                       LinkedList.class,
                       HashSet.class