Added support for firing up multiple raft partitions + Workaround for an issue where db calls timeout when a raft cluster node is down.

Change-Id: I67406da34c8a96b8ab9371d4d9b14653edfd2e2d
diff --git a/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DatabaseDefinition.java b/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DatabaseDefinition.java
new file mode 100644
index 0000000..46754e5
--- /dev/null
+++ b/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DatabaseDefinition.java
@@ -0,0 +1,65 @@
+/*
+ * Copyright 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.consistent.impl;
+
+import java.util.Map;
+import java.util.Set;
+
+import org.onosproject.store.cluster.impl.NodeInfo;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+
+/**
+ * Partitioned database configuration.
+ */
+public class DatabaseDefinition {
+    private Map<String, Set<NodeInfo>> partitions;
+    private Set<NodeInfo> nodes;
+
+    /**
+     * Creates a new DatabaseDefinition.
+     * @param partitions partition map
+     * @param nodes set of nodes
+     * @return database definition
+     */
+    public static DatabaseDefinition from(Map<String, Set<NodeInfo>> partitions, Set<NodeInfo> nodes) {
+        checkNotNull(partitions);
+        checkNotNull(nodes);
+        DatabaseDefinition definition = new DatabaseDefinition();
+        definition.partitions = ImmutableMap.copyOf(partitions);
+        definition.nodes = ImmutableSet.copyOf(nodes);
+        return definition;
+    }
+
+    /**
+     * Returns the map of database partitions.
+     * @return db partition map
+     */
+    public Map<String, Set<NodeInfo>> getPartitions() {
+        return partitions;
+    }
+
+    /**
+     * Returns the set of nodes.
+     * @return nodes
+     */
+    public Set<NodeInfo> getNodes() {
+        return nodes;
+    }
+}
\ No newline at end of file
diff --git a/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DatabaseDefinitionStore.java b/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DatabaseDefinitionStore.java
index dce978e..e54fe3c 100644
--- a/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DatabaseDefinitionStore.java
+++ b/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DatabaseDefinitionStore.java
@@ -16,39 +16,16 @@
 
 package org.onosproject.store.consistent.impl;
 
-import static com.google.common.base.Preconditions.checkArgument;
 import static com.google.common.base.Preconditions.checkNotNull;
-import static org.slf4j.LoggerFactory.getLogger;
-
 import java.io.File;
 import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-
-import org.onosproject.cluster.DefaultControllerNode;
-import org.onosproject.cluster.NodeId;
-import org.onlab.packet.IpAddress;
-import org.slf4j.Logger;
-
-import com.fasterxml.jackson.core.JsonEncoding;
-import com.fasterxml.jackson.core.JsonFactory;
-import com.fasterxml.jackson.databind.JsonNode;
 import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.node.ArrayNode;
-import com.fasterxml.jackson.databind.node.ObjectNode;
-import com.google.common.collect.Maps;
 
 /**
  * Allows for reading and writing partitioned database definition as a JSON file.
  */
 public class DatabaseDefinitionStore {
 
-    private final Logger log = getLogger(getClass());
-
     private final File definitionfile;
 
     /**
@@ -57,7 +34,7 @@
      * @param filePath location of the definition file
      */
     public DatabaseDefinitionStore(String filePath) {
-        definitionfile = new File(filePath);
+        definitionfile = new File(checkNotNull(filePath));
     }
 
     /**
@@ -70,72 +47,27 @@
     }
 
     /**
-     * Returns the Map from database partition name to set of initial active member nodes.
+     * Returns the database definition.
      *
-     * @return Map from partition name to set of active member nodes
+     * @return database definition
      * @throws IOException when I/O exception of some sort has occurred.
      */
-    public Map<String, Set<DefaultControllerNode>> read() throws IOException {
-
-        final Map<String, Set<DefaultControllerNode>> partitions = Maps.newHashMap();
-
-        final ObjectMapper mapper = new ObjectMapper();
-        final ObjectNode tabletNodes = (ObjectNode) mapper.readTree(definitionfile);
-        final Iterator<Entry<String, JsonNode>> fields = tabletNodes.fields();
-        while (fields.hasNext()) {
-            final Entry<String, JsonNode> next = fields.next();
-            final Set<DefaultControllerNode> nodes = new HashSet<>();
-            final Iterator<JsonNode> elements = next.getValue().elements();
-            while (elements.hasNext()) {
-                ObjectNode nodeDef = (ObjectNode) elements.next();
-                nodes.add(new DefaultControllerNode(new NodeId(nodeDef.get("id").asText()),
-                                                    IpAddress.valueOf(nodeDef.get("ip").asText()),
-                                                    nodeDef.get("tcpPort").asInt(DatabaseManager.COPYCAT_TCP_PORT)));
-            }
-
-            partitions.put(next.getKey(), nodes);
-        }
-        return partitions;
+    public DatabaseDefinition read() throws IOException {
+        ObjectMapper mapper = new ObjectMapper();
+        DatabaseDefinition definition = mapper.readValue(definitionfile, DatabaseDefinition.class);
+        return definition;
     }
 
     /**
-     * Updates the Map from database partition name to set of member nodes.
+     * Writes the specified database definition to file.
      *
-     * @param partitionName name of the database partition to update
-     * @param nodes set of initial member nodes
+     * @param definition database definition
      * @throws IOException when I/O exception of some sort has occurred.
      */
-    public void write(String partitionName, Set<DefaultControllerNode> nodes) throws IOException {
-        checkNotNull(partitionName);
-        checkArgument(partitionName.isEmpty(), "Partition name cannot be empty");
-
-        // load current
-        Map<String, Set<DefaultControllerNode>> config;
-        try {
-            config = read();
-        } catch (IOException e) {
-            log.info("Reading partition config failed, assuming empty definition.");
-            config = new HashMap<>();
-        }
-        // update with specified
-        config.put(partitionName, nodes);
-
+    public void write(DatabaseDefinition definition) throws IOException {
+        checkNotNull(definition);
         // write back to file
         final ObjectMapper mapper = new ObjectMapper();
-        final ObjectNode partitionNodes = mapper.createObjectNode();
-        for (Entry<String, Set<DefaultControllerNode>> tablet : config.entrySet()) {
-            ArrayNode nodeDefs = mapper.createArrayNode();
-            partitionNodes.set(tablet.getKey(), nodeDefs);
-
-            for (DefaultControllerNode node : tablet.getValue()) {
-                ObjectNode nodeDef = mapper.createObjectNode();
-                nodeDef.put("id", node.id().toString())
-                       .put("ip", node.ip().toString())
-                       .put("tcpPort", node.tcpPort());
-                nodeDefs.add(nodeDef);
-            }
-        }
-        mapper.writeTree(new JsonFactory().createGenerator(definitionfile, JsonEncoding.UTF8),
-                         partitionNodes);
+        mapper.writeValue(definitionfile, definition);
     }
 }
diff --git a/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DatabaseManager.java b/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DatabaseManager.java
index 09cc645..7ccdd80 100644
--- a/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DatabaseManager.java
+++ b/core/store/dist/src/main/java/org/onosproject/store/consistent/impl/DatabaseManager.java
@@ -31,8 +31,7 @@
 import org.apache.felix.scr.annotations.ReferenceCardinality;
 import org.apache.felix.scr.annotations.Service;
 import org.onosproject.cluster.ClusterService;
-import org.onosproject.cluster.ControllerNode;
-import org.onosproject.cluster.DefaultControllerNode;
+import org.onosproject.store.cluster.impl.NodeInfo;
 import org.onosproject.store.service.ConsistentMap;
 import org.onosproject.store.service.PartitionInfo;
 import org.onosproject.store.service.Serializer;
@@ -69,8 +68,8 @@
     @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
     protected ClusterService clusterService;
 
-    protected String nodeToUri(ControllerNode node) {
-        return String.format("tcp://%s:%d", node.ip(), COPYCAT_TCP_PORT);
+    protected String nodeToUri(NodeInfo node) {
+        return String.format("tcp://%s:%d", node.getIp(), COPYCAT_TCP_PORT);
     }
 
     @Activate
@@ -82,12 +81,11 @@
         File file = new File(CONFIG_DIR, PARTITION_DEFINITION_FILE);
         log.info("Loading database definition: {}", file.getAbsolutePath());
 
-        DatabaseDefinitionStore databaseDef = new DatabaseDefinitionStore(file);
-        Map<String, Set<DefaultControllerNode>> partitionMap;
+        Map<String, Set<NodeInfo>> partitionMap;
         try {
-            partitionMap = databaseDef.read();
+            DatabaseDefinitionStore databaseDef = new DatabaseDefinitionStore(file);
+            partitionMap = databaseDef.read().getPartitions();
         } catch (IOException e) {
-            log.error("Failed to load database config {}", file);
             throw new IllegalStateException("Failed to load database config", e);
         }
 
@@ -99,7 +97,7 @@
                     .map(this::nodeToUri)
                     .toArray(String[]::new);
 
-        String localNodeUri = nodeToUri(clusterService.getLocalNode());
+        String localNodeUri = nodeToUri(NodeInfo.of(clusterService.getLocalNode()));
 
         ClusterConfig clusterConfig = new ClusterConfig()
             .withProtocol(new NettyTcpProtocol()