<?xml version="1.0" encoding="UTF-8"?>
<commit>
  <added type="array">
    <added>
      <filename>src/java/voldemort/client/ClientConfig.java</filename>
    </added>
    <added>
      <filename>src/java/voldemort/client/ClientThreadPool.java</filename>
    </added>
    <added>
      <filename>src/java/voldemort/client/protocol/RequestFormat.java</filename>
    </added>
    <added>
      <filename>src/java/voldemort/client/protocol/RequestFormatFactory.java</filename>
    </added>
    <added>
      <filename>src/java/voldemort/client/protocol/RequestFormatType.java</filename>
    </added>
    <added>
      <filename>src/java/voldemort/client/protocol/package.html</filename>
    </added>
    <added>
      <filename>src/java/voldemort/client/protocol/pb/ProtoBuffClientRequestFormat.java</filename>
    </added>
    <added>
      <filename>src/java/voldemort/client/protocol/pb/ProtoUtils.java</filename>
    </added>
    <added>
      <filename>src/java/voldemort/client/protocol/pb/VProto.java</filename>
    </added>
    <added>
      <filename>src/java/voldemort/client/protocol/pb/package.html</filename>
    </added>
    <added>
      <filename>src/java/voldemort/client/protocol/vold/VoldemortNativeClientRequestFormat.java</filename>
    </added>
    <added>
      <filename>src/java/voldemort/server/ServiceType.java</filename>
    </added>
    <added>
      <filename>src/java/voldemort/server/StoreRepository.java</filename>
    </added>
    <added>
      <filename>src/java/voldemort/server/protocol/AbstractRequestHandler.java</filename>
    </added>
    <added>
      <filename>src/java/voldemort/server/protocol/RequestHandler.java</filename>
    </added>
    <added>
      <filename>src/java/voldemort/server/protocol/RequestHandlerFactory.java</filename>
    </added>
    <added>
      <filename>src/java/voldemort/server/protocol/package.html</filename>
    </added>
    <added>
      <filename>src/java/voldemort/server/protocol/pb/ProtoBuffRequestHandler.java</filename>
    </added>
    <added>
      <filename>src/java/voldemort/server/protocol/pb/package.html</filename>
    </added>
    <added>
      <filename>src/java/voldemort/server/protocol/vold/VoldemortNativeRequestHandler.java</filename>
    </added>
    <added>
      <filename>src/java/voldemort/server/socket/SocketServerSession.java</filename>
    </added>
    <added>
      <filename>src/java/voldemort/utils/DaemonThreadFactory.java</filename>
    </added>
    <added>
      <filename>src/proto/voldemort-client.proto</filename>
    </added>
    <added>
      <filename>src/proto/voldemort_protocol.proto</filename>
    </added>
    <added>
      <filename>test/unit/voldemort/protocol/AbstractRequestFormatTest.java</filename>
    </added>
    <added>
      <filename>test/unit/voldemort/protocol/pb/ProtocolBuffersRequestFormatTest.java</filename>
    </added>
    <added>
      <filename>test/unit/voldemort/protocol/vold/VoldemortNativeRequestFormatTest.java</filename>
    </added>
    <added>
      <filename>test/unit/voldemort/server/storage/StorageServiceTest.java</filename>
    </added>
  </added>
  <modified type="array">
    <modified>
      <diff>@@ -1,6 +1,7 @@
 &lt;?xml version=&quot;1.0&quot; encoding=&quot;UTF-8&quot;?&gt;
 &lt;classpath&gt;
 	&lt;classpathentry kind=&quot;src&quot; path=&quot;src/java&quot;/&gt;
+	&lt;classpathentry kind=&quot;src&quot; path=&quot;contrib/mongodb/example&quot;/&gt;
 	&lt;classpathentry kind=&quot;src&quot; path=&quot;contrib/batch-indexer/test&quot;/&gt;
 	&lt;classpathentry kind=&quot;src&quot; path=&quot;contrib/mongodb/src/java&quot;/&gt;
 	&lt;classpathentry kind=&quot;src&quot; path=&quot;contrib/mongodb/test&quot;/&gt;</diff>
      <filename>.classpath</filename>
    </modified>
    <modified>
      <diff>@@ -57,4 +57,18 @@ Background Resources
 - Origin of vector clocks: http://research.microsoft.com/users/lamport/pubs/time-clocks.pdf
 - Brewer's conjecture: http://citeseer.ist.psu.edu/544596.html
 
-Current build is from r19 
\ No newline at end of file
+Current build is from r19 
+
+
+Supporting other clients
+- Each store is available via all protcols, they are seperated by port
+- Wire format vs. protocol (HTTP vs. Tcp/IP), protocol buffers
+- How to abstract wire format?
+- A given serialization type may or may not be supported by the client language
+
+Socket servers share threadpool? 
+
+A WireProtocol takes bytes and creates a voldemort request, and takes objects to create a voldemort response.
+Likewise the client does the opposite
+
+Connector.handleRequest()
\ No newline at end of file</diff>
      <filename>NOTES</filename>
    </modified>
    <modified>
      <diff>@@ -30,4 +30,4 @@ done
 CLASSPATH=$CLASSPATH:$base_dir/dist/resources
 
 export CLASSPATH
-java -Xmx2G -server -cp $CLASSPATH ${1} ${2} ${3} ${4} ${5} ${6} ${7} 
\ No newline at end of file
+java -Xmx2G -server -Dcom.sun.management.jmxremote -cp $CLASSPATH ${1} ${2} ${3} ${4} ${5} ${6} ${7} 
\ No newline at end of file</diff>
      <filename>bin/run-class.sh</filename>
    </modified>
    <modified>
      <diff>@@ -1,6 +1,7 @@
 ## Main source 
 src.dir=src
 java.dir=src/java
+protobuff.dir=src/proto
 lib.dir=lib
 classes.dir=dist/classes
 resources.dir=dist/resources
@@ -31,4 +32,4 @@ tomcat.manager.password=tomcat
 tomcat.context=/voldemort
 
 ## Release
-curr.release=0.51
+curr.release=0.52</diff>
      <filename>build.properties</filename>
    </modified>
    <modified>
      <diff>@@ -94,6 +94,19 @@
       &lt;fileset dir=&quot;${testclasses.dir}&quot; /&gt;
     &lt;/jar&gt;
   &lt;/target&gt;
+  
+  &lt;target name=&quot;protobuff&quot; description=&quot;Generate source files from .proto files&quot;&gt;
+    &lt;path id=&quot;proto-files&quot;&gt;
+      &lt;fileset dir=&quot;${protobuff.dir}&quot; /&gt;
+    &lt;/path&gt;
+    &lt;property name=&quot;proto.path&quot; location=&quot;${protobuff.dir}&quot;/&gt; 
+    &lt;property name=&quot;javaout.path&quot; location=&quot;${java.dir}&quot;/&gt;
+    &lt;exec executable=&quot;protoc&quot; failonerror=&quot;true&quot;&gt;
+      &lt;arg value=&quot;--proto_path=${proto.path}&quot;/&gt;
+      &lt;arg value=&quot;--java_out=${javaout.path}&quot;/&gt;
+      &lt;arg pathref=&quot;proto-files&quot;/&gt;
+    &lt;/exec&gt;
+  &lt;/target&gt;
 
   &lt;target name=&quot;jar&quot; depends=&quot;build&quot; description=&quot;Build server jar file&quot;&gt;
     &lt;jar destfile=&quot;${dist.dir}/${name}-${curr.release}.jar&quot;&gt;
@@ -115,6 +128,16 @@
       &lt;/fileset&gt;
     &lt;/jar&gt;
   &lt;/target&gt;
+  
+  &lt;target name=&quot;war&quot; depends=&quot;build&quot; description=&quot;Build server war file&quot;&gt;
+    &lt;war destfile=&quot;${dist.dir}/${name}.war&quot; webxml=&quot;web.xml&quot; basedir=&quot;${classes.dir}&quot;&gt;
+      &lt;classes dir=&quot;${classes.dir}&quot;/&gt;
+      &lt;lib dir=&quot;${lib.dir}&quot;&gt;
+        &lt;exclude name=&quot;${lib.dir}/servlet-api.jar&quot;/&gt;
+        &lt;exclude name=&quot;${lib.dir}/src/**&quot;/&gt;
+      &lt;/lib&gt;
+    &lt;/war&gt;
+  &lt;/target&gt;
 
   &lt;target name=&quot;contrib-build&quot; depends=&quot;build, buildtest&quot; description=&quot;Compile contrib packages (java and test) &quot;&gt;
     &lt;replace-dir dir=&quot;${contrib.classes.dir}&quot; /&gt;
@@ -206,16 +229,6 @@
   &lt;target name=&quot;junit-all&quot; depends=&quot;junit, contrib-junit&quot; description=&quot;Run All junit tests including contrib.&quot;&gt;
   &lt;/target&gt;
 
-  &lt;target name=&quot;war&quot; depends=&quot;build&quot; description=&quot;Build server war file&quot;&gt;
-    &lt;war destfile=&quot;${dist.dir}/${name}.war&quot; webxml=&quot;web.xml&quot; basedir=&quot;${classes.dir}&quot;&gt;
-      &lt;classes dir=&quot;${classes.dir}&quot; /&gt;
-      &lt;lib dir=&quot;${lib.dir}&quot;&gt;
-        &lt;exclude name=&quot;${lib.dir}/servlet-api.jar&quot; /&gt;
-        &lt;exclude name=&quot;${lib.dir}/src/**&quot; /&gt;
-      &lt;/lib&gt;
-    &lt;/war&gt;
-  &lt;/target&gt;
-
   &lt;macrodef name=&quot;make-javadocs&quot;&gt;
     &lt;attribute name=&quot;dir&quot; /&gt;
     &lt;attribute name=&quot;packagenames&quot; /&gt;</diff>
      <filename>build.xml</filename>
    </modified>
    <modified>
      <diff>@@ -37,11 +37,11 @@ import voldemort.utils.ByteUtils;
 public abstract class ReadOnlyBatchIndexMapper&lt;K, V&gt; implements
         Mapper&lt;K, V, BytesWritable, BytesWritable&gt; {
 
-    private Cluster _cluster = null;
-    private StoreDefinition _storeDef = null;
-    private ConsistentRoutingStrategy _routingStrategy = null;
-    private Serializer&lt;Object&gt; _keySerializer;
-    private Serializer&lt;Object&gt; _valueSerializer;
+    private Cluster cluster = null;
+    private StoreDefinition storeDef = null;
+    private ConsistentRoutingStrategy routingStrategy = null;
+    private Serializer&lt;Object&gt; keySerializer;
+    private Serializer&lt;Object&gt; valueSerializer;
 
     public abstract Object getKeyBytes(K key, V value);
 
@@ -51,10 +51,10 @@ public abstract class ReadOnlyBatchIndexMapper&lt;K, V&gt; implements
                     V value,
                     OutputCollector&lt;BytesWritable, BytesWritable&gt; output,
                     Reporter reporter) throws IOException {
-        byte[] keyBytes = _keySerializer.toBytes(getKeyBytes(key, value));
-        byte[] valBytes = _valueSerializer.toBytes(getValueBytes(key, value));
+        byte[] keyBytes = keySerializer.toBytes(getKeyBytes(key, value));
+        byte[] valBytes = valueSerializer.toBytes(getValueBytes(key, value));
 
-        List&lt;Node&gt; nodes = _routingStrategy.routeRequest(keyBytes);
+        List&lt;Node&gt; nodes = routingStrategy.routeRequest(keyBytes);
         for(Node node: nodes) {
             ByteArrayOutputStream versionedValue = new ByteArrayOutputStream();
             DataOutputStream valueDin = new DataOutputStream(versionedValue);
@@ -82,17 +82,17 @@ public abstract class ReadOnlyBatchIndexMapper&lt;K, V&gt; implements
             }
 
             // get Cluster and Store details
-            _cluster = ContribUtils.getVoldemortClusterDetails(clusterFilePath);
-            _storeDef = ContribUtils.getVoldemortStoreDetails(storeFilePath,
-                                                              conf.get(&quot;voldemort.store.name&quot;));
+            cluster = ContribUtils.getVoldemortClusterDetails(clusterFilePath);
+            storeDef = ContribUtils.getVoldemortStoreDetails(storeFilePath,
+                                                             conf.get(&quot;voldemort.store.name&quot;));
 
-            _keySerializer = (Serializer&lt;Object&gt;) new DefaultSerializerFactory().getSerializer(_storeDef.getKeySerializer());
-            _valueSerializer = (Serializer&lt;Object&gt;) new DefaultSerializerFactory().getSerializer(_storeDef.getValueSerializer());
+            keySerializer = (Serializer&lt;Object&gt;) new DefaultSerializerFactory().getSerializer(storeDef.getKeySerializer());
+            valueSerializer = (Serializer&lt;Object&gt;) new DefaultSerializerFactory().getSerializer(storeDef.getValueSerializer());
 
-            _routingStrategy = new ConsistentRoutingStrategy(_cluster.getNodes(),
-                                                             _storeDef.getReplicationFactor());
+            routingStrategy = new ConsistentRoutingStrategy(cluster.getNodes(),
+                                                            storeDef.getReplicationFactor());
 
-            if(_routingStrategy == null) {
+            if(routingStrategy == null) {
                 throw new RuntimeException(&quot;Failed to create routing strategy&quot;);
             }
         } catch(Exception e) {</diff>
      <filename>contrib/batch-indexer/src/java/voldemort/contrib/batchindexer/ReadOnlyBatchIndexMapper.java</filename>
    </modified>
    <modified>
      <diff>@@ -20,14 +20,14 @@ import voldemort.utils.ByteUtils;
 
 public class ReadOnlyBatchIndexReducer implements Reducer&lt;BytesWritable, BytesWritable, Text, Text&gt; {
 
-    private DataOutputStream _indexFileStream = null;
-    private DataOutputStream _valueFileStream = null;
+    private DataOutputStream indexFileStream = null;
+    private DataOutputStream valueFileStream = null;
 
-    private long _position = 0;
+    private long position = 0;
 
-    private JobConf _conf = null;
-    private String _taskId = null;
-    private int _nodeId = -1;
+    private JobConf conf = null;
+    private String taskId = null;
+    private int nodeId = -1;
 
     Path taskIndexFileName;
     Path taskValueFileName;
@@ -46,21 +46,21 @@ public class ReadOnlyBatchIndexReducer implements Reducer&lt;BytesWritable, BytesWr
             BytesWritable value = values.next();
             byte[] valBytes = ByteUtils.copy(value.get(), 0, value.getSize());
 
-            if(_nodeId == -1) {
+            if(nodeId == -1) {
                 DataInputStream buffer = new DataInputStream(new ByteArrayInputStream(valBytes));
-                _nodeId = buffer.readInt();
+                nodeId = buffer.readInt();
             }
             // strip first 4 bytes as node_id
             byte[] value1 = ByteUtils.copy(valBytes, 4, valBytes.length);
 
             // Write Index Key/ position
-            _indexFileStream.write(keyBytes);
-            _indexFileStream.writeLong(_position);
-            _valueFileStream.writeInt(value1.length);
-            _valueFileStream.write(value1);
-            _position += value1.length + 4;
+            indexFileStream.write(keyBytes);
+            indexFileStream.writeLong(position);
+            valueFileStream.writeInt(value1.length);
+            valueFileStream.write(value1);
+            position += value1.length + 4;
 
-            if(_position &lt; 0) {
+            if(position &lt; 0) {
                 throw new RuntimeException(&quot;Position bigger than Integer size, split input files.&quot;);
             }
         }
@@ -69,20 +69,20 @@ public class ReadOnlyBatchIndexReducer implements Reducer&lt;BytesWritable, BytesWr
 
     public void configure(JobConf job) {
         try {
-            _position = 0;
-            _conf = job;
+            position = 0;
+            conf = job;
 
-            _taskId = job.get(&quot;mapred.task.id&quot;);
+            taskId = job.get(&quot;mapred.task.id&quot;);
 
-            taskIndexFileName = new Path(FileOutputFormat.getOutputPath(_conf),
-                                         _conf.get(&quot;voldemort.index.filename&quot;) + &quot;_&quot; + _taskId);
-            taskValueFileName = new Path(FileOutputFormat.getOutputPath(_conf),
-                                         _conf.get(&quot;voldemort.data.filename&quot;) + &quot;_&quot; + _taskId);
+            taskIndexFileName = new Path(FileOutputFormat.getOutputPath(conf),
+                                         conf.get(&quot;voldemort.index.filename&quot;) + &quot;_&quot; + taskId);
+            taskValueFileName = new Path(FileOutputFormat.getOutputPath(conf),
+                                         conf.get(&quot;voldemort.data.filename&quot;) + &quot;_&quot; + taskId);
 
             FileSystem fs = taskIndexFileName.getFileSystem(job);
 
-            _indexFileStream = fs.create(taskIndexFileName, (short) 1);
-            _valueFileStream = fs.create(taskValueFileName, (short) 1);
+            indexFileStream = fs.create(taskIndexFileName, (short) 1);
+            valueFileStream = fs.create(taskValueFileName, (short) 1);
         } catch(IOException e) {
             throw new RuntimeException(&quot;Failed to open Input/OutputStream&quot;, e);
         }
@@ -90,13 +90,13 @@ public class ReadOnlyBatchIndexReducer implements Reducer&lt;BytesWritable, BytesWr
 
     public void close() throws IOException {
 
-        _indexFileStream.close();
-        _valueFileStream.close();
+        indexFileStream.close();
+        valueFileStream.close();
 
-        Path hdfsIndexFile = new Path(FileOutputFormat.getOutputPath(_conf), _nodeId + &quot;.index&quot;);
-        Path hdfsValueFile = new Path(FileOutputFormat.getOutputPath(_conf), _nodeId + &quot;.data&quot;);
+        Path hdfsIndexFile = new Path(FileOutputFormat.getOutputPath(conf), nodeId + &quot;.index&quot;);
+        Path hdfsValueFile = new Path(FileOutputFormat.getOutputPath(conf), nodeId + &quot;.data&quot;);
 
-        FileSystem fs = hdfsIndexFile.getFileSystem(_conf);
+        FileSystem fs = hdfsIndexFile.getFileSystem(conf);
         fs.rename(taskIndexFileName, hdfsIndexFile);
         fs.rename(taskValueFileName, hdfsValueFile);
     }</diff>
      <filename>contrib/batch-indexer/src/java/voldemort/contrib/batchindexer/ReadOnlyBatchIndexReducer.java</filename>
    </modified>
    <modified>
      <diff>@@ -74,7 +74,7 @@ public class BdbBuildPerformanceTest {
 
                     reader.next(key, value);
                     store.put(new ByteArray(ByteUtils.copy(key.get(), 0, key.getSize())),
-                              Versioned.of(ByteUtils.copy(value.get(), 0, value.getSize())));
+                              Versioned.value(ByteUtils.copy(value.get(), 0, value.getSize())));
                 } catch(ObsoleteVersionException e) {
                     obsoletes.incrementAndGet();
                 }</diff>
      <filename>contrib/batch-indexer/test/voldemort/contrib/batchindexer/performance/BdbBuildPerformanceTest.java</filename>
    </modified>
    <modified>
      <diff>@@ -74,7 +74,7 @@ public class MysqlBuildPerformanceTest {
 
                     reader.next(key, value);
                     store.put(new ByteArray(ByteUtils.copy(key.get(), 0, key.getSize())),
-                              Versioned.of(ByteUtils.copy(value.get(), 0, value.getSize())));
+                              Versioned.value(ByteUtils.copy(value.get(), 0, value.getSize())));
                 } catch(ObsoleteVersionException e) {
                     obsoletes.incrementAndGet();
                 }</diff>
      <filename>contrib/batch-indexer/test/voldemort/contrib/batchindexer/performance/MysqlBuildPerformanceTest.java</filename>
    </modified>
    <modified>
      <diff>@@ -103,8 +103,8 @@ public class ReadOnlyHadoopSwapperTest extends TestCase {
 
     public void testswap() throws Throwable {
         // assert that read-only store is working
-        Store&lt;ByteArray, byte[]&gt; store1 = server1.getStoreMap().get(storeName);
-        Store&lt;ByteArray, byte[]&gt; store2 = server2.getStoreMap().get(storeName);
+        Store&lt;ByteArray, byte[]&gt; store1 = server1.getStoreRepository().getStorageEngine(storeName);
+        Store&lt;ByteArray, byte[]&gt; store2 = server2.getStoreRepository().getStorageEngine(storeName);
 
         SerializerDefinition serDef = new SerializerDefinition(&quot;json&quot;, &quot;'string'&quot;);
         Serializer&lt;Object&gt; serializer = StoreUtils.unsafeGetSerializer(new DefaultSerializerFactory(),</diff>
      <filename>contrib/batch-indexer/test/voldemort/contrib/batchswapper/ReadOnlyHadoopSwapperTest.java</filename>
    </modified>
    <modified>
      <diff>@@ -97,8 +97,8 @@ public class ReadOnlySimpleSwapperTest extends TestCase {
 
     public void testswap() throws Throwable {
         // assert that read-only store is working
-        Store&lt;ByteArray, byte[]&gt; store1 = server1.getStoreMap().get(storeName);
-        Store&lt;ByteArray, byte[]&gt; store2 = server2.getStoreMap().get(storeName);
+        Store&lt;ByteArray, byte[]&gt; store1 = server1.getStoreRepository().getStorageEngine(storeName);
+        Store&lt;ByteArray, byte[]&gt; store2 = server2.getStoreRepository().getStorageEngine(storeName);
 
         SerializerDefinition serDef = new SerializerDefinition(&quot;json&quot;, &quot;'string'&quot;);
         Serializer&lt;Object&gt; serializer = StoreUtils.unsafeGetSerializer(new DefaultSerializerFactory(),</diff>
      <filename>contrib/batch-indexer/test/voldemort/contrib/batchswapper/ReadOnlySimpleSwapperTest.java</filename>
    </modified>
    <modified>
      <diff>@@ -1,12 +1,12 @@
 /*
  * Copyright 2008-2009 LinkedIn, Inc
- *
+ * 
  * Licensed under the Apache License, Version 2.0 (the &quot;License&quot;); 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 &quot;AS IS&quot; BASIS, WITHOUT
  * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
@@ -14,62 +14,39 @@
  * the License.
  */
 
+import java.util.Random;
+
+import org.mongodb.driver.ts.Doc;
+
+import voldemort.client.ClientConfig;
 import voldemort.client.SocketStoreClientFactory;
 import voldemort.client.StoreClient;
 import voldemort.client.StoreClientFactory;
-import voldemort.client.DaemonThreadFactory;
-import voldemort.client.AbstractStoreClientFactory;
-import voldemort.versioning.Versioned;
 import voldemort.serialization.mongodb.MongoDBSerializationFactory;
-import org.mongodb.driver.ts.Doc;
-
-import java.util.concurrent.ThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.Random;
+import voldemort.versioning.Versioned;
 
 public class MongoDBClient {
 
-    protected final StoreClientFactory _factory;
-    Random _rand = new Random();
-
-
-    public MongoDBClient(String bootstrapURL) {
-        // In real life this stuff would get wired in
-        int numThreads = 10;
-        int maxQueuedRequests = 10;
-        int maxConnectionsPerNode = 10;
-        int maxTotalConnections = 100;
-
-        _factory = new SocketStoreClientFactory(
-                new ThreadPoolExecutor(numThreads,
-                                    numThreads,
-                                    10000L,
-                                    TimeUnit.MILLISECONDS,
-                                    new LinkedBlockingQueue&lt;Runnable&gt;(maxQueuedRequests),
-                                    new DaemonThreadFactory(&quot;voldemort-client-thread-&quot;),
-                                    new ThreadPoolExecutor.CallerRunsPolicy()),
-                                maxConnectionsPerNode,
-                                maxTotalConnections,
-                                5000,
-                                AbstractStoreClientFactory.DEFAULT_ROUTING_TIMEOUT_MS,
-                                AbstractStoreClientFactory.DEFAULT_NODE_BANNAGE_MS,
-                                new MongoDBSerializationFactory(),
-                                bootstrapURL);
-    }
+    protected final StoreClientFactory factory;
+    Random rand = new Random();
 
+    public MongoDBClient(String bootstrapUrl) {
+
+        factory = new SocketStoreClientFactory(new ClientConfig().setSerializerFactory(new MongoDBSerializationFactory())
+                                                                 .setBootstrapUrls(bootstrapUrl));
+    }
 
     public long multiWriteLarge(int count, String keyRoot) {
 
-        StoreClient&lt;String , Doc&gt; client = _factory.getStoreClient(&quot;test&quot;);
+        StoreClient&lt;String, Doc&gt; client = factory.getStoreClient(&quot;test&quot;);
 
         long start = System.currentTimeMillis();
 
-        for (int i=0; i &lt; count; i++) {
+        for(int i = 0; i &lt; count; i++) {
             Doc d = makeLargeDoc();
             d.add(&quot;x&quot;, 1);
 
-            Versioned&lt;Doc&gt; v  = new Versioned&lt;Doc&gt;(d);
+            Versioned&lt;Doc&gt; v = new Versioned&lt;Doc&gt;(d);
 
             client.put(keyRoot + i, v);
         }
@@ -79,17 +56,17 @@ public class MongoDBClient {
         return end - start;
     }
 
-    public long multiWrite(int count, String keyRoot){
+    public long multiWrite(int count, String keyRoot) {
 
-        StoreClient&lt;String , Doc&gt; client = _factory.getStoreClient(&quot;test&quot;);
+        StoreClient&lt;String, Doc&gt; client = factory.getStoreClient(&quot;test&quot;);
 
         long start = System.currentTimeMillis();
 
-        for (int i=0; i &lt; count; i++) {
+        for(int i = 0; i &lt; count; i++) {
             Doc d = new Doc(&quot;name&quot;, &quot;geir&quot;);
             d.add(&quot;x&quot;, 1);
 
-            Versioned&lt;Doc&gt; v  = new Versioned&lt;Doc&gt;(d);
+            Versioned&lt;Doc&gt; v = new Versioned&lt;Doc&gt;(d);
 
             client.put(keyRoot + i, v);
         }
@@ -99,18 +76,18 @@ public class MongoDBClient {
         return end - start;
     }
 
-    public long multiRead(int count, String keyRoot){
+    public long multiRead(int count, String keyRoot) {
 
-        StoreClient&lt;String , Doc&gt; client = _factory.getStoreClient(&quot;test&quot;);
+        StoreClient&lt;String, Doc&gt; client = factory.getStoreClient(&quot;test&quot;);
 
         long start = System.currentTimeMillis();
 
         int found = 0;
 
-        for (int i=0; i &lt; count; i++) {
-            Versioned&lt;Doc&gt; v  = client.get(keyRoot + i);
+        for(int i = 0; i &lt; count; i++) {
+            Versioned&lt;Doc&gt; v = client.get(keyRoot + i);
 
-            if (v != null) {
+            if(v != null) {
                 found++;
             }
         }
@@ -118,36 +95,36 @@ public class MongoDBClient {
         long end = System.currentTimeMillis();
 
         System.out.println(&quot;Found : &quot; + found);
-        
+
         return end - start;
     }
 
     public String getRandomKey(int n) {
 
-        if (n &lt;= 0) {
+        if(n &lt;= 0) {
             return &quot;&quot;;
         }
 
         StringBuilder sb = new StringBuilder();
 
-        for (int i=0; i &lt; n; i++) {
-            sb.append(_rand.nextInt(9));
+        for(int i = 0; i &lt; n; i++) {
+            sb.append(rand.nextInt(9));
         }
 
         return sb.toString();
     }
 
-    public void simple(){
+    public void simple() {
 
-        StoreClient&lt;String , Doc&gt; client = _factory.getStoreClient(&quot;test&quot;);
+        StoreClient&lt;String, Doc&gt; client = factory.getStoreClient(&quot;test&quot;);
 
         Versioned&lt;Doc&gt; v = client.get(&quot;key&quot;);
 
-        if (v == null) {
+        if(v == null) {
             Doc d = new Doc(&quot;name&quot;, &quot;geir&quot;);
             d.add(&quot;x&quot;, 1);
 
-            v  = new Versioned&lt;Doc&gt;(d);
+            v = new Versioned&lt;Doc&gt;(d);
         }
 
         // update the value
@@ -179,13 +156,13 @@ public class MongoDBClient {
         mt = new Doc();
 
         mt.add(&quot;counted_tags&quot;, 3450);
-        mt.add(&quot;no_of_js_attached&quot;,10);
+        mt.add(&quot;no_of_js_attached&quot;, 10);
         mt.add(&quot;no_of_images&quot;, 6);
 
         d.put(&quot;page_structure&quot;, mt);
 
         mt = new Doc();
-        for (int i = 0; i &lt; 10; i ++) {
+        for(int i = 0; i &lt; 10; i++) {
             mt.add(Integer.toString(i), &quot;woog&quot;);
         }
 
@@ -201,13 +178,14 @@ public class MongoDBClient {
         client.simple();
 
         String keyRoot = client.getRandomKey(15);
-        
+
         System.out.println(10000.0 / client.multiWrite(10000, keyRoot) * 1000 + &quot; writes per sec&quot;);
         System.out.println(10000.0 / client.multiRead(10000, keyRoot) * 1000 + &quot; reads per sec&quot;);
 
         keyRoot = client.getRandomKey(15);
 
-        System.out.println(10000.0 / client.multiWriteLarge(10000, keyRoot) * 1000 + &quot; large writes per sec&quot;);
+        System.out.println(10000.0 / client.multiWriteLarge(10000, keyRoot) * 1000
+                           + &quot; large writes per sec&quot;);
     }
 
 }</diff>
      <filename>contrib/mongodb/example/MongoDBClient.java</filename>
    </modified>
    <modified>
      <diff>@@ -96,23 +96,22 @@ public class MongoDBStorageEngine implements StorageEngine&lt;ByteArray, byte[]&gt; {
     public static final String VALUE = &quot;v&quot;;
     public static final String DB_NAME = &quot;voldemort&quot;;
 
-    protected Mongo _mongoDB;
-    protected DB _db;
-    protected DBCollection _coll;
-    protected final String _collectionName;
+    protected Mongo mongoDb;
+    protected DB db;
+    protected DBCollection coll;
+    protected final String collectionName;
 
     public MongoDBStorageEngine(String name) throws MongoDBException {
         logger.info(&quot;MongoDB Storage Engine : v0.1&quot;);
-        _collectionName = name;
+        collectionName = name;
         init();
     }
 
     protected final void init() throws MongoDBException {
-        _mongoDB = new Mongo(&quot;127.0.0.1&quot;, 27017);
-        _db = _mongoDB.getDB(DB_NAME);
-        _coll = _db.getCollection(_collectionName);
-
-        _coll.createIndex(new IndexInfo(&quot;k_1&quot;, KEY));
+        mongoDb = new Mongo(&quot;127.0.0.1&quot;, 27017);
+        db = mongoDb.getDB(DB_NAME);
+        coll = db.getCollection(collectionName);
+        coll.createIndex(new IndexInfo(&quot;k_1&quot;, KEY));
     }
 
     public ClosableIterator&lt;Pair&lt;ByteArray, Versioned&lt;byte[]&gt;&gt;&gt; entries() {
@@ -135,7 +134,7 @@ public class MongoDBStorageEngine implements StorageEngine&lt;ByteArray, byte[]&gt; {
         String strKey = new String(key.get());
         DBCursor cur = null;
         try {
-            cur = _coll.find(new MongoSelector(KEY, strKey));
+            cur = coll.find(new MongoSelector(KEY, strKey));
             for(Doc d: cur) {
 
                 BSONObject bo = new BSONObject(tls.getReadBuffer());
@@ -193,7 +192,7 @@ public class MongoDBStorageEngine implements StorageEngine&lt;ByteArray, byte[]&gt; {
              * and also delete them
              */
 
-            cur = _coll.find(new Doc(KEY, strKey));
+            cur = coll.find(new Doc(KEY, strKey));
 
             for(Doc d: cur) {
 
@@ -205,7 +204,7 @@ public class MongoDBStorageEngine implements StorageEngine&lt;ByteArray, byte[]&gt; {
                 if(occured == Occured.BEFORE) {
                     throw new ObsoleteVersionException(&quot;Key '&quot; + strKey + &quot; is obsolete.&quot;);
                 } else if(occured == Occured.AFTER) {
-                    _coll.remove(new MongoSelector(d));
+                    coll.remove(new MongoSelector(d));
                 }
 
                 // TODO - why not concurrent? need to understand better...
@@ -219,7 +218,7 @@ public class MongoDBStorageEngine implements StorageEngine&lt;ByteArray, byte[]&gt; {
             newData.put(VALUE, new BSONBytes(value.getValue()));
             newData.put(CLOCK, ((VectorClock) value.getVersion()).toBytes());
 
-            _coll.insert(newData);
+            coll.insert(newData);
         } catch(MongoDBIOException mioe) {
             try {
                 init();
@@ -241,14 +240,14 @@ public class MongoDBStorageEngine implements StorageEngine&lt;ByteArray, byte[]&gt; {
         boolean deleted = false;
         DBCursor cur = null;
         try {
-            cur = _coll.find(new Doc(KEY, strKey));
+            cur = coll.find(new Doc(KEY, strKey));
             for(Doc d: cur) {
                 VectorClock existingClock = new VectorClock(d.getBytes(CLOCK));
                 Occured occured = version.compare(existingClock);
 
                 // TODO - Q : why not concurrently?
                 if(occured == Occured.BEFORE) {
-                    _coll.remove(new MongoSelector(d));
+                    coll.remove(new MongoSelector(d));
                     deleted = true;
                 }
             }
@@ -269,13 +268,13 @@ public class MongoDBStorageEngine implements StorageEngine&lt;ByteArray, byte[]&gt; {
     }
 
     public String getName() {
-        return _coll.getName();
+        return coll.getName();
     }
 
     public void close() throws VoldemortException {
         try {
-            if(_db != null)
-                _db.close();
+            if(db != null)
+                db.close();
         } catch(Exception e) {
             throw new VoldemortException(e);
         }
@@ -287,7 +286,7 @@ public class MongoDBStorageEngine implements StorageEngine&lt;ByteArray, byte[]&gt; {
      */
     protected void clearStore() {
         try {
-            _coll.clear();
+            coll.clear();
         } catch(MongoDBException e) {
             logger.error(&quot;Error while clearing store.&quot;, e);
         }
@@ -334,30 +333,31 @@ public class MongoDBStorageEngine implements StorageEngine&lt;ByteArray, byte[]&gt; {
     public class MongoDBClosableIterator implements
             ClosableIterator&lt;Pair&lt;ByteArray, Versioned&lt;byte[]&gt;&gt;&gt; {
 
-        BSONObject _bo = new BSONObject();
-        protected DBCursor _cursor;
+        BSONObject bo = new BSONObject();
+        protected DBCursor cursor;
 
         public MongoDBClosableIterator() throws MongoDBException {
-            getTLS(); // TODO - will be a problem if someone hands this iterator
-            // across threads
-            _cursor = _coll.find();
+            // TODO - will be a problem if someone hands this iterator across
+            // threads
+            getTLS();
+            cursor = coll.find();
         }
 
         public void close() {
-            closeCursor(_cursor);
-            _cursor = null;
+            closeCursor(cursor);
+            cursor = null;
         }
 
         public boolean hasNext() {
-            return _cursor.hasMoreElements();
+            return cursor.hasMoreElements();
         }
 
         public Pair&lt;ByteArray, Versioned&lt;byte[]&gt;&gt; next() {
             try {
-                Doc d = _cursor.getNextObject();
-                _bo.serialize(d.getDoc(VALUE));
+                Doc d = cursor.getNextObject();
+                bo.serialize(d.getDoc(VALUE));
 
-                Versioned&lt;byte[]&gt; val = new Versioned&lt;byte[]&gt;(_bo.toArray(),
+                Versioned&lt;byte[]&gt; val = new Versioned&lt;byte[]&gt;(bo.toArray(),
                                                               new VectorClock(d.getBytes(CLOCK)));
 
                 return new Pair&lt;ByteArray, Versioned&lt;byte[]&gt;&gt;(new ByteArray(d.getString(KEY)</diff>
      <filename>contrib/mongodb/src/java/voldemort/store/mongodb/MongoDBStorageEngine.java</filename>
    </modified>
    <modified>
      <diff>@@ -44,7 +44,7 @@ import com.google.common.collect.ImmutableMap;
  */
 public class MongoDBStorageEngineTest extends AbstractStoreTest&lt;ByteArray, byte[]&gt; {
 
-    MongoDBDocSerializer _mds = new MongoDBDocSerializer();
+    MongoDBDocSerializer mds = new MongoDBDocSerializer();
 
     @Override
     protected boolean valuesEqual(byte[] t1, byte[] t2) {
@@ -63,7 +63,7 @@ public class MongoDBStorageEngineTest extends AbstractStoreTest&lt;ByteArray, byte[
     public List&lt;byte[]&gt; getValues(int numValues) {
         List&lt;byte[]&gt; list = new ArrayList&lt;byte[]&gt;();
         for(int i = 0; i &lt; numValues; i++) {
-            list.add(_mds.toBytes(new Doc(&quot;x&quot;, i)));
+            list.add(mds.toBytes(new Doc(&quot;x&quot;, i)));
         }
         return list;
 
@@ -110,7 +110,7 @@ public class MongoDBStorageEngineTest extends AbstractStoreTest&lt;ByteArray, byte[
         Map&lt;String, String&gt; vals = ImmutableMap.of(&quot;a&quot;, &quot;a&quot;, &quot;b&quot;, &quot;b&quot;, &quot;c&quot;, &quot;c&quot;, &quot;d&quot;, &quot;d&quot;, &quot;e&quot;, &quot;e&quot;);
         for(Map.Entry&lt;String, String&gt; entry: vals.entrySet()) {
             store.put(new ByteArray(entry.getKey().getBytes()),
-                      new Versioned&lt;byte[]&gt;(_mds.toBytes(new Doc(entry.getKey(), entry.getValue()))));
+                      new Versioned&lt;byte[]&gt;(mds.toBytes(new Doc(entry.getKey(), entry.getValue()))));
         }
 
         ClosableIterator&lt;Pair&lt;ByteArray, Versioned&lt;byte[]&gt;&gt;&gt; iter = store.entries();
@@ -121,7 +121,7 @@ public class MongoDBStorageEngineTest extends AbstractStoreTest&lt;ByteArray, byte[
             String key = new String(entry.getFirst().get());
             assertTrue(vals.containsKey(key));
 
-            assertEquals(vals.get(key), _mds.toObject(entry.getSecond().getValue()).getString(key));
+            assertEquals(vals.get(key), mds.toObject(entry.getSecond().getValue()).getString(key));
             count++;
         }
 
@@ -132,11 +132,11 @@ public class MongoDBStorageEngineTest extends AbstractStoreTest&lt;ByteArray, byte[
     public void testPruneOnWrite() {
         StorageEngine&lt;ByteArray, byte[]&gt; engine = getStore();
         Doc d = new Doc(&quot;x&quot;, 1);
-        Versioned&lt;byte[]&gt; v1 = new Versioned&lt;byte[]&gt;(_mds.toBytes(d.add(&quot;x&quot;, 1)),
+        Versioned&lt;byte[]&gt; v1 = new Versioned&lt;byte[]&gt;(mds.toBytes(d.add(&quot;x&quot;, 1)),
                                                      TestUtils.getClock(1));
-        Versioned&lt;byte[]&gt; v2 = new Versioned&lt;byte[]&gt;(_mds.toBytes(d.add(&quot;x&quot;, 2)),
+        Versioned&lt;byte[]&gt; v2 = new Versioned&lt;byte[]&gt;(mds.toBytes(d.add(&quot;x&quot;, 2)),
                                                      TestUtils.getClock(2));
-        Versioned&lt;byte[]&gt; v3 = new Versioned&lt;byte[]&gt;(_mds.toBytes(d.add(&quot;x&quot;, 3)),
+        Versioned&lt;byte[]&gt; v3 = new Versioned&lt;byte[]&gt;(mds.toBytes(d.add(&quot;x&quot;, 3)),
                                                      TestUtils.getClock(1, 2));
 
         ByteArray key = new ByteArray(&quot;foo&quot;.getBytes());</diff>
      <filename>contrib/mongodb/test/voldemort/store/mongodb/MongoDBStorageEngineTest.java</filename>
    </modified>
    <modified>
      <diff>@@ -16,6 +16,7 @@
 
 package voldemort.examples;
 
+import voldemort.client.ClientConfig;
 import voldemort.client.SocketStoreClientFactory;
 import voldemort.client.StoreClient;
 import voldemort.client.StoreClientFactory;
@@ -26,17 +27,8 @@ public class ClientExample {
     public static void main(String[] args) {
 
         // In real life this stuff would get wired in
-        int numThreads = 10;
-        int maxQueuedRequests = 10;
-        int maxConnectionsPerNode = 10;
-        int maxTotalConnections = 100;
         String bootstrapUrl = &quot;tcp://localhost:6666&quot;;
-        StoreClientFactory factory = new SocketStoreClientFactory(numThreads,
-                                                                  numThreads,
-                                                                  maxQueuedRequests,
-                                                                  maxConnectionsPerNode,
-                                                                  maxTotalConnections,
-                                                                  bootstrapUrl);
+        StoreClientFactory factory = new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrl));
 
         StoreClient&lt;String, String&gt; client = factory.getStoreClient(&quot;my_store_name&quot;);
 </diff>
      <filename>example/java/voldemort/examples/ClientExample.java</filename>
    </modified>
    <modified>
      <diff>@@ -22,17 +22,19 @@ import java.io.IOException;
 import java.io.InputStreamReader;
 import java.io.StringReader;
 import java.text.DateFormat;
+import java.util.ArrayList;
 import java.util.Date;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
-import java.util.concurrent.Executors;
 
+import voldemort.client.ClientConfig;
 import voldemort.client.DefaultStoreClient;
 import voldemort.client.SocketStoreClientFactory;
 import voldemort.client.StoreClientFactory;
 import voldemort.cluster.Node;
 import voldemort.serialization.SerializationException;
+import voldemort.serialization.json.EndOfFileException;
 import voldemort.serialization.json.JsonReader;
 import voldemort.utils.Utils;
 import voldemort.versioning.Versioned;
@@ -63,13 +65,7 @@ public class VoldemortClientShell {
             Utils.croak(&quot;Failure to open input stream: &quot; + e.getMessage());
         }
 
-        StoreClientFactory factory = new SocketStoreClientFactory(Executors.newFixedThreadPool(5),
-                                                                  3,
-                                                                  10,
-                                                                  2000,
-                                                                  2000,
-                                                                  2000,
-                                                                  bootstrapUrl);
+        StoreClientFactory factory = new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrl));
         DefaultStoreClient&lt;Object, Object&gt; client = null;
         try {
             client = (DefaultStoreClient&lt;Object, Object&gt;) factory.getStoreClient(storeName);
@@ -87,6 +83,21 @@ public class VoldemortClientShell {
                     JsonReader jsonReader = new JsonReader(new StringReader(line.substring(&quot;put&quot;.length())));
                     client.put(tightenNumericTypes(jsonReader.read()),
                                tightenNumericTypes(jsonReader.read()));
+                } else if(line.toLowerCase().startsWith(&quot;getall&quot;)) {
+                    JsonReader jsonReader = new JsonReader(new StringReader(line.substring(&quot;getall&quot;.length())));
+                    List&lt;Object&gt; keys = new ArrayList&lt;Object&gt;();
+                    try {
+                        while(true)
+                            keys.add(jsonReader.read());
+                    } catch(EndOfFileException e) {
+                        // this is okay, just means we are done reading
+                    }
+                    Map&lt;Object, Versioned&lt;Object&gt;&gt; vals = client.getAll(keys);
+                    for(Map.Entry&lt;Object, Versioned&lt;Object&gt;&gt; entry: vals.entrySet()) {
+                        System.out.print(entry.getKey());
+                        System.out.print(&quot; =&gt; &quot;);
+                        printVersioned(entry.getValue());
+                    }
                 } else if(line.toLowerCase().startsWith(&quot;get&quot;)) {
                     JsonReader jsonReader = new JsonReader(new StringReader(line.substring(&quot;get&quot;.length())));
                     printVersioned(client.get(tightenNumericTypes(jsonReader.read())));
@@ -112,6 +123,8 @@ public class VoldemortClientShell {
                 } else {
                     System.err.println(&quot;Invalid command.&quot;);
                 }
+            } catch(EndOfFileException e) {
+                System.err.println(&quot;Expected additional token.&quot;);
             } catch(SerializationException e) {
                 System.err.print(&quot;Error serializing values: &quot;);
                 e.printStackTrace();</diff>
      <filename>src/java/voldemort/VoldemortClientShell.java</filename>
    </modified>
    <modified>
      <diff>@@ -17,14 +17,20 @@
 package voldemort.client;
 
 import java.io.StringReader;
+import java.lang.management.ManagementFactory;
 import java.net.URI;
 import java.net.URISyntaxException;
 import java.util.List;
 import java.util.Map;
 import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import javax.management.MBeanServer;
+import javax.management.ObjectName;
 
 import org.apache.log4j.Logger;
 
+import voldemort.client.protocol.RequestFormatType;
 import voldemort.cluster.Cluster;
 import voldemort.cluster.Node;
 import voldemort.routing.ConsistentRoutingStrategy;
@@ -38,8 +44,10 @@ import voldemort.store.logging.LoggingStore;
 import voldemort.store.metadata.MetadataStore;
 import voldemort.store.routed.RoutedStore;
 import voldemort.store.serialized.SerializingStore;
+import voldemort.store.stats.StatTrackingStore;
 import voldemort.store.versioned.InconsistencyResolvingStore;
 import voldemort.utils.ByteArray;
+import voldemort.utils.JmxUtils;
 import voldemort.utils.SystemTime;
 import voldemort.versioning.ChainedResolver;
 import voldemort.versioning.InconsistencyResolver;
@@ -68,23 +76,27 @@ public abstract class AbstractStoreClientFactory implements StoreClientFactory {
     private static final Logger logger = Logger.getLogger(AbstractStoreClientFactory.class);
 
     private final URI[] bootstrapUrls;
-    private final long routingTimeoutMs;
-    private final long nodeBannageMs;
+    private final int routingTimeoutMs;
+    private final int nodeBannageMs;
     private final ExecutorService threadPool;
     private final SerializerFactory serializerFactory;
-    private final boolean enableVerboseLogging;
-
-    public AbstractStoreClientFactory(ExecutorService threadPool,
-                                      SerializerFactory serializerFactory,
-                                      int routingTimeoutMs,
-                                      int nodeBannageMs,
-                                      String... bootstrapUrls) {
-        this.threadPool = threadPool;
-        this.serializerFactory = serializerFactory;
-        this.bootstrapUrls = validateUrls(bootstrapUrls);
-        this.routingTimeoutMs = routingTimeoutMs;
-        this.nodeBannageMs = nodeBannageMs;
-        this.enableVerboseLogging = true;
+    private final boolean isJmxEnabled;
+    private final MBeanServer mbeanServer;
+
+    public AbstractStoreClientFactory(ClientConfig config) {
+        this.threadPool = new ClientThreadPool(config.getMaxThreads(),
+                                               config.getThreadIdleTime(TimeUnit.MILLISECONDS),
+                                               config.getMaxQueuedRequests());
+        this.serializerFactory = config.getSerializerFactory();
+        this.bootstrapUrls = validateUrls(config.getBootstrapUrls());
+        this.routingTimeoutMs = config.getRoutingTimeout(TimeUnit.MILLISECONDS);
+        this.nodeBannageMs = config.getNodeBannagePeriod(TimeUnit.MILLISECONDS);
+        this.isJmxEnabled = config.isJmxEnabled();
+        if(isJmxEnabled)
+            this.mbeanServer = ManagementFactory.getPlatformMBeanServer();
+        else
+            this.mbeanServer = null;
+        registerJmx(JmxUtils.createObjectName(threadPool.getClass()), threadPool);
     }
 
     public &lt;K, V&gt; StoreClient&lt;K, V&gt; getStoreClient(String storeName) {
@@ -121,20 +133,18 @@ public abstract class AbstractStoreClientFactory implements StoreClientFactory {
         for(Node node: cluster.getNodes()) {
             Store&lt;ByteArray, byte[]&gt; store = getStore(storeDef.getName(),
                                                       node.getHost(),
-                                                      getPort(node));
-            if(enableVerboseLogging)
-                store = new LoggingStore&lt;ByteArray, byte[]&gt;(store);
+                                                      getPort(node),
+                                                      RequestFormatType.VOLDEMORT);
+            store = new LoggingStore(store);
             clientMapping.put(node.getId(), store);
         }
 
         Store&lt;ByteArray, byte[]&gt; store = new RoutedStore(storeName,
                                                          clientMapping,
                                                          routingStrategy,
-                                                         storeDef.getPreferredReads() == null ? storeDef.getRequiredReads()
-                                                                                             : storeDef.getPreferredReads(),
+                                                         storeDef.getPreferredReads(),
                                                          storeDef.getRequiredReads(),
-                                                         storeDef.getPreferredWrites() == null ? storeDef.getRequiredWrites()
-                                                                                              : storeDef.getPreferredWrites(),
+                                                         storeDef.getPreferredWrites(),
                                                          storeDef.getRequiredWrites(),
                                                          true,
                                                          threadPool,
@@ -142,20 +152,26 @@ public abstract class AbstractStoreClientFactory implements StoreClientFactory {
                                                          nodeBannageMs,
                                                          SystemTime.INSTANCE);
 
+        if(isJmxEnabled) {
+            store = new StatTrackingStore(store);
+            registerJmx(JmxUtils.createObjectName(JmxUtils.getPackageName(store.getClass()),
+                                                  store.getName()), store);
+        }
+
         Serializer&lt;K&gt; keySerializer = (Serializer&lt;K&gt;) serializerFactory.getSerializer(storeDef.getKeySerializer());
         Serializer&lt;V&gt; valueSerializer = (Serializer&lt;V&gt;) serializerFactory.getSerializer(storeDef.getValueSerializer());
-        Store&lt;K, V&gt; serializingStore = new SerializingStore&lt;K, V&gt;(store,
-                                                                  keySerializer,
-                                                                  valueSerializer);
+        Store&lt;K, V&gt; serializedStore = new SerializingStore&lt;K, V&gt;(store,
+                                                                 keySerializer,
+                                                                 valueSerializer);
 
         // Add inconsistency resolving decorator, using their inconsistency
         // resolver (if they gave us one)
         InconsistencyResolver&lt;Versioned&lt;V&gt;&gt; secondaryResolver = resolver == null ? new TimeBasedInconsistencyResolver()
                                                                                 : resolver;
-        Store&lt;K, V&gt; resolvingStore = new InconsistencyResolvingStore&lt;K, V&gt;(serializingStore,
-                                                                           new ChainedResolver&lt;Versioned&lt;V&gt;&gt;(new VectorClockInconsistencyResolver(),
-                                                                                                             secondaryResolver));
-        return resolvingStore;
+        serializedStore = new InconsistencyResolvingStore&lt;K, V&gt;(serializedStore,
+                                                                new ChainedResolver&lt;Versioned&lt;V&gt;&gt;(new VectorClockInconsistencyResolver(),
+                                                                                                  secondaryResolver));
+        return serializedStore;
     }
 
     private String bootstrapMetadata(String key, URI[] urls) {
@@ -163,7 +179,8 @@ public abstract class AbstractStoreClientFactory implements StoreClientFactory {
             try {
                 Store&lt;ByteArray, byte[]&gt; remoteStore = getStore(MetadataStore.METADATA_STORE_NAME,
                                                                 url.getHost(),
-                                                                url.getPort());
+                                                                url.getPort(),
+                                                                RequestFormatType.VOLDEMORT);
                 Store&lt;String, String&gt; store = new SerializingStore&lt;String, String&gt;(remoteStore,
                                                                                    new StringSerializer(&quot;UTF-8&quot;),
                                                                                    new StringSerializer(&quot;UTF-8&quot;));
@@ -206,7 +223,10 @@ public abstract class AbstractStoreClientFactory implements StoreClientFactory {
         return uris;
     }
 
-    protected abstract Store&lt;ByteArray, byte[]&gt; getStore(String storeName, String host, int port);
+    protected abstract Store&lt;ByteArray, byte[]&gt; getStore(String storeName,
+                                                         String host,
+                                                         int port,
+                                                         RequestFormatType type);
 
     protected abstract int getPort(Node node);
 
@@ -228,8 +248,12 @@ public abstract class AbstractStoreClientFactory implements StoreClientFactory {
         return serializerFactory;
     }
 
-    public boolean isEnableVerboseLogging() {
-        return enableVerboseLogging;
+    protected void registerJmx(ObjectName name, Object object) {
+        if(this.isJmxEnabled) {
+            if(mbeanServer.isRegistered(name))
+                JmxUtils.unregisterMbean(mbeanServer, name);
+            JmxUtils.registerMbean(mbeanServer, JmxUtils.createModelMBean(object), name);
+        }
     }
 
 }</diff>
      <filename>src/java/voldemort/client/AbstractStoreClientFactory.java</filename>
    </modified>
    <modified>
      <diff>@@ -17,7 +17,7 @@
 package voldemort.client;
 
 import java.net.URI;
-import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
 
 import org.apache.commons.httpclient.DefaultHttpMethodRetryHandler;
 import org.apache.commons.httpclient.HostConfiguration;
@@ -29,9 +29,9 @@ import org.apache.commons.httpclient.params.HttpClientParams;
 import org.apache.commons.httpclient.params.HttpConnectionManagerParams;
 import org.apache.commons.httpclient.params.HttpMethodParams;
 
+import voldemort.client.protocol.RequestFormatFactory;
+import voldemort.client.protocol.RequestFormatType;
 import voldemort.cluster.Node;
-import voldemort.serialization.DefaultSerializerFactory;
-import voldemort.serialization.SerializerFactory;
 import voldemort.store.Store;
 import voldemort.store.http.HttpStore;
 import voldemort.utils.ByteArray;
@@ -47,94 +47,49 @@ public class HttpStoreClientFactory extends AbstractStoreClientFactory {
 
     public static final String URL_SCHEME = &quot;http&quot;;
 
-    private static final int DEFAULT_CONNECTION_TIMEOUT = 5000;
-    private static final int DEFAULT_SO_TIMEOUT = 5000;
-    private static final int DEFAULT_CONNECTION_MANAGER_TIMEOUT = 5000;
-    private static final int DEFAULT_MAX_CONNECTIONS = 40;
-    private static final int DEFAULT_MAX_HOST_CONNECTIONS = 40;
-    private static final int DEFAULT_NUM_RETRIES = 0;
     private static final String VOLDEMORT_USER_AGENT = &quot;vldmrt/0.01&quot;;
 
     private final HttpClient httpClient;
     private final MultiThreadedHttpConnectionManager connectionManager;
+    private final RequestFormatFactory requestFormatFactory;
+    private final boolean reroute;
 
-    public HttpStoreClientFactory(int numThreads, String... bootstrapUrls) {
-        this(numThreads,
-             DEFAULT_CONNECTION_MANAGER_TIMEOUT,
-             DEFAULT_SO_TIMEOUT,
-             DEFAULT_NUM_RETRIES,
-             DEFAULT_CONNECTION_TIMEOUT,
-             DEFAULT_ROUTING_TIMEOUT_MS,
-             AbstractStoreClientFactory.DEFAULT_NODE_BANNAGE_MS,
-             DEFAULT_MAX_CONNECTIONS,
-             DEFAULT_MAX_HOST_CONNECTIONS,
-             bootstrapUrls);
-    }
-
-    public HttpStoreClientFactory(int numThreads,
-                                  int connectionManagerTimeoutMs,
-                                  int socketSoTimeoutMs,
-                                  int numRetries,
-                                  int connectionTimeoutMs,
-                                  int routingTimeoutMs,
-                                  int nodeBannageMs,
-                                  int maxConnections,
-                                  int maxSingleHostConnections,
-                                  String... bootstrapUrls) {
-        this(numThreads,
-             connectionManagerTimeoutMs,
-             socketSoTimeoutMs,
-             numRetries,
-             connectionTimeoutMs,
-             routingTimeoutMs,
-             nodeBannageMs,
-             maxConnections,
-             maxSingleHostConnections,
-             new DefaultSerializerFactory(),
-             bootstrapUrls);
-    }
-
-    public HttpStoreClientFactory(int numThreads,
-                                  int connectionManagerTimeoutMs,
-                                  int socketSoTimeoutMs,
-                                  int numRetries,
-                                  int connectionTimeoutMs,
-                                  int routingTimeoutMs,
-                                  int nodeBannageMs,
-                                  int maxConnections,
-                                  int maxSingleHostConnections,
-                                  SerializerFactory serializerFactory,
-                                  String... bootstrapUrls) {
-        // TODO: Customize this threadpool??
-        super(Executors.newFixedThreadPool(numThreads),
-              serializerFactory,
-              routingTimeoutMs,
-              nodeBannageMs,
-              bootstrapUrls);
+    public HttpStoreClientFactory(ClientConfig config) {
+        super(config);
         HostConfiguration hostConfig = new HostConfiguration();
         hostConfig.getParams().setParameter(&quot;http.protocol.version&quot;, HttpVersion.HTTP_1_1);
         this.connectionManager = new MultiThreadedHttpConnectionManager();
         this.httpClient = new HttpClient(connectionManager);
         this.httpClient.setHostConfiguration(hostConfig);
         HttpClientParams clientParams = this.httpClient.getParams();
-        clientParams.setConnectionManagerTimeout(connectionManagerTimeoutMs);
-        clientParams.setSoTimeout(socketSoTimeoutMs);
+        clientParams.setConnectionManagerTimeout(config.getConnectionTimeout(TimeUnit.MILLISECONDS));
+        clientParams.setSoTimeout(config.getSocketTimeout(TimeUnit.MILLISECONDS));
         clientParams.setParameter(HttpMethodParams.RETRY_HANDLER,
-                                  new DefaultHttpMethodRetryHandler(numRetries, false));
+                                  new DefaultHttpMethodRetryHandler(0, false));
         clientParams.setCookiePolicy(CookiePolicy.IGNORE_COOKIES);
         clientParams.setParameter(&quot;http.useragent&quot;, VOLDEMORT_USER_AGENT);
         HttpConnectionManagerParams managerParams = this.httpClient.getHttpConnectionManager()
                                                                    .getParams();
-        managerParams.setConnectionTimeout(connectionTimeoutMs);
-        managerParams.setMaxTotalConnections(maxConnections);
+        managerParams.setConnectionTimeout(config.getConnectionTimeout(TimeUnit.MILLISECONDS));
+        managerParams.setMaxTotalConnections(config.getMaxTotalConnections());
         managerParams.setStaleCheckingEnabled(false);
         managerParams.setMaxConnectionsPerHost(httpClient.getHostConfiguration(),
-                                               maxSingleHostConnections);
+                                               config.getMaxConnectionsPerNode());
+        this.reroute = config.getRoutingTier().equals(RoutingTier.SERVER);
+        this.requestFormatFactory = new RequestFormatFactory();
     }
 
     @Override
-    protected Store&lt;ByteArray, byte[]&gt; getStore(String name, String host, int port) {
-        return new HttpStore(name, host, port, httpClient);
+    protected Store&lt;ByteArray, byte[]&gt; getStore(String name,
+                                                String host,
+                                                int port,
+                                                RequestFormatType type) {
+        return new HttpStore(name,
+                             host,
+                             port,
+                             httpClient,
+                             requestFormatFactory.getRequestFormat(type),
+                             reroute);
     }
 
     @Override</diff>
      <filename>src/java/voldemort/client/HttpStoreClientFactory.java</filename>
    </modified>
    <modified>
      <diff>@@ -17,18 +17,15 @@
 package voldemort.client;
 
 import java.net.URI;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ThreadPoolExecutor;
 import java.util.concurrent.TimeUnit;
 
+import voldemort.client.protocol.RequestFormatType;
 import voldemort.cluster.Node;
-import voldemort.serialization.DefaultSerializerFactory;
-import voldemort.serialization.SerializerFactory;
 import voldemort.store.Store;
 import voldemort.store.socket.SocketPool;
 import voldemort.store.socket.SocketStore;
 import voldemort.utils.ByteArray;
+import voldemort.utils.JmxUtils;
 import voldemort.utils.Utils;
 
 /**
@@ -43,141 +40,32 @@ import voldemort.utils.Utils;
 public class SocketStoreClientFactory extends AbstractStoreClientFactory {
 
     public static final String URL_SCHEME = &quot;tcp&quot;;
-    public static final int DEFAULT_SOCKET_TIMEOUT_MS = 5000;
-    public static final int DEFAULT_NUM_THREADS = 5;
-    public static final int DEFAULT_MAX_QUEUED_REQUESTS = 1000;
-    public static final int DEFAULT_MAX_CONNECTIONS_PER_NODE = 10;
-    public static final int DEFAULT_MAX_CONNECTIONS = 50;
-    public static final int DEFAULT_SOCKET_BUFFER_SIZE = 32 * 1024;
 
-    private SocketPool socketPool;
-
-    public SocketStoreClientFactory(String bootstrapUrl) {
-        this(DEFAULT_NUM_THREADS,
-             DEFAULT_NUM_THREADS,
-             DEFAULT_MAX_QUEUED_REQUESTS,
-             DEFAULT_MAX_CONNECTIONS_PER_NODE,
-             DEFAULT_MAX_CONNECTIONS,
-             bootstrapUrl);
-    }
-
-    public SocketStoreClientFactory(int coreThreads,
-                                    int maxThreads,
-                                    int maxQueuedRequests,
-                                    int maxConnectionsPerNode,
-                                    int maxTotalConnections,
-                                    String... bootstrapUrls) {
-        this(coreThreads,
-             maxThreads,
-             maxQueuedRequests,
-             maxConnectionsPerNode,
-             maxTotalConnections,
-             DEFAULT_SOCKET_TIMEOUT_MS,
-             AbstractStoreClientFactory.DEFAULT_ROUTING_TIMEOUT_MS,
-             bootstrapUrls);
-    }
-
-    public SocketStoreClientFactory(int coreThreads,
-                                    int maxThreads,
-                                    int maxQueuedRequests,
-                                    int maxConnectionsPerNode,
-                                    int maxTotalConnections,
-                                    int socketTimeoutMs,
-                                    int routingTimeoutMs,
-                                    String... bootstrapUrls) {
-        this(new ThreadPoolExecutor(coreThreads,
-                                    maxThreads,
-                                    10000L,
-                                    TimeUnit.MILLISECONDS,
-                                    new LinkedBlockingQueue&lt;Runnable&gt;(maxQueuedRequests),
-                                    new DaemonThreadFactory(&quot;voldemort-client-thread-&quot;),
-                                    new ThreadPoolExecutor.CallerRunsPolicy()),
-             maxConnectionsPerNode,
-             maxTotalConnections,
-             socketTimeoutMs,
-             routingTimeoutMs,
-             AbstractStoreClientFactory.DEFAULT_NODE_BANNAGE_MS,
-             new DefaultSerializerFactory(),
-             bootstrapUrls);
-    }
-
-    public SocketStoreClientFactory(ExecutorService service,
-                                    int maxConnectionsPerNode,
-                                    int maxTotalConnections,
-                                    int socketTimeoutMs,
-                                    int routingTimeoutMs,
-                                    int defaultNodeBannageMs,
-                                    String... bootstrapUrls) {
-        this(service,
-             maxConnectionsPerNode,
-             maxTotalConnections,
-             socketTimeoutMs,
-             routingTimeoutMs,
-             defaultNodeBannageMs,
-             new DefaultSerializerFactory(),
-             bootstrapUrls);
-    }
-
-    public SocketStoreClientFactory(int coreThreads,
-                                    int maxThreads,
-                                    int maxQueuedRequests,
-                                    int maxConnectionsPerNode,
-                                    int maxTotalConnections,
-                                    int socketTimeoutMs,
-                                    int socketBufferSize,
-                                    int routingTimeoutMs,
-                                    String... bootstrapUrls) {
-        this(new ThreadPoolExecutor(coreThreads,
-                                    maxThreads,
-                                    10000L,
-                                    TimeUnit.MILLISECONDS,
-                                    new LinkedBlockingQueue&lt;Runnable&gt;(maxQueuedRequests),
-                                    new DaemonThreadFactory(&quot;voldemort-client-thread-&quot;),
-                                    new ThreadPoolExecutor.CallerRunsPolicy()),
-             maxConnectionsPerNode,
-             maxTotalConnections,
-             socketTimeoutMs,
-             routingTimeoutMs,
-             AbstractStoreClientFactory.DEFAULT_NODE_BANNAGE_MS,
-             socketBufferSize,
-             new DefaultSerializerFactory(),
-             bootstrapUrls);
-    }
-
-    public SocketStoreClientFactory(ExecutorService service,
-                                    int maxConnectionsPerNode,
-                                    int maxTotalConnections,
-                                    int socketTimeoutMs,
-                                    int routingTimeoutMs,
-                                    int defaultNodeBannageMs,
-                                    SerializerFactory serializerFactory,
-                                    String... boostrapUrls) {
-        super(service, serializerFactory, routingTimeoutMs, defaultNodeBannageMs, boostrapUrls);
-        this.socketPool = new SocketPool(maxConnectionsPerNode,
-                                         maxTotalConnections,
-                                         socketTimeoutMs,
-                                         DEFAULT_SOCKET_BUFFER_SIZE);
-    }
-
-    public SocketStoreClientFactory(ExecutorService service,
-                                    int maxConnectionsPerNode,
-                                    int maxTotalConnections,
-                                    int socketTimeoutMs,
-                                    int routingTimeoutMs,
-                                    int defaultNodeBannageMs,
-                                    int socketBufferSize,
-                                    SerializerFactory serializerFactory,
-                                    String... boostrapUrls) {
-        super(service, serializerFactory, routingTimeoutMs, defaultNodeBannageMs, boostrapUrls);
-        this.socketPool = new SocketPool(maxConnectionsPerNode,
-                                         maxTotalConnections,
-                                         socketTimeoutMs,
-                                         socketBufferSize);
+    private final SocketPool socketPool;
+    private final RoutingTier routingTier;
+
+    public SocketStoreClientFactory(ClientConfig config) {
+        super(config);
+        this.routingTier = config.getRoutingTier();
+        this.socketPool = new SocketPool(config.getMaxConnectionsPerNode(),
+                                         config.getMaxTotalConnections(),
+                                         config.getConnectionTimeout(TimeUnit.MILLISECONDS),
+                                         config.getSocketTimeout(TimeUnit.MILLISECONDS),
+                                         config.getSocketBufferSize());
+        registerJmx(JmxUtils.createObjectName(SocketPool.class), socketPool);
     }
 
     @Override
-    protected Store&lt;ByteArray, byte[]&gt; getStore(String storeName, String host, int port) {
-        return new SocketStore(Utils.notNull(storeName), Utils.notNull(host), port, socketPool);
+    protected Store&lt;ByteArray, byte[]&gt; getStore(String storeName,
+                                                String host,
+                                                int port,
+                                                RequestFormatType type) {
+        return new SocketStore(Utils.notNull(storeName),
+                               Utils.notNull(host),
+                               port,
+                               socketPool,
+                               type,
+                               RoutingTier.SERVER.equals(routingTier));
     }
 
     @Override</diff>
      <filename>src/java/voldemort/client/SocketStoreClientFactory.java</filename>
    </modified>
    <modified>
      <diff>@@ -24,10 +24,13 @@ import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 
+import javax.management.MBeanOperationInfo;
+
 import voldemort.VoldemortException;
 import voldemort.annotations.concurrency.Threadsafe;
 import voldemort.annotations.jmx.JmxGetter;
 import voldemort.annotations.jmx.JmxManaged;
+import voldemort.annotations.jmx.JmxOperation;
 import voldemort.utils.Utils;
 
 import com.google.common.collect.Sets;
@@ -93,7 +96,16 @@ public class Cluster implements Serializable {
         return nodesById.size();
     }
 
-    public int getNumberOfTags() {
+    @JmxOperation(impact = MBeanOperationInfo.INFO, description = &quot;The time since last marked unavailable&quot;)
+    public int getNumberOfAvailableNodes(long time) {
+        int available = 0;
+        for(Node node: this.nodesById.values())
+            if(!node.getStatus().isUnavailable(time))
+                available += 1;
+        return available;
+    }
+
+    public int getNumberOfPartitions() {
         return numberOfTags;
     }
 </diff>
      <filename>src/java/voldemort/cluster/Cluster.java</filename>
    </modified>
    <modified>
      <diff>@@ -21,4 +21,5 @@ public class VoldemortOpCode {
     public static final byte GET_OP_CODE = 1;
     public static final byte PUT_OP_CODE = 2;
     public static final byte DELETE_OP_CODE = 3;
+    public static final byte GET_ALL_OP_CODE = 4;
 }</diff>
      <filename>src/java/voldemort/serialization/VoldemortOpCode.java</filename>
    </modified>
    <modified>
      <diff>@@ -24,6 +24,7 @@ import org.apache.log4j.Logger;
 
 import voldemort.annotations.jmx.JmxGetter;
 import voldemort.annotations.jmx.JmxOperation;
+import voldemort.utils.Utils;
 
 /**
  * A helper template for implementing VoldemortService
@@ -36,15 +37,15 @@ public abstract class AbstractService implements VoldemortService {
     private static final Logger logger = Logger.getLogger(VoldemortService.class);
 
     private final AtomicBoolean isStarted;
-    private final String name;
+    private final ServiceType type;
 
-    public AbstractService(String name) {
-        this.name = name;
+    public AbstractService(ServiceType type) {
+        this.type = Utils.notNull(type);
         this.isStarted = new AtomicBoolean(false);
     }
 
-    public String getName() {
-        return name;
+    public ServiceType getType() {
+        return type;
     }
 
     @JmxGetter(name = &quot;started&quot;, description = &quot;Determine if the service has been started.&quot;)
@@ -58,13 +59,13 @@ public abstract class AbstractService implements VoldemortService {
         if(!isntStarted)
             throw new IllegalStateException(&quot;Server is already started!&quot;);
 
-        logger.info(&quot;Starting &quot; + getName());
+        logger.info(&quot;Starting &quot; + getType().getDisplayName());
         startInner();
     }
 
     @JmxOperation(description = &quot;Stop the service.&quot;, impact = MBeanOperationInfo.ACTION)
     public void stop() {
-        logger.info(&quot;Stopping &quot; + getName());
+        logger.info(&quot;Stopping &quot; + getType().getDisplayName());
         synchronized(this) {
             if(!isStarted()) {
                 logger.info(&quot;The service is already stopped, ignoring duplicate attempt.&quot;);</diff>
      <filename>src/java/voldemort/server/AbstractService.java</filename>
    </modified>
    <modified>
      <diff>@@ -22,9 +22,11 @@ import java.io.Serializable;
 import java.util.List;
 import java.util.Properties;
 
+import voldemort.client.protocol.RequestFormatType;
 import voldemort.store.bdb.BdbStorageConfiguration;
 import voldemort.store.memory.CacheStorageConfiguration;
 import voldemort.store.memory.InMemoryStorageConfiguration;
+import voldemort.store.mysql.MysqlStorageConfiguration;
 import voldemort.store.readonly.RandomAccessFileStorageConfiguration;
 import voldemort.utils.ConfigurationException;
 import voldemort.utils.Props;
@@ -82,17 +84,29 @@ public class VoldemortConfig implements Serializable {
     private int socketTimeoutMs;
     private int socketBufferSize;
 
-    private int routingTimeoutMs;
+    private int clientRoutingTimeoutMs;
+    private int clientMaxConnectionsPerNode;
+    private int clientMaxTotalConnections;
+    private int clientConnectionTimeoutMs;
+    private int clientNodeBannageMs;
+    private int clientMaxThreads;
+    private int clientThreadIdleMs;
+    private int clientMaxQueuedRequests;
 
     private int schedulerThreads;
 
-    private boolean enableSlopDetection;
+    private int numCleanupPermits;
+
+    private RequestFormatType requestFormatType;
+
+    private boolean enableSlop;
     private boolean enableGui;
     private boolean enableHttpServer;
     private boolean enableSocketServer;
     private boolean enableJmx;
     private boolean enableVerboseLogging;
     private boolean enableStatTracking;
+    private boolean enableServerRouting;
 
     private List&lt;String&gt; storageConfigurations;
 
@@ -100,6 +114,10 @@ public class VoldemortConfig implements Serializable {
 
     private final long pusherPollMs;
 
+    public VoldemortConfig(int nodeId, String voldemortHome) {
+        this(new Props().with(&quot;node.id&quot;, nodeId).with(&quot;voldemort.home&quot;, voldemortHome));
+    }
+
     public VoldemortConfig(Properties props) {
         this(new Props(props));
     }
@@ -149,21 +167,32 @@ public class VoldemortConfig implements Serializable {
         this.socketTimeoutMs = props.getInt(&quot;socket.timeout.ms&quot;, 4000);
         this.socketBufferSize = (int) props.getBytes(&quot;socket.buffer.size&quot;, 32 * 1024);
 
-        this.routingTimeoutMs = props.getInt(&quot;routing.timeout.ms&quot;, 5000);
+        this.clientMaxConnectionsPerNode = props.getInt(&quot;client.max.connections.per.node&quot;, 5);
+        this.clientMaxTotalConnections = props.getInt(&quot;client.max.total.connections&quot;, 100);
+        this.clientConnectionTimeoutMs = props.getInt(&quot;client.connection.timeout.ms&quot;, 400);
+        this.clientRoutingTimeoutMs = props.getInt(&quot;client.routing.timeout.ms&quot;, 5000);
+        this.clientNodeBannageMs = props.getInt(&quot;client.node.bannage.ms&quot;, 10000);
+        this.clientMaxThreads = props.getInt(&quot;client.max.threads&quot;, 100);
+        this.clientThreadIdleMs = props.getInt(&quot;client.thread.idle.ms&quot;, 5000);
+        this.clientMaxQueuedRequests = props.getInt(&quot;client.max.queued.requests&quot;, 1000);
 
         this.enableHttpServer = props.getBoolean(&quot;http.enable&quot;, true);
         this.enableSocketServer = props.getBoolean(&quot;socket.enable&quot;, true);
         this.enableJmx = props.getBoolean(&quot;jmx.enable&quot;, true);
-        this.enableSlopDetection = props.getBoolean(&quot;slop.detection.enable&quot;, false);
+        this.enableSlop = props.getBoolean(&quot;slop.enable&quot;, true);
         this.enableVerboseLogging = props.getBoolean(&quot;enable.verbose.logging&quot;, true);
         this.enableStatTracking = props.getBoolean(&quot;enable.stat.tracking&quot;, true);
+        this.enableServerRouting = props.getBoolean(&quot;enable.server.routing&quot;, true);
 
         this.pusherPollMs = props.getInt(&quot;pusher.poll.ms&quot;, 2 * 60 * 1000);
 
         this.schedulerThreads = props.getInt(&quot;scheduler.threads&quot;, 3);
 
+        this.numCleanupPermits = props.getInt(&quot;num.cleanup.permits&quot;, 1);
+
         this.storageConfigurations = props.getList(&quot;storage.configs&quot;,
                                                    ImmutableList.of(BdbStorageConfiguration.class.getName(),
+                                                                    MysqlStorageConfiguration.class.getName(),
                                                                     InMemoryStorageConfiguration.class.getName(),
                                                                     CacheStorageConfiguration.class.getName(),
                                                                     RandomAccessFileStorageConfiguration.class.getName()));
@@ -171,6 +200,10 @@ public class VoldemortConfig implements Serializable {
         // save props for access from plugins
         this.allProps = props;
 
+        String requestFormatName = props.getString(&quot;request.format&quot;,
+                                                   RequestFormatType.VOLDEMORT.getName());
+        this.requestFormatType = RequestFormatType.fromName(requestFormatName);
+
         validateParams();
     }
 
@@ -185,11 +218,13 @@ public class VoldemortConfig implements Serializable {
             throw new ConfigurationException(&quot;pusher.poll.ms cannot be less than 1.&quot;);
         if(socketTimeoutMs &lt; 0)
             throw new ConfigurationException(&quot;socket.timeout.ms must be 0 or more ms.&quot;);
-        if(routingTimeoutMs &lt; 0)
+        if(clientRoutingTimeoutMs &lt; 0)
             throw new ConfigurationException(&quot;routing.timeout.ms must be 0 or more ms.&quot;);
         if(schedulerThreads &lt; 1)
             throw new ConfigurationException(&quot;Must have at least 1 scheduler thread, &quot;
                                              + this.schedulerThreads + &quot; set.&quot;);
+        if(enableServerRouting &amp;&amp; !enableSocketServer)
+            throw new ConfigurationException(&quot;Server-side routing is enabled, this requires the socket server to also be enabled.&quot;);
     }
 
     private int getIntEnvVariable(String name) {
@@ -461,19 +496,75 @@ public class VoldemortConfig implements Serializable {
     }
 
     public int getRoutingTimeoutMs() {
-        return this.routingTimeoutMs;
+        return this.clientRoutingTimeoutMs;
+    }
+
+    public void setClientRoutingTimeoutMs(int routingTimeoutMs) {
+        this.clientRoutingTimeoutMs = routingTimeoutMs;
+    }
+
+    public int getClientMaxConnectionsPerNode() {
+        return clientMaxConnectionsPerNode;
+    }
+
+    public void setClientMaxConnectionsPerNode(int maxConnectionsPerNode) {
+        this.clientMaxConnectionsPerNode = maxConnectionsPerNode;
+    }
+
+    public int getClientMaxTotalConnections() {
+        return clientMaxTotalConnections;
+    }
+
+    public void setClientMaxTotalConnections(int maxTotalConnections) {
+        this.clientMaxTotalConnections = maxTotalConnections;
+    }
+
+    public int getClientConnectionTimeoutMs() {
+        return clientConnectionTimeoutMs;
+    }
+
+    public void setClientConnectionTimeoutMs(int connectionTimeoutMs) {
+        this.clientConnectionTimeoutMs = connectionTimeoutMs;
+    }
+
+    public int getClientNodeBannageMs() {
+        return clientNodeBannageMs;
+    }
+
+    public void setClientNodeBannageMs(int nodeBannageMs) {
+        this.clientNodeBannageMs = nodeBannageMs;
+    }
+
+    public int getClientMaxThreads() {
+        return clientMaxThreads;
+    }
+
+    public void setClientMaxThreads(int clientMaxThreads) {
+        this.clientMaxThreads = clientMaxThreads;
+    }
+
+    public int getClientThreadIdleMs() {
+        return clientThreadIdleMs;
+    }
+
+    public void setClientThreadIdleMs(int clientThreadIdleMs) {
+        this.clientThreadIdleMs = clientThreadIdleMs;
     }
 
-    public void setRoutingTimeoutMs(int routingTimeoutMs) {
-        this.routingTimeoutMs = routingTimeoutMs;
+    public int getClientMaxQueuedRequests() {
+        return clientMaxQueuedRequests;
     }
 
-    public boolean isSlopDetectionEnabled() {
-        return this.enableSlopDetection;
+    public void setClientMaxQueuedRequests(int clientMaxQueuedRequests) {
+        this.clientMaxQueuedRequests = clientMaxQueuedRequests;
     }
 
-    public void setEnableSlopDetection(boolean enableSlopDetection) {
-        this.enableSlopDetection = enableSlopDetection;
+    public boolean isSlopEnabled() {
+        return this.enableSlop;
+    }
+
+    public void setEnableSlop(boolean enableSlop) {
+        this.enableSlop = enableSlop;
     }
 
     public boolean isVerboseLoggingEnabled() {
@@ -588,4 +679,28 @@ public class VoldemortConfig implements Serializable {
         return this.allProps;
     }
 
+    public void setRequestFormatType(RequestFormatType type) {
+        this.requestFormatType = type;
+    }
+
+    public RequestFormatType getRequestFormatType() {
+        return this.requestFormatType;
+    }
+
+    public boolean isServerRoutingEnabled() {
+        return this.enableServerRouting;
+    }
+
+    public void setEnableServerRouting(boolean enableServerRouting) {
+        this.enableServerRouting = enableServerRouting;
+    }
+
+    public int getNumCleanupPermits() {
+        return numCleanupPermits;
+    }
+
+    public void setNumCleanupPermits(int numCleanupPermits) {
+        this.numCleanupPermits = numCleanupPermits;
+    }
+
 }</diff>
      <filename>src/java/voldemort/server/VoldemortConfig.java</filename>
    </modified>
    <modified>
      <diff>@@ -20,28 +20,28 @@ import static voldemort.utils.Utils.croak;
 
 import java.io.File;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
 
 import org.apache.log4j.Logger;
 
 import voldemort.VoldemortException;
+import voldemort.client.protocol.RequestFormatType;
 import voldemort.cluster.Cluster;
 import voldemort.cluster.Node;
 import voldemort.server.http.HttpService;
 import voldemort.server.jmx.JmxService;
+import voldemort.server.protocol.RequestHandler;
+import voldemort.server.protocol.RequestHandlerFactory;
 import voldemort.server.scheduler.SchedulerService;
 import voldemort.server.socket.SocketService;
 import voldemort.server.storage.StorageService;
-import voldemort.store.Store;
 import voldemort.store.metadata.MetadataStore;
-import voldemort.utils.ByteArray;
 import voldemort.utils.Props;
 import voldemort.utils.SystemTime;
 import voldemort.utils.Utils;
 
+import com.google.common.collect.ImmutableList;
+
 /**
  * This is the main server, it bootstraps all the services.
  * 
@@ -56,61 +56,56 @@ public class VoldemortServer extends AbstractService {
     public static final long DEFAULT_PUSHER_POLL_MS = 60 * 1000;
 
     private final Node identityNode;
-    private final Cluster cluster;
     private final MetadataStore metadataStore;
     private final List&lt;VoldemortService&gt; services;
-    private final ConcurrentMap&lt;String, Store&lt;ByteArray, byte[]&gt;&gt; storeMap;
+    private final StoreRepository storeRepository;
     private final VoldemortConfig voldemortConfig;
 
     public VoldemortServer(VoldemortConfig config) {
-        super(&quot;voldemort-server&quot;);
+        super(ServiceType.VOLDEMORT);
         this.voldemortConfig = config;
-        this.storeMap = new ConcurrentHashMap&lt;String, Store&lt;ByteArray, byte[]&gt;&gt;();
-        this.metadataStore = new MetadataStore(new File(voldemortConfig.getMetadataDirectory()));
-        this.cluster = this.metadataStore.getCluster();
-        this.identityNode = this.cluster.getNodeById(voldemortConfig.getNodeId());
+        this.storeRepository = new StoreRepository();
+        this.metadataStore = MetadataStore.readFromDirectory(new File(voldemortConfig.getMetadataDirectory()));
+        this.identityNode = metadataStore.getCluster().getNodeById(voldemortConfig.getNodeId());
         this.services = createServices();
     }
 
-    public VoldemortServer(Props props, Cluster cluster) {
-        super(&quot;voldemort-server&quot;);
+    public VoldemortServer(Props props, MetadataStore metadataStore) {
+        super(ServiceType.VOLDEMORT);
         this.voldemortConfig = new VoldemortConfig(props);
-        this.cluster = cluster;
-        this.identityNode = cluster.getNodeById(voldemortConfig.getNodeId());
-        this.storeMap = new ConcurrentHashMap&lt;String, Store&lt;ByteArray, byte[]&gt;&gt;();
+        this.identityNode = metadataStore.getCluster().getNodeById(voldemortConfig.getNodeId());
+        this.storeRepository = new StoreRepository();
         this.services = createServices();
-        this.metadataStore = new MetadataStore(new File(voldemortConfig.getMetadataDirectory()));
+        this.metadataStore = metadataStore;
     }
 
     private List&lt;VoldemortService&gt; createServices() {
-        List&lt;VoldemortService&gt; services = Collections.synchronizedList(new ArrayList&lt;VoldemortService&gt;());
-        SchedulerService scheduler = new SchedulerService(&quot;scheduler-service&quot;,
-                                                          voldemortConfig.getSchedulerThreads(),
+        /* Services are given in the order they must be started */
+        List&lt;VoldemortService&gt; services = new ArrayList&lt;VoldemortService&gt;();
+        SchedulerService scheduler = new SchedulerService(voldemortConfig.getSchedulerThreads(),
                                                           SystemTime.INSTANCE);
         services.add(scheduler);
-        services.add(new StorageService(&quot;storage-service&quot;,
-                                        this.storeMap,
-                                        scheduler,
-                                        voldemortConfig));
+        services.add(new StorageService(storeRepository, metadataStore, scheduler, voldemortConfig));
         if(voldemortConfig.isHttpServerEnabled())
-            services.add(new HttpService(&quot;http-service&quot;,
-                                         this,
+            services.add(new HttpService(this,
+                                         storeRepository,
+                                         RequestFormatType.VOLDEMORT,
                                          voldemortConfig.getMaxThreads(),
                                          identityNode.getHttpPort()));
+        RequestHandler requestHandler = new RequestHandlerFactory(this.storeRepository).getRequestHandler(voldemortConfig.getRequestFormatType());
         if(voldemortConfig.isSocketServerEnabled())
-            services.add(new SocketService(&quot;socket-service&quot;,
-                                           storeMap,
+            services.add(new SocketService(requestHandler,
                                            identityNode.getSocketPort(),
                                            voldemortConfig.getCoreThreads(),
                                            voldemortConfig.getMaxThreads(),
                                            voldemortConfig.getSocketBufferSize()));
         if(voldemortConfig.isJmxEnabled())
-            services.add(new JmxService(&quot;jmx-service&quot;, this, cluster, storeMap, services));
-
-        // we want services to stop in the opposite order they started
-        Collections.reverse(services);
+            services.add(new JmxService(this,
+                                        this.metadataStore.getCluster(),
+                                        storeRepository,
+                                        services));
 
-        return services;
+        return ImmutableList.copyOf(services);
     }
 
     @Override
@@ -121,16 +116,6 @@ public class VoldemortServer extends AbstractService {
             service.start();
         long end = System.currentTimeMillis();
 
-        // add a shutdown hook to stop the server
-        Runtime.getRuntime().addShutdownHook(new Thread() {
-
-            @Override
-            public void run() {
-                if(VoldemortServer.this.isStarted())
-                    VoldemortServer.this.stop();
-            }
-        });
-
         logger.info(&quot;Startup completed in &quot; + (end - start) + &quot; ms.&quot;);
     }
 
@@ -144,7 +129,8 @@ public class VoldemortServer extends AbstractService {
     protected void stopInner() throws VoldemortException {
         List&lt;VoldemortException&gt; exceptions = new ArrayList&lt;VoldemortException&gt;();
         logger.info(&quot;Stopping services:&quot;);
-        for(VoldemortService service: services) {
+        /* Stop in reverse order */
+        for(VoldemortService service: Utils.reversed(services)) {
             try {
                 service.stop();
             } catch(VoldemortException e) {
@@ -172,9 +158,19 @@ public class VoldemortServer extends AbstractService {
             Utils.croak(&quot;Error while loading configuration: &quot; + e.getMessage());
         }
 
-        VoldemortServer server = new VoldemortServer(config);
+        final VoldemortServer server = new VoldemortServer(config);
         if(!server.isStarted())
             server.start();
+
+        // add a shutdown hook to stop the server
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+
+            @Override
+            public void run() {
+                if(server.isStarted())
+                    server.stop();
+            }
+        });
     }
 
     public Node getIdentityNode() {
@@ -182,26 +178,26 @@ public class VoldemortServer extends AbstractService {
     }
 
     public Cluster getCluster() {
-        return cluster;
+        return metadataStore.getCluster();
     }
 
     public List&lt;VoldemortService&gt; getServices() {
         return services;
     }
 
-    public VoldemortService getService(String name) {
+    public VoldemortService getService(ServiceType type) {
         for(VoldemortService service: services)
-            if(service.getName().equals(name))
+            if(service.getType().equals(type))
                 return service;
-        return null;
-    }
-
-    public ConcurrentMap&lt;String, Store&lt;ByteArray, byte[]&gt;&gt; getStoreMap() {
-        return storeMap;
+        throw new IllegalStateException(type.getDisplayName() + &quot; has not been initialized.&quot;);
     }
 
     public VoldemortConfig getVoldemortConfig() {
         return this.voldemortConfig;
     }
 
+    public StoreRepository getStoreRepository() {
+        return this.storeRepository;
+    }
+
 }</diff>
      <filename>src/java/voldemort/server/VoldemortServer.java</filename>
    </modified>
    <modified>
      <diff>@@ -25,9 +25,9 @@ package voldemort.server;
 public interface VoldemortService {
 
     /**
-     * @return The name of this service
+     * @return The type of this service
      */
-    public String getName();
+    public ServiceType getType();
 
     /**
      * Start the service.</diff>
      <filename>src/java/voldemort/server/VoldemortService.java</filename>
    </modified>
    <modified>
      <diff>@@ -16,6 +16,7 @@
 
 package voldemort.server.http;
 
+import org.apache.log4j.Logger;
 import org.mortbay.jetty.Connector;
 import org.mortbay.jetty.Server;
 import org.mortbay.jetty.nio.SelectChannelConnector;
@@ -26,11 +27,16 @@ import org.mortbay.thread.BoundedThreadPool;
 import voldemort.VoldemortException;
 import voldemort.annotations.jmx.JmxGetter;
 import voldemort.annotations.jmx.JmxManaged;
+import voldemort.client.protocol.RequestFormatType;
 import voldemort.server.AbstractService;
+import voldemort.server.ServiceType;
+import voldemort.server.StoreRepository;
 import voldemort.server.VoldemortServer;
 import voldemort.server.http.gui.AdminServlet;
 import voldemort.server.http.gui.ReadOnlyStoreManagementServlet;
 import voldemort.server.http.gui.VelocityEngine;
+import voldemort.server.protocol.RequestHandler;
+import voldemort.server.protocol.RequestHandlerFactory;
 
 /**
  * An embedded http server that uses jetty
@@ -41,19 +47,27 @@ import voldemort.server.http.gui.VelocityEngine;
 @JmxManaged(description = &quot;A store connector that serves remote clients via HTTP.&quot;)
 public class HttpService extends AbstractService {
 
+    private final Logger logger = Logger.getLogger(HttpService.class);
+
     private final int port;
     private final int numberOfThreads;
     private final VoldemortServer server;
     private final VelocityEngine velocityEngine;
+    private final RequestHandler requestHandler;
     private Server httpServer;
     private Context context;
 
-    public HttpService(String name, VoldemortServer server, int numberOfThreads, int httpPort) {
-        super(name);
+    public HttpService(VoldemortServer server,
+                       StoreRepository storeRepository,
+                       RequestFormatType requestType,
+                       int numberOfThreads,
+                       int httpPort) {
+        super(ServiceType.HTTP);
         this.port = httpPort;
         this.numberOfThreads = numberOfThreads;
         this.server = server;
         this.velocityEngine = new VelocityEngine(VoldemortServletContextListener.VOLDEMORT_TEMPLATE_DIR);
+        this.requestHandler = new RequestHandlerFactory(storeRepository).getRequestHandler(requestType);
     }
 
     @Override
@@ -76,13 +90,14 @@ public class HttpService extends AbstractService {
                                  velocityEngine);
             context.addServlet(new ServletHolder(new AdminServlet(server, velocityEngine)),
                                &quot;/admin&quot;);
-            context.addServlet(new ServletHolder(new StoreServlet(server.getStoreMap())), &quot;/*&quot;);
+            context.addServlet(new ServletHolder(new StoreServlet(requestHandler)), &quot;/stores&quot;);
             context.addServlet(new ServletHolder(new ReadOnlyStoreManagementServlet(server,
                                                                                     velocityEngine)),
                                &quot;/read-only/mgmt&quot;);
             this.context = context;
             this.httpServer = httpServer;
             this.httpServer.start();
+            logger.info(&quot;HTTP service started on port &quot; + this.port);
         } catch(Exception e) {
             throw new VoldemortException(e);
         }
@@ -91,8 +106,10 @@ public class HttpService extends AbstractService {
     @Override
     public void stopInner() {
         try {
-            httpServer.stop();
-            context.destroy();
+            if(httpServer != null)
+                httpServer.stop();
+            if(context != null)
+                context.destroy();
         } catch(Exception e) {
             throw new VoldemortException(e);
         }
@@ -110,4 +127,8 @@ public class HttpService extends AbstractService {
         return port;
     }
 
+    public RequestHandler getRequestHandler() {
+        return requestHandler;
+    }
+
 }</diff>
      <filename>src/java/voldemort/server/http/HttpService.java</filename>
    </modified>
    <modified>
      <diff>@@ -16,12 +16,10 @@
 
 package voldemort.server.http;
 
+import java.io.DataInputStream;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.net.HttpURLConnection;
-import java.util.List;
-import java.util.concurrent.ConcurrentMap;
-import java.util.regex.Pattern;
 
 import javax.servlet.ServletContext;
 import javax.servlet.ServletException;
@@ -29,22 +27,12 @@ import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
-import org.apache.commons.codec.DecoderException;
-import org.apache.commons.codec.binary.Base64;
-import org.apache.commons.codec.binary.Hex;
+import org.apache.log4j.Logger;
 
-import voldemort.VoldemortException;
+import voldemort.server.ServiceType;
 import voldemort.server.VoldemortServer;
-import voldemort.store.Store;
-import voldemort.store.http.HttpResponseCodeErrorMapper;
-import voldemort.utils.ByteArray;
-import voldemort.utils.ByteUtils;
-import voldemort.utils.Pair;
+import voldemort.server.protocol.RequestHandler;
 import voldemort.utils.Utils;
-import voldemort.versioning.VectorClock;
-import voldemort.versioning.Versioned;
-
-import com.google.common.base.Join;
 
 /**
  * Handles requests from HttpStores and multiplexes them to the appropriate
@@ -55,19 +43,16 @@ import com.google.common.base.Join;
  */
 public class StoreServlet extends HttpServlet {
 
-    private static final Pattern SLASH_PATTERN = Pattern.compile(&quot;/&quot;);
+    private static final Logger logger = Logger.getLogger(StoreServlet.class);
     private static final long serialVersionUID = 1;
-    private static final String VERSION_EXTENSION = &quot;X-vldmt-version&quot;;
-    private static final HttpResponseCodeErrorMapper httpResponseCodeErrorMapper = new HttpResponseCodeErrorMapper();
-    private static final Hex urlCodec = new Hex();
 
-    private ConcurrentMap&lt;String, Store&lt;ByteArray, byte[]&gt;&gt; stores;
+    private RequestHandler requestHandler;
 
     /* For use by servlet container */
     public StoreServlet() {}
 
-    public StoreServlet(ConcurrentMap&lt;String, Store&lt;ByteArray, byte[]&gt;&gt; stores) {
-        this.stores = stores;
+    public StoreServlet(RequestHandler handler) {
+        this.requestHandler = handler;
     }
 
     @Override
@@ -75,108 +60,24 @@ public class StoreServlet extends HttpServlet {
         super.init();
         // if we don't already have a stores map, attempt to initialize from the
         // servlet context
-        if(this.stores == null) {
+        if(this.requestHandler == null) {
             ServletContext context = this.getServletContext();
             VoldemortServer server = (VoldemortServer) Utils.notNull(context.getAttribute(VoldemortServletContextListener.SERVER_CONFIG_KEY));
-            this.stores = server.getStoreMap();
+            HttpService httpService = (HttpService) server.getService(ServiceType.HTTP);
+            this.requestHandler = httpService.getRequestHandler();
         }
     }
 
-    private Store&lt;ByteArray, byte[]&gt; getStore(String name) {
-        Store&lt;ByteArray, byte[]&gt; store = stores.get(name);
-        if(store == null)
-            throw new VoldemortException(&quot;No store named '&quot; + name + &quot;'.&quot;);
-        return store;
-    }
-
     @Override
-    protected void doGet(HttpServletRequest request, HttpServletResponse response)
+    protected void doPost(HttpServletRequest request, HttpServletResponse response)
             throws ServletException, IOException {
-        String[] path = SLASH_PATTERN.split(request.getPathInfo());
-        Pair&lt;ByteArray, String&gt; keyAndStore = getKeyAndStore(path);
-        Store&lt;ByteArray, byte[]&gt; store = getStore(keyAndStore.getSecond());
-        DataOutputStream stream = new DataOutputStream(response.getOutputStream());
         try {
-            List&lt;Versioned&lt;byte[]&gt;&gt; values = store.get(keyAndStore.getFirst());
-            for(Versioned&lt;byte[]&gt; versioned: values) {
-                byte[] clock = ((VectorClock) versioned.getVersion()).toBytes();
-                byte[] value = versioned.getValue();
-                stream.writeInt(clock.length + value.length);
-                stream.write(clock);
-                stream.write(value);
-            }
-        } catch(VoldemortException v) {
-            HttpResponseCodeErrorMapper.ResponseCode code = httpResponseCodeErrorMapper.mapErrorToResponseCode(v);
-            response.setContentType(&quot;text/xml&quot;);
-            response.sendError(code.getCode(), errorXml(v, code.getText()));
-        }
-    }
-
-    @Override
-    protected void doPut(HttpServletRequest request, HttpServletResponse response)
-            throws ServletException, IOException {
-        String[] path = SLASH_PATTERN.split(request.getPathInfo());
-        Pair&lt;ByteArray, String&gt; keyAndStore = getKeyAndStore(path);
-        Store&lt;ByteArray, byte[]&gt; store = getStore(keyAndStore.getSecond());
-        int size = request.getContentLength();
-        byte[] contents = new byte[size];
-        ByteUtils.read(request.getInputStream(), contents);
-        try {
-            VectorClock clock = new VectorClock(Base64.decodeBase64(request.getHeader(VERSION_EXTENSION)
-                                                                           .getBytes()));
-            store.put(keyAndStore.getFirst(), new Versioned&lt;byte[]&gt;(contents, clock));
-        } catch(VoldemortException v) {
-            HttpResponseCodeErrorMapper.ResponseCode code = httpResponseCodeErrorMapper.mapErrorToResponseCode(v);
-            response.setContentType(&quot;text/xml&quot;);
-            response.sendError(code.getCode(), errorXml(v, code.getText()));
-        }
-    }
-
-    @Override
-    public void doDelete(HttpServletRequest request, HttpServletResponse response)
-            throws ServletException, IOException {
-        String[] path = SLASH_PATTERN.split(request.getPathInfo());
-        Pair&lt;ByteArray, String&gt; keyAndStore = getKeyAndStore(path);
-        Store&lt;ByteArray, byte[]&gt; store = getStore(keyAndStore.getSecond());
-        try {
-            byte[] versionBytes = ByteUtils.getBytes(request.getHeader(VERSION_EXTENSION), &quot;UTF-8&quot;);
-            VectorClock clock = new VectorClock(Base64.decodeBase64(versionBytes));
-            boolean succeeded = store.delete(keyAndStore.getFirst(), clock);
-            if(!succeeded)
-                response.sendError(HttpURLConnection.HTTP_NOT_FOUND);
-        } catch(VoldemortException v) {
-            HttpResponseCodeErrorMapper.ResponseCode code = httpResponseCodeErrorMapper.mapErrorToResponseCode(v);
-            response.setContentType(&quot;text/xml&quot;);
-            response.sendError(code.getCode(), errorXml(v, code.getText()));
-        }
-    }
-
-    public static String getKey(String url) {
-        String[] path = url.split(&quot;/&quot;);
-        return path[path.length - 1];
-    }
-
-    public String errorXml(VoldemortException type, String message) {
-        return &quot;&lt;?xml version='1.0' encoding='UTF-8'?&gt;&quot; + &quot;&lt;error&gt;&quot; + &quot;&lt;name&gt;&quot;
-               + type.getClass().getName() + &quot;&lt;/name&gt;&quot; + &quot;&lt;message&gt;&quot; + message + &quot;&lt;/message&gt;&quot;
-               + &quot;&lt;/error&gt;&quot;;
-    }
+            requestHandler.handleRequest(new DataInputStream(request.getInputStream()),
+                                         new DataOutputStream(response.getOutputStream()));
 
-    private Pair&lt;ByteArray, String&gt; getKeyAndStore(String[] urlPieces) {
-        if(urlPieces.length &lt; 2) {
-            throw new VoldemortException(&quot;Invalid request for &quot; + Join.join(&quot;.&quot;, urlPieces)
-                                         + &quot;: must specify both a store and key.&quot;);
-        } else if(urlPieces.length == 2) {
-            return Pair.create(ByteArray.EMPTY, urlPieces[urlPieces.length - 1]);
-        } else {
-            String keyStr = urlPieces[urlPieces.length - 1];
-            String store = urlPieces[urlPieces.length - 2];
-            try {
-                byte[] key = ByteUtils.getBytes(keyStr, &quot;UTF-8&quot;);
-                return Pair.create(new ByteArray(urlCodec.decode(key)), store);
-            } catch(DecoderException e) {
-                throw new VoldemortException(&quot;Corrupt key format.&quot;, e);
-            }
+        } catch(Exception e) {
+            logger.error(&quot;Uncaught exception in store servlet:&quot;, e);
+            response.sendError(HttpURLConnection.HTTP_UNAVAILABLE, e.getMessage());
         }
     }
 </diff>
      <filename>src/java/voldemort/server/http/StoreServlet.java</filename>
    </modified>
    <modified>
      <diff>@@ -63,7 +63,7 @@ public class AdminServlet extends HttpServlet {
             throws ServletException, IOException {
         Map&lt;String, Object&gt; params = Maps.newHashMap();
         params.put(&quot;cluster&quot;, server.getCluster());
-        params.put(&quot;stores&quot;, server.getStoreMap());
+        params.put(&quot;repository&quot;, server.getStoreRepository());
         params.put(&quot;services&quot;, server.getServices());
         velocityEngine.render(&quot;admin.vm&quot;, params, response.getOutputStream());
     }</diff>
      <filename>src/java/voldemort/server/http/gui/AdminServlet.java</filename>
    </modified>
    <modified>
      <diff>@@ -26,6 +26,7 @@ import javax.servlet.http.HttpServlet;
 import javax.servlet.http.HttpServletRequest;
 import javax.servlet.http.HttpServletResponse;
 
+import voldemort.client.ClientConfig;
 import voldemort.client.HttpStoreClientFactory;
 import voldemort.client.StoreClient;
 import voldemort.client.StoreClientFactory;
@@ -59,7 +60,8 @@ public class QueryServlet extends HttpServlet {
     public QueryServlet(VelocityEngine engine, URI bootstrap) {
         this.engine = engine;
         this.uri = bootstrap;
-        this.clientFactory = new HttpStoreClientFactory(1, bootstrap.toString());
+        this.clientFactory = new HttpStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrap.toString())
+                                                                          .setMaxThreads(1));
         this.serializerFactory = new DefaultSerializerFactory();
     }
 </diff>
      <filename>src/java/voldemort/server/http/gui/QueryServlet.java</filename>
    </modified>
    <modified>
      <diff>@@ -18,6 +18,7 @@ package voldemort.server.http.gui;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.List;
 import java.util.Map;
 
 import javax.servlet.ServletException;
@@ -28,13 +29,17 @@ import javax.servlet.http.HttpServletResponse;
 import org.apache.log4j.Logger;
 
 import voldemort.VoldemortException;
+import voldemort.server.ServiceType;
 import voldemort.server.VoldemortServer;
 import voldemort.server.http.VoldemortServletContextListener;
 import voldemort.server.storage.StorageService;
+import voldemort.store.StorageEngine;
 import voldemort.store.readonly.FileFetcher;
 import voldemort.store.readonly.RandomAccessFileStore;
+import voldemort.utils.ByteArray;
 import voldemort.utils.Utils;
 
+import com.google.common.collect.Lists;
 import com.google.common.collect.Maps;
 
 public class ReadOnlyStoreManagementServlet extends HttpServlet {
@@ -42,7 +47,7 @@ public class ReadOnlyStoreManagementServlet extends HttpServlet {
     private static final long serialVersionUID = 1;
     private static final Logger logger = Logger.getLogger(ReadOnlyStoreManagementServlet.class);
 
-    private Map&lt;String, RandomAccessFileStore&gt; stores;
+    private List&lt;RandomAccessFileStore&gt; stores;
     private VelocityEngine velocityEngine;
     private FileFetcher fileFetcher;
 
@@ -70,10 +75,15 @@ public class ReadOnlyStoreManagementServlet extends HttpServlet {
         this.velocityEngine = (VelocityEngine) Utils.notNull(getServletContext().getAttribute(VoldemortServletContextListener.VELOCITY_ENGINE_KEY));
     }
 
-    private Map&lt;String, RandomAccessFileStore&gt; getReadOnlyStores(VoldemortServer server) {
+    private List&lt;RandomAccessFileStore&gt; getReadOnlyStores(VoldemortServer server) {
         StorageService storage = (StorageService) Utils.notNull(server)
-                                                       .getService(&quot;storage-service&quot;);
-        return storage.getReadOnlyStores();
+                                                       .getService(ServiceType.STORAGE);
+        List&lt;RandomAccessFileStore&gt; l = Lists.newArrayList();
+        for(StorageEngine&lt;ByteArray, byte[]&gt; engine: storage.getStoreRepository()
+                                                            .getStorageEnginesByClass(RandomAccessFileStore.class)) {
+            l.add((RandomAccessFileStore) engine);
+        }
+        return l;
     }
 
     @Override
@@ -92,7 +102,8 @@ public class ReadOnlyStoreManagementServlet extends HttpServlet {
                 String indexFile = getRequired(req, &quot;index&quot;);
                 String dataFile = getRequired(req, &quot;data&quot;);
                 String storeName = getRequired(req, &quot;store&quot;);
-                if(!stores.containsKey(storeName))
+                RandomAccessFileStore store = this.getStore(storeName);
+                if(store == null)
                     throw new ServletException(&quot;'&quot; + storeName
                                                + &quot;' is not a registered read-only store.&quot;);
                 if(!Utils.isReadableFile(indexFile))
@@ -102,7 +113,6 @@ public class ReadOnlyStoreManagementServlet extends HttpServlet {
                     throw new ServletException(&quot;Data file '&quot; + dataFile
                                                + &quot;' is not a readable file.&quot;);
 
-                RandomAccessFileStore store = stores.get(storeName);
                 store.swapFiles(indexFile, dataFile);
                 resp.getWriter().write(&quot;Swap completed.&quot;);
             } else if(&quot;fetch&quot;.equals(getRequired(req, &quot;operation&quot;))) {
@@ -140,4 +150,11 @@ public class ReadOnlyStoreManagementServlet extends HttpServlet {
             throw new ServletException(&quot;Missing required parameter '&quot; + name + &quot;'.&quot;);
         return val;
     }
+
+    private RandomAccessFileStore getStore(String storeName) throws ServletException {
+        for(RandomAccessFileStore store: this.stores)
+            if(store.getName().equals(storeName))
+                return store;
+        throw new ServletException(&quot;'&quot; + storeName + &quot;' is not a registered read-only store.&quot;);
+    }
 }</diff>
      <filename>src/java/voldemort/server/http/gui/ReadOnlyStoreManagementServlet.java</filename>
    </modified>
    <modified>
      <diff>@@ -46,7 +46,7 @@
 	        &lt;th&gt;Store Name&lt;/th&gt;
 	        &lt;th&gt;Type&lt;/th&gt;
 	      &lt;/tr&gt;
-	    #foreach($store in $stores.values())
+	    #foreach($store in $repository.allStorageEngines)
 	   	  &lt;tr&gt;
 	   	    &lt;td&gt;$store.getName()&lt;/td&gt;
 	   	    &lt;td&gt;$store.getClass().getName()&lt;/td&gt;</diff>
      <filename>src/java/voldemort/server/http/gui/templates/admin.vm</filename>
    </modified>
    <modified>
      <diff>@@ -17,9 +17,10 @@
 package voldemort.server.jmx;
 
 import java.lang.management.ManagementFactory;
+import java.util.ArrayList;
+import java.util.Collection;
 import java.util.HashSet;
 import java.util.List;
-import java.util.Map;
 import java.util.Set;
 
 import javax.management.MBeanServer;
@@ -30,6 +31,8 @@ import org.apache.log4j.Logger;
 import voldemort.annotations.jmx.JmxManaged;
 import voldemort.cluster.Cluster;
 import voldemort.server.AbstractService;
+import voldemort.server.ServiceType;
+import voldemort.server.StoreRepository;
 import voldemort.server.VoldemortServer;
 import voldemort.server.VoldemortService;
 import voldemort.store.Store;
@@ -52,20 +55,19 @@ public class JmxService extends AbstractService {
     private final Cluster cluster;
     private final List&lt;VoldemortService&gt; services;
     private final Set&lt;ObjectName&gt; registeredBeans;
-    private final Map&lt;String, Store&lt;ByteArray, byte[]&gt;&gt; storeMap;
+    private final StoreRepository storeRepository;
 
-    public JmxService(String name,
-                      VoldemortServer server,
+    public JmxService(VoldemortServer server,
                       Cluster cluster,
-                      Map&lt;String, Store&lt;ByteArray, byte[]&gt;&gt; storeMap,
-                      List&lt;VoldemortService&gt; services) {
-        super(name);
+                      StoreRepository storeRepository,
+                      Collection&lt;VoldemortService&gt; services) {
+        super(ServiceType.JMX);
         this.server = server;
         this.cluster = cluster;
-        this.services = services;
+        this.services = new ArrayList&lt;VoldemortService&gt;(services);
         this.mbeanServer = ManagementFactory.getPlatformMBeanServer();
         this.registeredBeans = new HashSet&lt;ObjectName&gt;();
-        this.storeMap = storeMap;
+        this.storeRepository = storeRepository;
     }
 
     @Override
@@ -73,10 +75,11 @@ public class JmxService extends AbstractService {
         registerBean(server, JmxUtils.createObjectName(VoldemortServer.class));
         registerBean(cluster, JmxUtils.createObjectName(Cluster.class));
         for(VoldemortService service: services) {
-            logger.debug(&quot;Registering mbean for service '&quot; + service.getName() + &quot;'.&quot;);
+            logger.debug(&quot;Registering mbean for service '&quot; + service.getType().getDisplayName()
+                         + &quot;'.&quot;);
             registerBean(service, JmxUtils.createObjectName(service.getClass()));
         }
-        for(Store&lt;ByteArray, byte[]&gt; store: storeMap.values()) {
+        for(Store&lt;ByteArray, byte[]&gt; store: this.storeRepository.getAllStorageEngines()) {
             logger.info(&quot;Registering mbean for store '&quot; + store.getName() + &quot;'.&quot;);
             registerBean(store,
                          JmxUtils.createObjectName(JmxUtils.getPackageName(store.getClass()),</diff>
      <filename>src/java/voldemort/server/jmx/JmxService.java</filename>
    </modified>
    <modified>
      <diff>@@ -17,12 +17,12 @@
 package voldemort.server.scheduler;
 
 import java.util.List;
-import java.util.Map;
 
 import org.apache.log4j.Logger;
 
 import voldemort.cluster.Node;
 import voldemort.routing.RoutingStrategy;
+import voldemort.server.StoreRepository;
 import voldemort.store.StorageEngine;
 import voldemort.store.Store;
 import voldemort.utils.ByteArray;
@@ -44,27 +44,22 @@ public class RebalancingJob implements Runnable {
     private static Logger logger = Logger.getLogger(RebalancingJob.class);
 
     private final int localNodeId;
-    private final RoutingStrategy routingStrategy;
-    private final Map&lt;String, StorageEngine&lt;ByteArray, byte[]&gt;&gt; localEngines;
-    private final Map&lt;String, Store&lt;ByteArray, byte[]&gt;&gt; remoteStores;
+    private final RoutingStrategy router;
+    private final StoreRepository storeRepository;
 
-    public RebalancingJob(int localNodeId,
-                          RoutingStrategy routingStrategy,
-                          Map&lt;String, StorageEngine&lt;ByteArray, byte[]&gt;&gt; engines,
-                          Map&lt;String, Store&lt;ByteArray, byte[]&gt;&gt; remoteStores) {
+    public RebalancingJob(int localNodeId, RoutingStrategy router, StoreRepository storeRepository) {
         this.localNodeId = localNodeId;
-        this.localEngines = engines;
-        this.remoteStores = remoteStores;
-        this.routingStrategy = routingStrategy;
+        this.storeRepository = storeRepository;
+        this.router = router;
     }
 
     public void run() {
         logger.info(&quot;Rebalancing all keys...&quot;);
         int totalRebalanced = 0;
         long start = System.currentTimeMillis();
-        for(StorageEngine&lt;ByteArray, byte[]&gt; engine: localEngines.values()) {
+        for(StorageEngine&lt;ByteArray, byte[]&gt; engine: storeRepository.getAllStorageEngines()) {
             logger.info(&quot;Rebalancing &quot; + engine.getName());
-            Store&lt;ByteArray, byte[]&gt; remote = this.remoteStores.get(engine.getName());
+            Store&lt;ByteArray, byte[]&gt; remote = storeRepository.getRoutedStore(engine.getName());
             ClosableIterator&lt;Pair&lt;ByteArray, Versioned&lt;byte[]&gt;&gt;&gt; iterator = engine.entries();
             int rebalanced = 0;
             long currStart = System.currentTimeMillis();
@@ -88,7 +83,7 @@ public class RebalancingJob implements Runnable {
     }
 
     private boolean needsRebalancing(ByteArray key) {
-        List&lt;Node&gt; responsible = routingStrategy.routeRequest(key.get());
+        List&lt;Node&gt; responsible = router.routeRequest(key.get());
         for(Node n: responsible)
             if(n.getId() == localNodeId)
                 return false;</diff>
      <filename>src/java/voldemort/server/scheduler/RebalancingJob.java</filename>
    </modified>
    <modified>
      <diff>@@ -26,6 +26,7 @@ import org.apache.log4j.Logger;
 
 import voldemort.annotations.jmx.JmxManaged;
 import voldemort.server.AbstractService;
+import voldemort.server.ServiceType;
 import voldemort.utils.Time;
 
 /**
@@ -59,8 +60,8 @@ public class SchedulerService extends AbstractService {
     private final ScheduledThreadPoolExecutor scheduler;
     private final Time time;
 
-    public SchedulerService(String name, int schedulerThreads, Time time) {
-        super(name);
+    public SchedulerService(int schedulerThreads, Time time) {
+        super(ServiceType.SCHEDULER);
         this.time = time;
         this.scheduler = new ScheduledThreadPoolExecutor(schedulerThreads, schedulerThreadFactory);
     }</diff>
      <filename>src/java/voldemort/server/scheduler/SchedulerService.java</filename>
    </modified>
    <modified>
      <diff>@@ -16,13 +16,10 @@
 
 package voldemort.server.scheduler;
 
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
 import org.apache.log4j.Level;
 import org.apache.log4j.Logger;
 
+import voldemort.server.StoreRepository;
 import voldemort.store.StorageEngine;
 import voldemort.store.Store;
 import voldemort.store.slop.Slop;
@@ -44,13 +41,10 @@ public class SlopPusherJob implements Runnable {
 
     private static final Logger logger = Logger.getLogger(SlopPusherJob.class.getName());
 
-    private final StorageEngine&lt;ByteArray, Slop&gt; slopStore;
-    private final ConcurrentMap&lt;Integer, Store&lt;ByteArray, byte[]&gt;&gt; stores;
+    private final StoreRepository storeRepo;
 
-    public SlopPusherJob(StorageEngine&lt;ByteArray, Slop&gt; slop,
-                         Map&lt;Integer, ? extends Store&lt;ByteArray, byte[]&gt;&gt; stores) {
-        this.slopStore = slop;
-        this.stores = new ConcurrentHashMap&lt;Integer, Store&lt;ByteArray, byte[]&gt;&gt;(stores);
+    public SlopPusherJob(StoreRepository storeRepo) {
+        this.storeRepo = storeRepo;
     }
 
     /**
@@ -63,23 +57,27 @@ public class SlopPusherJob implements Runnable {
         int attemptedPushes = 0;
         ClosableIterator&lt;Pair&lt;ByteArray, Versioned&lt;Slop&gt;&gt;&gt; iterator = null;
         try {
+            StorageEngine&lt;ByteArray, Slop&gt; slopStore = storeRepo.getSlopStore();
             iterator = slopStore.entries();
             while(iterator.hasNext()) {
-                attemptedPushes++;
-                if(Thread.currentThread().isInterrupted())
+                if(Thread.interrupted())
                     throw new InterruptedException(&quot;Task cancelled!&quot;);
+                attemptedPushes++;
 
                 try {
                     Pair&lt;ByteArray, Versioned&lt;Slop&gt;&gt; keyAndVal = iterator.next();
                     Versioned&lt;Slop&gt; versioned = keyAndVal.getSecond();
                     Slop slop = versioned.getValue();
-                    Store&lt;ByteArray, byte[]&gt; store = stores.get(slop.getNodeId());
+                    Store&lt;ByteArray, byte[]&gt; store = storeRepo.getNodeStore(slop.getStoreName(),
+                                                                            slop.getNodeId());
                     try {
                         if(slop.getOperation() == Operation.PUT)
                             store.put(keyAndVal.getFirst(),
                                       new Versioned&lt;byte[]&gt;(slop.getValue(), versioned.getVersion()));
-                        else
+                        else if(slop.getOperation() == Operation.DELETE)
                             store.delete(keyAndVal.getFirst(), versioned.getVersion());
+                        else
+                            logger.error(&quot;Unknown slop operation: &quot; + slop.getOperation());
                         slopStore.delete(slop.makeKey(), versioned.getVersion());
                         slopsPushed++;
                     } catch(ObsoleteVersionException e) {
@@ -108,8 +106,4 @@ public class SlopPusherJob implements Runnable {
                            + slopsPushed + &quot; succeeded.&quot;);
     }
 
-    public void close() {
-        this.slopStore.close();
-    }
-
 }
\ No newline at end of file</diff>
      <filename>src/java/voldemort/server/scheduler/SlopPusherJob.java</filename>
    </modified>
    <modified>
      <diff>@@ -16,11 +16,6 @@
 
 package voldemort.server.socket;
 
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.EOFException;
 import java.io.IOException;
 import java.net.BindException;
 import java.net.InetSocketAddress;
@@ -28,9 +23,7 @@ import java.net.ServerSocket;
 import java.net.Socket;
 import java.net.SocketException;
 import java.util.Random;
-import java.util.concurrent.ConcurrentMap;
 import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
 import java.util.concurrent.RejectedExecutionHandler;
 import java.util.concurrent.SynchronousQueue;
 import java.util.concurrent.ThreadFactory;
@@ -40,8 +33,9 @@ import java.util.concurrent.TimeUnit;
 import org.apache.log4j.Logger;
 
 import voldemort.VoldemortException;
-import voldemort.store.Store;
-import voldemort.utils.ByteArray;
+import voldemort.annotations.jmx.JmxGetter;
+import voldemort.annotations.jmx.JmxManaged;
+import voldemort.server.protocol.RequestHandler;
 
 /**
  * A simple socket-based server for serving voldemort requests
@@ -49,19 +43,40 @@ import voldemort.utils.ByteArray;
  * @author jay
  * 
  */
+@JmxManaged
 public class SocketServer extends Thread {
 
-    private static final Logger logger = Logger.getLogger(SocketServer.class.getName());
+    static final Logger logger = Logger.getLogger(SocketServer.class.getName());
 
-    private final ExecutorService threadPool;
+    private final ThreadPoolExecutor threadPool;
     private final Random random = new Random();
     private final int port;
-    private final ConcurrentMap&lt;String, ? extends Store&lt;ByteArray, byte[]&gt;&gt; storeMap;
     private final ThreadGroup threadGroup;
     private final CountDownLatch isStarted = new CountDownLatch(1);
     private final int socketBufferSize;
+    private final RequestHandler requestHandler;
+    private final int maxThreads;
     private ServerSocket serverSocket = null;
 
+    public SocketServer(int port,
+                        int defaultThreads,
+                        int maxThreads,
+                        int socketBufferSize,
+                        RequestHandler requestHandler) {
+        this.port = port;
+        this.socketBufferSize = socketBufferSize;
+        this.threadGroup = new ThreadGroup(&quot;voldemort-socket-server&quot;);
+        this.requestHandler = requestHandler;
+        this.maxThreads = maxThreads;
+        this.threadPool = new ThreadPoolExecutor(defaultThreads,
+                                                 maxThreads,
+                                                 0,
+                                                 TimeUnit.MILLISECONDS,
+                                                 new SynchronousQueue&lt;Runnable&gt;(),
+                                                 threadFactory,
+                                                 rejectedExecutionHandler);
+    }
+
     private final ThreadFactory threadFactory = new ThreadFactory() {
 
         public Thread newThread(Runnable r) {
@@ -88,24 +103,6 @@ public class SocketServer extends Thread {
         }
     };
 
-    public SocketServer(ConcurrentMap&lt;String, ? extends Store&lt;ByteArray, byte[]&gt;&gt; storeMap,
-                        int port,
-                        int defaultThreads,
-                        int maxThreads,
-                        int socketBufferSize) {
-        this.port = port;
-        this.socketBufferSize = socketBufferSize;
-        this.threadGroup = new ThreadGroup(&quot;voldemort-socket-server&quot;);
-        this.storeMap = storeMap;
-        this.threadPool = new ThreadPoolExecutor(defaultThreads,
-                                                 maxThreads,
-                                                 1,
-                                                 TimeUnit.MILLISECONDS,
-                                                 new SynchronousQueue&lt;Runnable&gt;(),
-                                                 threadFactory,
-                                                 rejectedExecutionHandler);
-    }
-
     @Override
     public void run() {
         logger.info(&quot;Starting voldemort socket server on port &quot; + port + &quot;.&quot;);
@@ -117,7 +114,7 @@ public class SocketServer extends Thread {
             while(!isInterrupted() &amp;&amp; !serverSocket.isClosed()) {
                 final Socket socket = serverSocket.accept();
                 configureSocket(socket);
-                this.threadPool.execute(new SocketServerSession(socket));
+                this.threadPool.execute(new SocketServerSession(socket, requestHandler));
             }
         } catch(BindException e) {
             logger.error(&quot;Could not bind to port &quot; + port + &quot;.&quot;);
@@ -169,10 +166,26 @@ public class SocketServer extends Thread {
         }
     }
 
+    @JmxGetter(name = &quot;port&quot;, description = &quot;The port on which the server accepts connections.&quot;)
     public int getPort() {
         return this.port;
     }
 
+    @JmxGetter(name = &quot;maxThreads&quot;, description = &quot;The maximum number of threads that can be started on the server.&quot;)
+    public int getMaxThreads() {
+        return this.maxThreads;
+    }
+
+    @JmxGetter(name = &quot;currentThreads&quot;, description = &quot;The current number of utilized threads on the server.&quot;)
+    public int getCurrentThreads() {
+        return this.threadPool.getActiveCount();
+    }
+
+    @JmxGetter(name = &quot;remainingThreads&quot;, description = &quot;The number of additional threads that can be allocated before reaching the maximum.&quot;)
+    public int getRemainingThreads() {
+        return getMaxThreads() - getCurrentThreads();
+    }
+
     public void awaitStartupCompletion() {
         try {
             isStarted.await();
@@ -185,41 +198,4 @@ public class SocketServer extends Thread {
         return baseName + random.nextInt(1000000);
     }
 
-    private class SocketServerSession implements Runnable {
-
-        private final Socket socket;
-
-        public SocketServerSession(Socket socket) {
-            this.socket = socket;
-        }
-
-        public Socket getSocket() {
-            return socket;
-        }
-
-        public void run() {
-            try {
-                logger.info(&quot;Client &quot; + socket.getRemoteSocketAddress() + &quot; connected.&quot;);
-                StreamStoreRequestHandler handler = new StreamStoreRequestHandler(storeMap,
-                                                                                  new DataInputStream(new BufferedInputStream(socket.getInputStream(),
-                                                                                                                              1000)),
-                                                                                  new DataOutputStream(new BufferedOutputStream(socket.getOutputStream(),
-                                                                                                                                1000)));
-                while(!Thread.currentThread().isInterrupted()) {
-                    handler.handleRequest();
-                }
-            } catch(EOFException e) {
-                logger.info(&quot;Client &quot; + socket.getRemoteSocketAddress() + &quot; disconnected.&quot;);
-            } catch(IOException e) {
-                logger.error(e);
-            } finally {
-                try {
-                    socket.close();
-                } catch(Exception e) {
-                    logger.error(&quot;Error while closing socket&quot;, e);
-                }
-            }
-        }
-    }
-
 }</diff>
      <filename>src/java/voldemort/server/socket/SocketServer.java</filename>
    </modified>
    <modified>
      <diff>@@ -16,14 +16,12 @@
 
 package voldemort.server.socket;
 
-import java.util.concurrent.ConcurrentMap;
-
 import voldemort.annotations.jmx.JmxGetter;
 import voldemort.annotations.jmx.JmxManaged;
 import voldemort.server.AbstractService;
+import voldemort.server.ServiceType;
 import voldemort.server.VoldemortService;
-import voldemort.store.Store;
-import voldemort.utils.ByteArray;
+import voldemort.server.protocol.RequestHandler;
 
 /**
  * The VoldemortService that loads up the socket server
@@ -36,18 +34,17 @@ public class SocketService extends AbstractService implements VoldemortService {
 
     private final SocketServer server;
 
-    public SocketService(String name,
-                         ConcurrentMap&lt;String, ? extends Store&lt;ByteArray, byte[]&gt;&gt; storeMap,
+    public SocketService(RequestHandler requestHandler,
                          int port,
                          int coreConnections,
                          int maxConnections,
                          int socketBufferSize) {
-        super(name);
-        this.server = new SocketServer(storeMap,
-                                       port,
+        super(ServiceType.SOCKET);
+        this.server = new SocketServer(port,
                                        coreConnections,
                                        maxConnections,
-                                       socketBufferSize);
+                                       socketBufferSize,
+                                       requestHandler);
     }
 
     @Override</diff>
      <filename>src/java/voldemort/server/socket/SocketService.java</filename>
    </modified>
    <modified>
      <diff>@@ -16,32 +16,30 @@
 
 package voldemort.server.storage;
 
-import java.io.File;
 import java.util.Calendar;
 import java.util.Date;
 import java.util.GregorianCalendar;
-import java.util.HashSet;
+import java.util.HashMap;
 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.Semaphore;
 
-import javax.management.MBeanOperationInfo;
-
 import org.apache.log4j.Logger;
 
 import voldemort.VoldemortException;
-import voldemort.annotations.jmx.JmxGetter;
 import voldemort.annotations.jmx.JmxManaged;
-import voldemort.annotations.jmx.JmxOperation;
+import voldemort.client.ClientThreadPool;
+import voldemort.client.protocol.RequestFormatType;
 import voldemort.cluster.Cluster;
+import voldemort.cluster.Node;
 import voldemort.routing.ConsistentRoutingStrategy;
-import voldemort.routing.RoutingStrategy;
 import voldemort.serialization.ByteArraySerializer;
 import voldemort.serialization.SlopSerializer;
 import voldemort.server.AbstractService;
+import voldemort.server.ServiceType;
+import voldemort.server.StoreRepository;
 import voldemort.server.VoldemortConfig;
 import voldemort.server.scheduler.DataCleanupJob;
 import voldemort.server.scheduler.SchedulerService;
@@ -51,11 +49,11 @@ import voldemort.store.Store;
 import voldemort.store.StoreDefinition;
 import voldemort.store.logging.LoggingStore;
 import voldemort.store.metadata.MetadataStore;
-import voldemort.store.readonly.RandomAccessFileStorageConfiguration;
-import voldemort.store.readonly.RandomAccessFileStore;
-import voldemort.store.serialized.SerializingStore;
+import voldemort.store.routed.RoutedStore;
+import voldemort.store.serialized.SerializingStorageEngine;
 import voldemort.store.slop.Slop;
-import voldemort.store.slop.SlopDetectingStore;
+import voldemort.store.socket.SocketPool;
+import voldemort.store.socket.SocketStore;
 import voldemort.store.stats.StatTrackingStore;
 import voldemort.utils.ByteArray;
 import voldemort.utils.ConfigurationException;
@@ -75,96 +73,151 @@ public class StorageService extends AbstractService {
     private static final Logger logger = Logger.getLogger(StorageService.class.getName());
 
     private final VoldemortConfig voldemortConfig;
-    private final ConcurrentMap&lt;String, Store&lt;ByteArray, byte[]&gt;&gt; localStoreMap;
-    private final Map&lt;String, StorageEngine&lt;ByteArray, byte[]&gt;&gt; rawEngines;
-    private final ConcurrentMap&lt;String, StorageConfiguration&gt; storageConfigurations;
+    private final StoreRepository storeRepository;
     private final SchedulerService scheduler;
-    private final Map&lt;String, RandomAccessFileStore&gt; readOnlyStores;
-    private MetadataStore metadataStore;
-    private Store&lt;ByteArray, Slop&gt; slopStore;
+    private final MetadataStore metadataStore;
+    private final Semaphore cleanupPermits;
+    private final SocketPool socketPool;
+    private final ConcurrentMap&lt;String, StorageConfiguration&gt; storageConfigs;
+    private final ClientThreadPool clientThreadPool;
 
-    public StorageService(String name,
-                          ConcurrentMap&lt;String, Store&lt;ByteArray, byte[]&gt;&gt; storeMap,
+    public StorageService(StoreRepository storeRepository,
+                          MetadataStore metadataStore,
                           SchedulerService scheduler,
                           VoldemortConfig config) {
-        super(name);
+        super(ServiceType.STORAGE);
         this.voldemortConfig = config;
-        this.localStoreMap = storeMap;
-        this.rawEngines = new ConcurrentHashMap&lt;String, StorageEngine&lt;ByteArray, byte[]&gt;&gt;();
         this.scheduler = scheduler;
-        this.storageConfigurations = initStorageConfigurations(config);
-        this.metadataStore = new MetadataStore(new File(config.getMetadataDirectory()));
-        this.readOnlyStores = new ConcurrentHashMap&lt;String, RandomAccessFileStore&gt;();
+        this.storeRepository = storeRepository;
+        this.metadataStore = metadataStore;
+        this.cleanupPermits = new Semaphore(1);
+        this.storageConfigs = new ConcurrentHashMap&lt;String, StorageConfiguration&gt;();
+        this.clientThreadPool = new ClientThreadPool(config.getClientMaxThreads(),
+                                                     config.getClientThreadIdleMs(),
+                                                     config.getClientMaxQueuedRequests());
+        this.socketPool = new SocketPool(config.getClientMaxConnectionsPerNode(),
+                                         config.getClientMaxTotalConnections(),
+                                         config.getClientConnectionTimeoutMs(),
+                                         config.getSocketTimeoutMs(),
+                                         config.getSocketBufferSize());
     }
 
-    private ConcurrentMap&lt;String, StorageConfiguration&gt; initStorageConfigurations(VoldemortConfig config) {
-        ConcurrentMap&lt;String, StorageConfiguration&gt; configs = new ConcurrentHashMap&lt;String, StorageConfiguration&gt;();
-        for(String configClassName: config.getStorageConfigurations()) {
-            try {
-                Class&lt;?&gt; configClass = Utils.loadClass(configClassName);
-                StorageConfiguration configuration = (StorageConfiguration) Utils.callConstructor(configClass,
-                                                                                                  new Class&lt;?&gt;[] { VoldemortConfig.class },
-                                                                                                  new Object[] { config });
-                logger.info(&quot;Initializing &quot; + configuration.getType() + &quot; storage engine.&quot;);
-                configs.put(configuration.getType(), configuration);
-            } catch(IllegalStateException e) {
-                logger.error(&quot;Error loading storage configuration '&quot; + configClassName + &quot;'.&quot;, e);
-            }
+    private void initStorageConfig(String configClassName) {
+        try {
+            Class&lt;?&gt; configClass = Utils.loadClass(configClassName);
+            StorageConfiguration configuration = (StorageConfiguration) Utils.callConstructor(configClass,
+                                                                                              new Class&lt;?&gt;[] { VoldemortConfig.class },
+                                                                                              new Object[] { voldemortConfig });
+            logger.info(&quot;Initializing &quot; + configuration.getType() + &quot; storage engine.&quot;);
+            storageConfigs.put(configuration.getType(), configuration);
+        } catch(IllegalStateException e) {
+            logger.error(&quot;Error loading storage configuration '&quot; + configClassName + &quot;'.&quot;, e);
         }
 
-        if(configs.size() == 0)
+        if(storageConfigs.size() == 0)
             throw new ConfigurationException(&quot;No storage engine has been enabled!&quot;);
-
-        return configs;
     }
 
     @Override
     protected void startInner() {
-        this.localStoreMap.clear();
-        this.localStoreMap.put(MetadataStore.METADATA_STORE_NAME, metadataStore);
-        Store&lt;ByteArray, byte[]&gt; slopStorage = getStore(&quot;slop&quot;, voldemortConfig.getSlopStoreType());
-        this.slopStore = new SerializingStore&lt;ByteArray, Slop&gt;(slopStorage,
-                                                               new ByteArraySerializer(),
-                                                               new SlopSerializer());
-        Cluster cluster = this.metadataStore.getCluster();
+        /* Initialize storage configurations */
+        for(String configClassName: voldemortConfig.getStorageConfigurations())
+            initStorageConfig(configClassName);
+
+        /* Register metadata and slop stores */
+        registerEngine(metadataStore);
+        if(voldemortConfig.isSlopEnabled()) {
+            StorageEngine&lt;ByteArray, byte[]&gt; slopEngine = getStorageEngine(&quot;slop&quot;,
+                                                                           voldemortConfig.getSlopStoreType());
+            registerEngine(slopEngine);
+            storeRepository.setSlopStore(new SerializingStorageEngine&lt;ByteArray, Slop&gt;(slopEngine,
+                                                                                       new ByteArraySerializer(),
+                                                                                       new SlopSerializer()));
+        }
         List&lt;StoreDefinition&gt; storeDefs = this.metadataStore.getStores();
         logger.info(&quot;Initializing stores:&quot;);
-        Time time = new SystemTime();
         for(StoreDefinition def: storeDefs) {
-            if(!def.getName().equals(MetadataStore.METADATA_STORE_NAME)) {
-                logger.info(&quot;Opening store '&quot; + def.getName() + &quot;'.&quot;);
-                StorageEngine&lt;ByteArray, byte[]&gt; engine = getStore(def.getName(), def.getType());
-                rawEngines.put(engine.getName(), engine);
+            openStore(def);
+        }
+        logger.info(&quot;All stores initialized.&quot;);
+    }
+
+    public void openStore(StoreDefinition storeDef) {
+        logger.info(&quot;Opening store '&quot; + storeDef.getName() + &quot;' (&quot; + storeDef.getType() + &quot;).&quot;);
+        StorageEngine&lt;ByteArray, byte[]&gt; engine = getStorageEngine(storeDef.getName(),
+                                                                   storeDef.getType());
+        registerEngine(engine);
+
+        if(voldemortConfig.isServerRoutingEnabled())
+            registerNodeStores(storeDef, metadataStore.getCluster(), voldemortConfig.getNodeId());
+
+        if(storeDef.hasRetentionPeriod())
+            scheduleCleanupJob(storeDef, engine);
+    }
 
-                if(def.getType().equals(RandomAccessFileStorageConfiguration.TYPE_NAME))
-                    this.readOnlyStores.put(engine.getName(), (RandomAccessFileStore) engine);
+    /**
+     * Register the given engine with the storage repository
+     * 
+     * @param engine Register the storage engine
+     */
+    public void registerEngine(StorageEngine&lt;ByteArray, byte[]&gt; engine) {
+        Cluster cluster = this.metadataStore.getCluster();
+        storeRepository.addStorageEngine(engine);
+
+        /* Now add any store wrappers that are enabled */
+        Store&lt;ByteArray, byte[]&gt; store = engine;
+        if(voldemortConfig.isVerboseLoggingEnabled())
+            store = new LoggingStore&lt;ByteArray, byte[]&gt;(store,
+                                                        cluster.getName(),
+                                                        SystemTime.INSTANCE);
+        if(voldemortConfig.isStatTrackingEnabled())
+            store = new StatTrackingStore&lt;ByteArray, byte[]&gt;(store);
+        storeRepository.addLocalStore(store);
+    }
 
-                /* Now add any store wrappers that are enabled */
-                Store&lt;ByteArray, byte[]&gt; store = engine;
-                if(voldemortConfig.isSlopDetectionEnabled()) {
-                    RoutingStrategy routingStrategy = new ConsistentRoutingStrategy(cluster.getNodes(),
-                                                                                    def.getReplicationFactor());
-                    store = new SlopDetectingStore(store,
-                                                   this.slopStore,
-                                                   def.getReplicationFactor(),
-                                                   cluster.getNodeById(this.voldemortConfig.getNodeId()),
-                                                   routingStrategy);
-                }
-                if(voldemortConfig.isVerboseLoggingEnabled())
-                    store = new LoggingStore&lt;ByteArray, byte[]&gt;(store, cluster.getName(), time);
-                if(voldemortConfig.isStatTrackingEnabled())
-                    store = new StatTrackingStore&lt;ByteArray, byte[]&gt;(store);
-                this.localStoreMap.put(def.getName(), store);
+    public void registerNodeStores(StoreDefinition def, Cluster cluster, int localNode) {
+        Map&lt;Integer, Store&lt;ByteArray, byte[]&gt;&gt; nodeStores = new HashMap&lt;Integer, Store&lt;ByteArray, byte[]&gt;&gt;(cluster.getNumberOfNodes());
+        for(Node node: cluster.getNodes()) {
+            Store&lt;ByteArray, byte[]&gt; store;
+            if(node.getId() == localNode) {
+                store = this.storeRepository.getLocalStore(def.getName());
+            } else {
+                store = new SocketStore(def.getName(),
+                                        node.getHost(),
+                                        node.getSocketPort(),
+                                        socketPool,
+                                        RequestFormatType.VOLDEMORT,
+                                        false);
             }
+            this.storeRepository.addNodeStore(node.getId(), store);
+            nodeStores.put(node.getId(), store);
         }
-        logger.info(&quot;All stores initialized.&quot;);
 
-        scheduleCleanupJobs(storeDefs, rawEngines);
+        RoutedStore routedStore = new RoutedStore(def.getName(),
+                                                  nodeStores,
+                                                  new ConsistentRoutingStrategy(cluster.getNodes(),
+                                                                                def.getReplicationFactor()),
+                                                  def.getPreferredReads(),
+                                                  def.getRequiredReads(),
+                                                  def.getPreferredWrites(),
+                                                  def.getRequiredWrites(),
+                                                  true,
+                                                  this.clientThreadPool,
+                                                  voldemortConfig.getRoutingTimeoutMs(),
+                                                  voldemortConfig.getClientNodeBannageMs(),
+                                                  SystemTime.INSTANCE);
+        this.storeRepository.addRoutedStore(routedStore);
     }
 
-    private void scheduleCleanupJobs(List&lt;StoreDefinition&gt; storeDefs,
-                                     Map&lt;String, StorageEngine&lt;ByteArray, byte[]&gt;&gt; engines) {
-        // Schedule data retention cleanup jobs
+    /**
+     * Schedule a data retention cleanup job for the given store
+     * 
+     * @param storeDef The store definition
+     * @param engine The storage engine to do cleanup on
+     */
+    private void scheduleCleanupJob(StoreDefinition storeDef,
+                                    StorageEngine&lt;ByteArray, byte[]&gt; engine) {
+        // Schedule data retention cleanup job if applicable
         GregorianCalendar cal = new GregorianCalendar();
         cal.add(Calendar.DAY_OF_YEAR, 1);
         cal.set(Calendar.HOUR, 0);
@@ -174,103 +227,81 @@ public class StorageService extends AbstractService {
 
         // allow only one cleanup job at a time
         Date startTime = cal.getTime();
-        Semaphore cleanupPermits = new Semaphore(1);
-        for(StoreDefinition storeDef: storeDefs) {
-            if(storeDef.hasRetentionPeriod()) {
-                logger.info(&quot;Scheduling data retention cleanup job for store '&quot;
-                            + storeDef.getName() + &quot;' at &quot; + startTime + &quot;.&quot;);
-                StorageEngine&lt;ByteArray, byte[]&gt; engine = engines.get(storeDef.getName());
-                Runnable cleanupJob = new DataCleanupJob&lt;ByteArray, byte[]&gt;(engine,
-                                                                            cleanupPermits,
-                                                                            storeDef.getRetentionDays()
-                                                                                    * Time.MS_PER_DAY,
-                                                                            SystemTime.INSTANCE);
-                this.scheduler.schedule(cleanupJob, startTime, Time.MS_PER_DAY);
-            }
-        }
+        logger.info(&quot;Scheduling data retention cleanup job for store '&quot; + storeDef.getName()
+                    + &quot;' at &quot; + startTime + &quot;.&quot;);
+        Runnable cleanupJob = new DataCleanupJob&lt;ByteArray, byte[]&gt;(engine,
+                                                                    cleanupPermits,
+                                                                    storeDef.getRetentionDays()
+                                                                            * Time.MS_PER_DAY,
+                                                                    SystemTime.INSTANCE);
+        this.scheduler.schedule(cleanupJob, startTime, Time.MS_PER_DAY);
     }
 
-    private StorageEngine&lt;ByteArray, byte[]&gt; getStore(String name, String type) {
-        StorageConfiguration config = storageConfigurations.get(type);
+    private StorageEngine&lt;ByteArray, byte[]&gt; getStorageEngine(String name, String type) {
+        StorageConfiguration config = storageConfigs.get(type);
         if(config == null)
             throw new ConfigurationException(&quot;Attempt to open store &quot; + name + &quot; but &quot; + type
-                                             + &quot; storage engine has not been enabled.&quot;);
+                                             + &quot; storage engine of type &quot; + type
+                                             + &quot; has not been enabled.&quot;);
         return config.getStore(name);
     }
 
     @Override
     protected void stopInner() {
-        try {
-            if(metadataStore != null)
-                metadataStore.close();
-        } catch(VoldemortException e) {
-            logger.error(&quot;Error while closing metadata store:&quot;, e);
-        }
-        try {
-            if(slopStore != null)
-                slopStore.close();
-        } catch(VoldemortException e) {
-            logger.error(&quot;Error while closing metadata store:&quot;, e);
+        /*
+         * We may end up closing a given store more than once, but that is cool
+         * because close() is idempotent
+         */
+
+        Exception lastException = null;
+        logger.info(&quot;Closing all stores.&quot;);
+        /* This will also close the node stores including local stores */
+        for(Store&lt;ByteArray, byte[]&gt; store: this.storeRepository.getAllRoutedStores()) {
+            logger.info(&quot;Closing routed store for &quot; + store.getName());
+            try {
+                store.close();
+            } catch(Exception e) {
+                lastException = e;
+            }
         }
-        VoldemortException exception = null;
-        logger.info(&quot;Closing stores:&quot;);
-        for(Store&lt;ByteArray, byte[]&gt; s: this.localStoreMap.values()) {
+        logger.info(&quot;All stores closed.&quot;);
+
+        /* Close slop store if necessary */
+        if(this.storeRepository.hasSlopStore()) {
             try {
-                logger.info(&quot;Closing store '&quot; + s.getName() + &quot;'.&quot;);
-                s.close();
-            } catch(VoldemortException e) {
-                // in the event of a failure still attempt to close other stores
-                logger.error(e);
-                exception = e;
+                this.storeRepository.getSlopStore().close();
+            } catch(Exception e) {
+                lastException = e;
             }
         }
-        this.localStoreMap.clear();
 
-        logger.info(&quot;Closing storage configurations:&quot;);
-        for(StorageConfiguration config: storageConfigurations.values()) {
+        /* Close all storage configs */
+        logger.info(&quot;Closing storage configurations.&quot;);
+        for(StorageConfiguration config: storageConfigs.values()) {
+            logger.info(&quot;Closing &quot; + config.getType() + &quot; storage config.&quot;);
             try {
-                logger.info(&quot;Closing storage configuration for &quot; + config.getType());
                 config.close();
-            } catch(VoldemortException e) {
-                logger.error(&quot;Error when shutting down storage configuration: &quot;, e);
+            } catch(Exception e) {
+                lastException = e;
             }
         }
 
-        // propagate the first exception
-        if(exception != null)
-            throw exception;
-        logger.info(&quot;All stores closed.&quot;);
-    }
-
-    public ConcurrentMap&lt;String, Store&lt;ByteArray, byte[]&gt;&gt; getLocalStoreMap() {
-        return localStoreMap;
-    }
-
-    @JmxGetter(name = &quot;storeNames&quot;, description = &quot;Get the names of all open stores.&quot;)
-    public Set&lt;String&gt; getStoreNames() {
-        return new HashSet&lt;String&gt;(localStoreMap.keySet());
-    }
+        this.clientThreadPool.shutdownNow();
+        logger.info(&quot;Closed client threadpool.&quot;);
 
-    @JmxOperation(impact = MBeanOperationInfo.ACTION, description = &quot;Push all keys that do not belong to this store out to the correct store.&quot;)
-    public void rebalance() {
-    // this.scheduler.scheduleNow(new
-    // RebalancingJob(voldemortConfig.getNodeId(), this.rawEngines));
-    }
-
-    public StorageConfiguration getStorageConfiguration(String type) {
-        return storageConfigurations.get(type);
+        /* If there is an exception, throw it */
+        if(lastException instanceof VoldemortException)
+            throw (VoldemortException) lastException;
+        else if(lastException != null)
+            throw new VoldemortException(lastException);
     }
 
     public MetadataStore getMetadataStore() {
         return this.metadataStore;
     }
 
-    public Map&lt;String, RandomAccessFileStore&gt; getReadOnlyStores() {
-        return this.readOnlyStores;
-    }
-
-    public Store&lt;ByteArray, Slop&gt; getSlopStore() {
-        return this.slopStore;
+    public StoreRepository getStoreRepository() {
+        return this.storeRepository;
     }
 
 }</diff>
      <filename>src/java/voldemort/server/storage/StorageService.java</filename>
    </modified>
    <modified>
      <diff>@@ -127,12 +127,20 @@ public class StoreDefinition implements Serializable {
         return this.requiredReads;
     }
 
-    public Integer getPreferredWrites() {
-        return preferredWrites;
+    public boolean hasPreferredWrites() {
+        return preferredWrites != null;
     }
 
-    public Integer getPreferredReads() {
-        return preferredReads;
+    public int getPreferredWrites() {
+        return preferredWrites == null ? getRequiredWrites() : preferredWrites;
+    }
+
+    public int getPreferredReads() {
+        return preferredReads == null ? getRequiredReads() : preferredReads;
+    }
+
+    public boolean hasPreferredReads() {
+        return preferredReads != null;
     }
 
     public String getType() {</diff>
      <filename>src/java/voldemort/store/StoreDefinition.java</filename>
    </modified>
    <modified>
      <diff>@@ -21,6 +21,7 @@ import static voldemort.utils.Utils.assertNotNull;
 import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import org.apache.commons.codec.binary.Hex;
 import org.apache.log4j.Logger;
@@ -68,6 +69,7 @@ public class BdbStorageEngine implements StorageEngine&lt;ByteArray, byte[]&gt; {
     private final Database bdbDatabase;
     private final Environment environment;
     private final VersionedSerializer&lt;byte[]&gt; serializer;
+    private final AtomicBoolean isOpen;
 
     public BdbStorageEngine(String name, Environment environment, Database database) {
         assertNotNull(&quot;The store name cannot be null.&quot;, name);
@@ -77,6 +79,7 @@ public class BdbStorageEngine implements StorageEngine&lt;ByteArray, byte[]&gt; {
         this.bdbDatabase = database;
         this.environment = environment;
         this.serializer = new VersionedSerializer&lt;byte[]&gt;(new IdentitySerializer());
+        this.isOpen = new AtomicBoolean(true);
     }
 
     public String getName() {
@@ -297,7 +300,8 @@ public class BdbStorageEngine implements StorageEngine&lt;ByteArray, byte[]&gt; {
 
     public void close() throws PersistenceFailureException {
         try {
-            this.bdbDatabase.close();
+            if(this.isOpen.compareAndSet(true, false))
+                this.bdbDatabase.close();
         } catch(DatabaseException e) {
             throw new PersistenceFailureException(&quot;Shutdown failed.&quot;, e);
         }</diff>
      <filename>src/java/voldemort/store/bdb/BdbStorageEngine.java</filename>
    </modified>
    <modified>
      <diff>@@ -16,33 +16,27 @@
 
 package voldemort.store.http;
 
-import java.io.BufferedInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.DataInputStream;
-import java.io.EOFException;
+import java.io.DataOutputStream;
 import java.io.IOException;
-import java.io.InputStream;
 import java.net.HttpURLConnection;
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
-import org.apache.commons.codec.binary.Base64;
-import org.apache.commons.codec.binary.Hex;
 import org.apache.commons.httpclient.HttpClient;
 import org.apache.commons.httpclient.HttpException;
 import org.apache.commons.httpclient.methods.ByteArrayRequestEntity;
-import org.apache.commons.httpclient.methods.DeleteMethod;
-import org.apache.commons.httpclient.methods.GetMethod;
-import org.apache.commons.httpclient.methods.PutMethod;
+import org.apache.commons.httpclient.methods.PostMethod;
 
 import voldemort.VoldemortException;
+import voldemort.client.protocol.RequestFormat;
 import voldemort.store.NoSuchCapabilityException;
 import voldemort.store.Store;
 import voldemort.store.StoreCapabilityType;
 import voldemort.store.StoreUtils;
 import voldemort.store.UnreachableStoreException;
 import voldemort.utils.ByteArray;
-import voldemort.utils.ByteUtils;
 import voldemort.versioning.VectorClock;
 import voldemort.versioning.Version;
 import voldemort.versioning.Versioned;
@@ -55,122 +49,127 @@ import voldemort.versioning.Versioned;
  */
 public class HttpStore implements Store&lt;ByteArray, byte[]&gt; {
 
-    private static final Hex codec = new Hex();
-    private static final HttpResponseCodeErrorMapper httpResponseCodeErrorMapper = new HttpResponseCodeErrorMapper();
-    private static final String VERSION_EXTENSION = &quot;X-vldmt-version&quot;;
-
     private final String storeName;
-    private final String host;
-    private final int port;
     private final HttpClient httpClient;
-
-    public HttpStore(String storeName, String host, int port, HttpClient client) {
+    private final RequestFormat requestFormat;
+    private final boolean reroute;
+    private final String storeUrl;
+
+    public HttpStore(String storeName,
+                     String host,
+                     int port,
+                     HttpClient client,
+                     RequestFormat format,
+                     boolean reroute) {
         this.storeName = storeName;
-        this.host = host;
-        this.port = port;
         this.httpClient = client;
+        this.requestFormat = format;
+        this.reroute = reroute;
+        this.storeUrl = &quot;http://&quot; + host + &quot;:&quot; + port + &quot;/stores&quot;;
     }
 
     public boolean delete(ByteArray key, Version version) throws VoldemortException {
         StoreUtils.assertValidKey(key);
-        String url = getUrl(key);
-        DeleteMethod method = null;
+        PostMethod method = null;
         try {
-            method = new DeleteMethod(url);
-            VectorClock clock = (VectorClock) version;
-            method.setRequestHeader(VERSION_EXTENSION,
-                                    new String(Base64.encodeBase64(clock.toBytes())));
-            int response = httpClient.executeMethod(method);
-            if(response == HttpURLConnection.HTTP_NOT_FOUND)
-                return false;
-            if(response != HttpURLConnection.HTTP_OK)
-                httpResponseCodeErrorMapper.throwError(response, method.getStatusText());
-            return true;
-        } catch(HttpException e) {
-            throw new VoldemortException(e);
+            method = new PostMethod(this.storeUrl);
+            ByteArrayOutputStream outputBytes = new ByteArrayOutputStream();
+            requestFormat.writeDeleteRequest(new DataOutputStream(outputBytes),
+                                             storeName,
+                                             key,
+                                             (VectorClock) version,
+                                             reroute);
+            DataInputStream input = executeRequest(method, outputBytes);
+            return requestFormat.readDeleteResponse(input);
         } catch(IOException e) {
-            throw new UnreachableStoreException(&quot;Could not connect to &quot; + url + &quot; for &quot; + storeName,
-                                                e);
+            throw new UnreachableStoreException(&quot;Could not connect to &quot; + storeUrl + &quot; for &quot;
+                                                + storeName, e);
         } finally {
             if(method != null)
                 method.releaseConnection();
         }
     }
 
-    private BufferedInputStream getBufferedInputStream(InputStream inputStream) {
-        if(inputStream instanceof BufferedInputStream)
-            return (BufferedInputStream) inputStream;
-        return new BufferedInputStream(inputStream);
-    }
-
     public List&lt;Versioned&lt;byte[]&gt;&gt; get(ByteArray key) throws VoldemortException {
         StoreUtils.assertValidKey(key);
-        String url = getUrl(key);
-        GetMethod method = null;
+        PostMethod method = null;
         try {
-            method = new GetMethod(url);
-            int response = httpClient.executeMethod(method);
-            if(response != HttpURLConnection.HTTP_OK)
-                httpResponseCodeErrorMapper.throwError(response, method.getStatusText());
-            DataInputStream input = createDataInputStream(method);
-            List&lt;Versioned&lt;byte[]&gt;&gt; items = new ArrayList&lt;Versioned&lt;byte[]&gt;&gt;();
-            try {
-                while(true) {
-                    int size = input.readInt();
-                    byte[] bytes = new byte[size];
-                    ByteUtils.read(input, bytes);
-                    VectorClock clock = new VectorClock(bytes);
-                    byte[] data = ByteUtils.copy(bytes, clock.sizeInBytes(), bytes.length);
-                    items.add(new Versioned&lt;byte[]&gt;(data, clock));
-                }
-            } catch(EOFException e) {
-                return items;
-            } finally {
-                StoreUtils.close(input);
-            }
-        } catch(HttpException e) {
-            throw new VoldemortException(e);
+            method = new PostMethod(this.storeUrl);
+            ByteArrayOutputStream outputBytes = new ByteArrayOutputStream();
+            requestFormat.writeGetRequest(new DataOutputStream(outputBytes),
+                                          storeName,
+                                          key,
+                                          reroute);
+            DataInputStream input = executeRequest(method, outputBytes);
+            return requestFormat.readGetResponse(input);
         } catch(IOException e) {
-            throw new UnreachableStoreException(&quot;Could not connect to &quot; + url + &quot; for &quot; + storeName,
-                                                e);
+            throw new UnreachableStoreException(&quot;Could not connect to &quot; + storeUrl + &quot; for &quot;
+                                                + storeName, e);
         } finally {
             if(method != null)
                 method.releaseConnection();
         }
     }
 
-    private DataInputStream createDataInputStream(GetMethod method) throws IOException {
-        return new DataInputStream(getBufferedInputStream(method.getResponseBodyAsStream()));
-    }
-
     public Map&lt;ByteArray, List&lt;Versioned&lt;byte[]&gt;&gt;&gt; getAll(Iterable&lt;ByteArray&gt; keys)
             throws VoldemortException {
         StoreUtils.assertValidKeys(keys);
-        // TODO Consider retrieving the keys concurrently.
-        return StoreUtils.getAll(this, keys);
+        PostMethod method = null;
+        try {
+            method = new PostMethod(this.storeUrl);
+            ByteArrayOutputStream outputBytes = new ByteArrayOutputStream();
+            requestFormat.writeGetAllRequest(new DataOutputStream(outputBytes),
+                                             storeName,
+                                             keys,
+                                             reroute);
+            DataInputStream input = executeRequest(method, outputBytes);
+            return requestFormat.readGetAllResponse(input);
+        } catch(IOException e) {
+            throw new UnreachableStoreException(&quot;Could not connect to &quot; + storeUrl + &quot; for &quot;
+                                                + storeName, e);
+        } finally {
+            if(method != null)
+                method.releaseConnection();
+        }
     }
 
     public void put(ByteArray key, Versioned&lt;byte[]&gt; versioned) throws VoldemortException {
         StoreUtils.assertValidKey(key);
-        String url = getUrl(key);
-        PutMethod method = null;
+        PostMethod method = null;
+        try {
+            method = new PostMethod(this.storeUrl);
+            ByteArrayOutputStream outputBytes = new ByteArrayOutputStream();
+            requestFormat.writePutRequest(new DataOutputStream(outputBytes),
+                                          storeName,
+                                          key,
+                                          versioned.getValue(),
+                                          (VectorClock) versioned.getVersion(),
+                                          reroute);
+            DataInputStream input = executeRequest(method, outputBytes);
+            requestFormat.readPutResponse(input);
+        } catch(IOException e) {
+            throw new UnreachableStoreException(&quot;Could not connect to &quot; + storeUrl + &quot; for &quot;
+                                                + storeName, e);
+        } finally {
+            if(method != null)
+                method.releaseConnection();
+        }
+    }
+
+    private DataInputStream executeRequest(PostMethod method, ByteArrayOutputStream output) {
         try {
-            method = new PutMethod(url);
-            VectorClock clock = (VectorClock) versioned.getVersion();
-            method.setRequestHeader(VERSION_EXTENSION,
-                                    new String(Base64.encodeBase64(clock.toBytes()), &quot;UTF-8&quot;));
-            method.setRequestEntity(new ByteArrayRequestEntity(versioned.getValue()));
+            method.setRequestEntity(new ByteArrayRequestEntity(output.toByteArray()));
             int response = httpClient.executeMethod(method);
             if(response != HttpURLConnection.HTTP_OK)
-                httpResponseCodeErrorMapper.throwError(response, method.getStatusText());
+                throw new UnreachableStoreException(&quot;HTTP request to store &quot; + storeName
+                                                    + &quot; returned status code &quot; + response + &quot; &quot;
+                                                    + method.getStatusText());
+            return new DataInputStream(method.getResponseBodyAsStream());
         } catch(HttpException e) {
             throw new VoldemortException(e);
         } catch(IOException e) {
-            throw new UnreachableStoreException(&quot;Could not connect to &quot; + url + &quot; for &quot; + storeName,
-                                                e);
-        } finally {
-            if(method != null)
-                method.releaseConnection();
+            throw new UnreachableStoreException(&quot;Could not connect to &quot; + storeUrl + &quot; for &quot;
+                                                + storeName, e);
         }
     }
 
@@ -180,11 +179,6 @@ public class HttpStore implements Store&lt;ByteArray, byte[]&gt; {
         return storeName;
     }
 
-    private String getUrl(ByteArray key) throws VoldemortException {
-        return &quot;http://&quot; + host + &quot;:&quot; + port + &quot;/&quot; + getName() + &quot;/&quot;
-               + ByteUtils.getString(codec.encode(key.get()), &quot;UTF-8&quot;);
-    }
-
     public Object getCapability(StoreCapabilityType capability) {
         throw new NoSuchCapabilityException(capability, getName());
     }</diff>
      <filename>src/java/voldemort/store/http/HttpStore.java</filename>
    </modified>
    <modified>
      <diff>@@ -62,6 +62,10 @@ public class InMemoryStorageEngine&lt;K, V&gt; implements StorageEngine&lt;K, V&gt; {
 
     public void close() {}
 
+    public void deleteAll() {
+        this.map.clear();
+    }
+
     public boolean delete(K key) {
         return delete(key, null);
     }</diff>
      <filename>src/java/voldemort/store/memory/InMemoryStorageEngine.java</filename>
    </modified>
    <modified>
      <diff>@@ -18,14 +18,11 @@ package voldemort.store.metadata;
 
 import java.io.File;
 import java.io.IOException;
-import java.io.StringReader;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
-import org.apache.commons.io.FileUtils;
-
 import voldemort.VoldemortException;
 import voldemort.cluster.Cluster;
 import voldemort.store.NoSuchCapabilityException;
@@ -51,22 +48,32 @@ public class MetadataStore implements StorageEngine&lt;ByteArray, byte[]&gt; {
     public static final String CLUSTER_KEY = &quot;cluster.xml&quot;;
     public static final String STORES_KEY = &quot;stores.xml&quot;;
     public static final Set&lt;String&gt; KNOWN_KEYS = ImmutableSet.of(&quot;cluster.xml&quot;, &quot;stores.xml&quot;);
+    private static final ClusterMapper clusterMapper = new ClusterMapper();
+    private static final StoreDefinitionsMapper storeMapper = new StoreDefinitionsMapper();
+
+    private final Cluster cluster;
+    private final List&lt;StoreDefinition&gt; storeDefs;
+
+    public MetadataStore(Cluster cluster, List&lt;StoreDefinition&gt; defs) {
+        this.cluster = cluster;
+        this.storeDefs = defs;
+    }
 
-    private final File directory;
-    private final ClusterMapper clusterMapper;
-    private final StoreDefinitionsMapper storeMapper;
-
-    public MetadataStore(File directory) {
-        this.directory = directory;
-        this.storeMapper = new StoreDefinitionsMapper();
-        this.clusterMapper = new ClusterMapper();
-        if(this.directory.listFiles() == null)
-            throw new IllegalArgumentException(&quot;No configuration found in &quot;
-                                               + this.directory.getAbsolutePath() + &quot;.&quot;);
-        if(!this.directory.exists() &amp;&amp; this.directory.canRead())
-            throw new IllegalArgumentException(&quot;Metadata directory &quot;
-                                               + this.directory.getAbsolutePath()
+    public static MetadataStore readFromDirectory(File dir) {
+        if(!Utils.isReadableDir(dir))
+            throw new IllegalArgumentException(&quot;Metadata directory &quot; + dir.getAbsolutePath()
                                                + &quot; does not exist or can not be read.&quot;);
+        if(dir.listFiles() == null)
+            throw new IllegalArgumentException(&quot;No configuration found in &quot; + dir.getAbsolutePath()
+                                               + &quot;.&quot;);
+
+        try {
+            Cluster cluster = clusterMapper.readCluster(new File(dir, CLUSTER_KEY));
+            List&lt;StoreDefinition&gt; defs = storeMapper.readStoreList(new File(dir, STORES_KEY));
+            return new MetadataStore(cluster, defs);
+        } catch(IOException e) {
+            throw new VoldemortException(&quot;Error reading configuration.&quot;, e);
+        }
     }
 
     public String getName() {
@@ -94,18 +101,15 @@ public class MetadataStore implements StorageEngine&lt;ByteArray, byte[]&gt; {
     }
 
     public List&lt;Versioned&lt;byte[]&gt;&gt; get(ByteArray key) throws VoldemortException {
-        String keyStr = new String(key.get());
-        if(!KNOWN_KEYS.contains(keyStr))
-            throw new IllegalArgumentException(&quot;Unknown metadata key: &quot; + keyStr);
-        File file = new File(this.directory, keyStr);
-        if(!Utils.isReadableFile(file.getAbsolutePath()))
-            throw new VoldemortException(&quot;Attempt to read metadata failed: &quot;
-                                         + file.getAbsolutePath() + &quot; is not a readable file!&quot;);
-        try {
-            return Collections.singletonList(new Versioned&lt;byte[]&gt;(FileUtils.readFileToByteArray(file)));
-        } catch(IOException e) {
-            throw new VoldemortException(&quot;Error reading metadata value '&quot; + keyStr + &quot;': &quot;, e);
-        }
+        String keyStr = ByteUtils.getString(key.get(), &quot;UTF-8&quot;);
+        if(CLUSTER_KEY.equals(keyStr))
+            return Collections.singletonList(new Versioned&lt;byte[]&gt;(ByteUtils.getBytes(clusterMapper.writeCluster(cluster),
+                                                                                      &quot;UTF-8&quot;)));
+        else if(STORES_KEY.equals(keyStr))
+            return Collections.singletonList(new Versioned&lt;byte[]&gt;(ByteUtils.getBytes(storeMapper.writeStoreList(storeDefs),
+                                                                                      &quot;UTF-8&quot;)));
+        else
+            throw new VoldemortException(&quot;Unknown metadata key &quot; + keyStr);
     }
 
     public Map&lt;ByteArray, List&lt;Versioned&lt;byte[]&gt;&gt;&gt; getAll(Iterable&lt;ByteArray&gt; keys)
@@ -115,23 +119,11 @@ public class MetadataStore implements StorageEngine&lt;ByteArray, byte[]&gt; {
     }
 
     public Cluster getCluster() {
-        return clusterMapper.readCluster(createStringReader(CLUSTER_KEY));
-    }
-
-    private StringReader createStringReader(String keyName) {
-        return new StringReader(getSingleValue(get(new ByteArray(ByteUtils.getBytes(keyName,
-                                                                                    &quot;UTF-8&quot;)))));
+        return cluster;
     }
 
     public List&lt;StoreDefinition&gt; getStores() {
-        return storeMapper.readStoreList(createStringReader(STORES_KEY));
-    }
-
-    private String getSingleValue(List&lt;Versioned&lt;byte[]&gt;&gt; found) {
-        if(found.size() != 1)
-            throw new VoldemortException(&quot;Inconsistent metadata found: expected 1 version but found &quot;
-                                         + found.size());
-        return ByteUtils.getString(found.get(0).getValue(), &quot;UTF-8&quot;);
+        return storeDefs;
     }
 
     public ClosableIterator&lt;Pair&lt;ByteArray, Versioned&lt;byte[]&gt;&gt;&gt; entries() {</diff>
      <filename>src/java/voldemort/store/metadata/MetadataStore.java</filename>
    </modified>
    <modified>
      <diff>@@ -143,9 +143,9 @@ public class RoutedStore implements Store&lt;ByteArray, byte[]&gt; {
         if(preferredWrites &lt; requiredWrites)
             throw new IllegalArgumentException(&quot;preferredWrites must be greater or equal to requiredWrites.&quot;);
         if(preferredReads &gt; innerStores.size())
-            throw new IllegalArgumentException(&quot;preferredReads is larger than the total number of stores!&quot;);
+            throw new IllegalArgumentException(&quot;preferredReads is larger than the total number of nodes!&quot;);
         if(preferredWrites &gt; innerStores.size())
-            throw new IllegalArgumentException(&quot;preferredWrites is larger than the total number of stores!&quot;);
+            throw new IllegalArgumentException(&quot;preferredWrites is larger than the total number of nodes!&quot;);
 
         this.name = name;
         this.innerStores = new ConcurrentHashMap&lt;Integer, Store&lt;ByteArray, byte[]&gt;&gt;(innerStores);</diff>
      <filename>src/java/voldemort/store/routed/RoutedStore.java</filename>
    </modified>
    <modified>
      <diff>@@ -16,53 +16,98 @@
 
 package voldemort.store.socket;
 
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
 import org.apache.commons.pool.KeyedObjectPool;
 import org.apache.commons.pool.impl.GenericKeyedObjectPool;
 import org.apache.log4j.Logger;
 
 import voldemort.VoldemortException;
+import voldemort.annotations.jmx.JmxGetter;
+import voldemort.annotations.jmx.JmxManaged;
 import voldemort.store.UnreachableStoreException;
+import voldemort.utils.Time;
 
 /**
  * A pool of sockets keyed off the socket destination. This wrapper just
- * translates exceptions and delegates to commons pool
+ * translates exceptions and delegates to apache commons pool as well as
+ * providing some JMX access.
  * 
  * @author jay
  * 
  */
+@JmxManaged(description = &quot;Voldemort socket pool.&quot;)
 public class SocketPool {
 
     private static final Logger logger = Logger.getLogger(SocketPool.class);
+    private static int WAIT_MONITORING_INTERVAL = 10000;
 
+    private final AtomicInteger checkouts;
+    private final AtomicLong waitNs;
+    private final AtomicLong avgWaitNs;
     private final KeyedObjectPool pool;
     private final SocketPoolableObjectFactory objFactory;
 
     public SocketPool(int maxConnectionsPerNode,
                       int maxTotalConnections,
-                      int timeoutMs,
+                      int connectionTimeoutMs,
+                      int soTimeoutMs,
                       int socketBufferSize) {
         GenericKeyedObjectPool.Config config = new GenericKeyedObjectPool.Config();
         config.maxActive = maxConnectionsPerNode;
         config.maxTotal = maxTotalConnections;
         config.maxIdle = maxTotalConnections;
-        config.maxWait = timeoutMs;
+        config.maxWait = connectionTimeoutMs;
         config.testOnBorrow = true;
         config.testOnReturn = true;
-        config.minEvictableIdleTimeMillis = 200 * 1000;
         config.whenExhaustedAction = GenericKeyedObjectPool.WHEN_EXHAUSTED_BLOCK;
-        this.objFactory = new SocketPoolableObjectFactory(timeoutMs, socketBufferSize);
+        this.objFactory = new SocketPoolableObjectFactory(soTimeoutMs, socketBufferSize);
         this.pool = new GenericKeyedObjectPool(objFactory, config);
+        this.checkouts = new AtomicInteger(0);
+        this.waitNs = new AtomicLong(0);
+        this.avgWaitNs = new AtomicLong(0);
     }
 
+    /**
+     * Checkout a socket from the pool
+     * 
+     * @param destination The socket destination you want to connect to
+     * @return The socket
+     */
     public SocketAndStreams checkout(SocketDestination destination) {
         try {
-            return (SocketAndStreams) pool.borrowObject(destination);
+            // time checkout
+            long start = System.nanoTime();
+            SocketAndStreams sas = (SocketAndStreams) pool.borrowObject(destination);
+            updateStats(System.nanoTime() - start);
+
+            return sas;
         } catch(Exception e) {
             throw new UnreachableStoreException(&quot;Failure while checking out socket for &quot;
                                                 + destination + &quot;: &quot;, e);
         }
     }
 
+    private void updateStats(long checkoutTimeNs) {
+        long wait = waitNs.getAndAdd(checkoutTimeNs);
+        int count = checkouts.getAndIncrement();
+
+        // reset reporting inverval if we have used up the current interval
+        if(count % WAIT_MONITORING_INTERVAL == WAIT_MONITORING_INTERVAL - 1) {
+            // harmless race condition:
+            waitNs.set(0);
+            checkouts.set(0);
+            avgWaitNs.set(wait / count);
+        }
+    }
+
+    /**
+     * Check the socket back into the pool.
+     * 
+     * @param destination The socket destination of the socket
+     * @param socket The socket to check back in
+     */
     public void checkin(SocketDestination destination, SocketAndStreams socket) {
         try {
             pool.returnObject(destination, socket);
@@ -77,6 +122,9 @@ public class SocketPool {
         }
     }
 
+    /**
+     * Close the socket pool
+     */
     public void close() {
         try {
             pool.clear();
@@ -86,12 +134,29 @@ public class SocketPool {
         }
     }
 
+    @JmxGetter(name = &quot;socketsCreated&quot;, description = &quot;The number of sockets created by this pool.&quot;)
     public int getNumberSocketsCreated() {
         return this.objFactory.getNumberCreated();
     }
 
+    @JmxGetter(name = &quot;socketsDestroyed&quot;, description = &quot;The number of sockets destroyed by this pool.&quot;)
     public int getNumberSocketsDestroyed() {
         return this.objFactory.getNumberDestroyed();
     }
 
+    @JmxGetter(name = &quot;numberOfActiveConnections&quot;, description = &quot;The number of active connections.&quot;)
+    public int getNumberOfActiveConnections() {
+        return this.pool.getNumActive();
+    }
+
+    @JmxGetter(name = &quot;numberOfIdleConnections&quot;, description = &quot;The number of active connections.&quot;)
+    public int getNumberOfIdleConnections() {
+        return this.pool.getNumIdle();
+    }
+
+    @JmxGetter(name = &quot;avgWaitTimeMs&quot;, description = &quot;The avg. ms of wait time to acquire a connection.&quot;)
+    public double getAvgWaitTimeMs() {
+        return this.avgWaitNs.doubleValue() / Time.NS_PER_MS;
+    }
+
 }</diff>
      <filename>src/java/voldemort/store/socket/SocketPool.java</filename>
    </modified>
    <modified>
      <diff>@@ -18,6 +18,7 @@ package voldemort.store.socket;
 
 import java.net.InetSocketAddress;
 import java.net.Socket;
+import java.net.SocketException;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.pool.KeyedPoolableObjectFactory;
@@ -33,13 +34,13 @@ public class SocketPoolableObjectFactory implements KeyedPoolableObjectFactory {
 
     public static final Logger logger = Logger.getLogger(SocketPoolableObjectFactory.class);
 
-    private final int timeoutMs;
+    private final int soTimeoutMs;
     private final int socketBufferSize;
     public final AtomicInteger created;
     public final AtomicInteger destroyed;
 
-    public SocketPoolableObjectFactory(int timeoutMs, int socketBufferSize) {
-        this.timeoutMs = timeoutMs;
+    public SocketPoolableObjectFactory(int soTimeoutMs, int socketBufferSize) {
+        this.soTimeoutMs = soTimeoutMs;
         this.created = new AtomicInteger(0);
         this.destroyed = new AtomicInteger(0);
         this.socketBufferSize = socketBufferSize;
@@ -53,6 +54,9 @@ public class SocketPoolableObjectFactory implements KeyedPoolableObjectFactory {
     // nothing to see here
     }
 
+    /**
+     * Close the socket
+     */
     public void destroyObject(Object key, Object value) throws Exception {
         SocketDestination dest = (SocketDestination) key;
         SocketAndStreams sands = (SocketAndStreams) value;
@@ -63,42 +67,51 @@ public class SocketPoolableObjectFactory implements KeyedPoolableObjectFactory {
                          + &quot;:&quot; + dest.getPort());
     }
 
+    /**
+     * Create a socket for the given host/port
+     */
     public Object makeObject(Object key) throws Exception {
         SocketDestination dest = (SocketDestination) key;
         Socket socket = new Socket();
         socket.setReceiveBufferSize(this.socketBufferSize);
         socket.setSendBufferSize(this.socketBufferSize);
         socket.setTcpNoDelay(true);
-        socket.setSoTimeout(timeoutMs);
+        socket.setSoTimeout(soTimeoutMs);
         socket.connect(new InetSocketAddress(dest.getHost(), dest.getPort()));
 
-        // check buffer sizes
-        if(socket.getReceiveBufferSize() != this.socketBufferSize)
-            logger.debug(&quot;Requested socket receive buffer size was &quot; + this.socketBufferSize
-                         + &quot; bytes but actual size is &quot; + socket.getReceiveBufferSize() + &quot; bytes.&quot;);
-        if(socket.getSendBufferSize() != this.socketBufferSize)
-            logger.debug(&quot;Requested socket send buffer size was &quot; + this.socketBufferSize
-                         + &quot; bytes but actual size is &quot; + socket.getSendBufferSize() + &quot; bytes.&quot;);
+        recordSocketCreation(dest, socket);
+
+        return new SocketAndStreams(socket);
+    }
 
+    /* Log relevant socket creation details */
+    private void recordSocketCreation(SocketDestination dest, Socket socket) throws SocketException {
         int numCreated = created.incrementAndGet();
-        if(logger.isDebugEnabled())
-            logger.debug(&quot;Created socket &quot; + numCreated + &quot; for &quot; + dest.getHost() + &quot;:&quot;
-                         + dest.getPort());
+        logger.debug(&quot;Created socket &quot; + numCreated + &quot; for &quot; + dest.getHost() + &quot;:&quot;
+                     + dest.getPort());
 
-        return new SocketAndStreams(socket);
+        // check buffer sizes--you often don't get out what you put in!
+        int sendBufferSize = socket.getSendBufferSize();
+        int receiveBufferSize = socket.getReceiveBufferSize();
+        if(receiveBufferSize != this.socketBufferSize)
+            logger.debug(&quot;Requested socket receive buffer size was &quot; + this.socketBufferSize
+                         + &quot; bytes but actual size is &quot; + receiveBufferSize + &quot; bytes.&quot;);
+        if(sendBufferSize != this.socketBufferSize)
+            logger.debug(&quot;Requested socket send buffer size was &quot; + this.socketBufferSize
+                         + &quot; bytes but actual size is &quot; + sendBufferSize + &quot; bytes.&quot;);
     }
 
     public boolean validateObject(Object key, Object value) {
         SocketAndStreams sands = (SocketAndStreams) value;
         Socket s = sands.getSocket();
         boolean isValid = !s.isClosed() &amp;&amp; s.isBound() &amp;&amp; s.isConnected();
-        if(!isValid)
+        if(!isValid &amp;&amp; logger.isDebugEnabled())
             logger.debug(&quot;Socket connection &quot; + sands + &quot; is no longer valid, closing.&quot;);
         return isValid;
     }
 
     public int getTimeout() {
-        return this.timeoutMs;
+        return this.soTimeoutMs;
     }
 
     public int getNumberCreated() {</diff>
      <filename>src/java/voldemort/store/socket/SocketPoolableObjectFactory.java</filename>
    </modified>
    <modified>
      <diff>@@ -16,26 +16,23 @@
 
 package voldemort.store.socket;
 
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
 import java.net.Socket;
-import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.log4j.Logger;
 
 import voldemort.VoldemortException;
-import voldemort.serialization.VoldemortOpCode;
-import voldemort.store.ErrorCodeMapper;
+import voldemort.client.protocol.RequestFormat;
+import voldemort.client.protocol.RequestFormatFactory;
+import voldemort.client.protocol.RequestFormatType;
 import voldemort.store.NoSuchCapabilityException;
 import voldemort.store.Store;
 import voldemort.store.StoreCapabilityType;
 import voldemort.store.StoreUtils;
 import voldemort.store.UnreachableStoreException;
 import voldemort.utils.ByteArray;
-import voldemort.utils.ByteUtils;
 import voldemort.utils.Utils;
 import voldemort.versioning.VectorClock;
 import voldemort.versioning.Version;
@@ -51,16 +48,26 @@ import voldemort.versioning.Versioned;
 public class SocketStore implements Store&lt;ByteArray, byte[]&gt; {
 
     private static final Logger logger = Logger.getLogger(SocketStore.class);
-    private final ErrorCodeMapper errorCodeMapper = new ErrorCodeMapper();
+
+    private final RequestFormatFactory requestFormatFactory = new RequestFormatFactory();
 
     private final String name;
     private final SocketPool pool;
     private final SocketDestination destination;
-
-    public SocketStore(String name, String host, int port, SocketPool socketPool) {
-        this.name = name;
-        this.pool = socketPool;
+    private final RequestFormat requestFormat;
+    private final boolean reroute;
+
+    public SocketStore(String name,
+                       String host,
+                       int port,
+                       SocketPool socketPool,
+                       RequestFormatType requestFormatType,
+                       boolean reroute) {
+        this.name = Utils.notNull(name);
+        this.pool = Utils.notNull(socketPool);
         this.destination = new SocketDestination(Utils.notNull(host), port);
+        this.requestFormat = requestFormatFactory.getRequestFormat(requestFormatType);
+        this.reroute = reroute;
     }
 
     public void close() throws VoldemortException {
@@ -71,18 +78,13 @@ public class SocketStore implements Store&lt;ByteArray, byte[]&gt; {
         StoreUtils.assertValidKey(key);
         SocketAndStreams sands = pool.checkout(destination);
         try {
-            DataOutputStream outputStream = sands.getOutputStream();
-            outputStream.writeByte(VoldemortOpCode.DELETE_OP_CODE);
-            outputStream.writeUTF(name);
-            outputStream.writeInt(key.length());
-            outputStream.write(key.get());
-            VectorClock clock = (VectorClock) version;
-            outputStream.writeShort(clock.sizeInBytes());
-            outputStream.write(clock.toBytes());
-            outputStream.flush();
-            DataInputStream inputStream = sands.getInputStream();
-            checkException(inputStream);
-            return inputStream.readBoolean();
+            requestFormat.writeDeleteRequest(sands.getOutputStream(),
+                                          name,
+                                          key,
+                                          (VectorClock) version,
+                                          reroute);
+            sands.getOutputStream().flush();
+            return requestFormat.readDeleteResponse(sands.getInputStream());
         } catch(IOException e) {
             close(sands.getSocket());
             throw new UnreachableStoreException(&quot;Failure in delete on &quot; + destination + &quot;: &quot;
@@ -95,35 +97,26 @@ public class SocketStore implements Store&lt;ByteArray, byte[]&gt; {
     public Map&lt;ByteArray, List&lt;Versioned&lt;byte[]&gt;&gt;&gt; getAll(Iterable&lt;ByteArray&gt; keys)
             throws VoldemortException {
         StoreUtils.assertValidKeys(keys);
-        // TODO We can optimise this, but wait for protobuf protocol before
-        // considering
-        return StoreUtils.getAll(this, keys);
+        SocketAndStreams sands = pool.checkout(destination);
+        try {
+            requestFormat.writeGetAllRequest(sands.getOutputStream(), name, keys, reroute);
+            sands.getOutputStream().flush();
+            return requestFormat.readGetAllResponse(sands.getInputStream());
+        } catch(IOException e) {
+            close(sands.getSocket());
+            throw new VoldemortException(e);
+        } finally {
+            pool.checkin(destination, sands);
+        }
     }
 
     public List&lt;Versioned&lt;byte[]&gt;&gt; get(ByteArray key) throws VoldemortException {
         StoreUtils.assertValidKey(key);
         SocketAndStreams sands = pool.checkout(destination);
         try {
-            DataOutputStream outputStream = sands.getOutputStream();
-            outputStream.writeByte(VoldemortOpCode.GET_OP_CODE);
-            outputStream.writeUTF(name);
-            outputStream.writeInt(key.length());
-            outputStream.write(key.get());
-            outputStream.flush();
-            DataInputStream inputStream = sands.getInputStream();
-            checkException(inputStream);
-            int resultSize = inputStream.readInt();
-            List&lt;Versioned&lt;byte[]&gt;&gt; results = new ArrayList&lt;Versioned&lt;byte[]&gt;&gt;(resultSize);
-            for(int i = 0; i &lt; resultSize; i++) {
-                int valueSize = inputStream.readInt();
-                byte[] bytes = new byte[valueSize];
-                ByteUtils.read(inputStream, bytes);
-                VectorClock clock = new VectorClock(bytes);
-                results.add(new Versioned&lt;byte[]&gt;(ByteUtils.copy(bytes,
-                                                                 clock.sizeInBytes(),
-                                                                 bytes.length), clock));
-            }
-            return results;
+            requestFormat.writeGetRequest(sands.getOutputStream(), name, key, reroute);
+            sands.getOutputStream().flush();
+            return requestFormat.readGetResponse(sands.getInputStream());
         } catch(IOException e) {
             close(sands.getSocket());
             throw new UnreachableStoreException(&quot;Failure in get on &quot; + destination + &quot;: &quot;
@@ -133,22 +126,18 @@ public class SocketStore implements Store&lt;ByteArray, byte[]&gt; {
         }
     }
 
-    public void put(ByteArray key, Versioned&lt;byte[]&gt; value) throws VoldemortException {
+    public void put(ByteArray key, Versioned&lt;byte[]&gt; versioned) throws VoldemortException {
         StoreUtils.assertValidKey(key);
         SocketAndStreams sands = pool.checkout(destination);
         try {
-            DataOutputStream outputStream = sands.getOutputStream();
-            outputStream.writeByte(VoldemortOpCode.PUT_OP_CODE);
-            outputStream.writeUTF(name);
-            outputStream.writeInt(key.length());
-            outputStream.write(key.get());
-            VectorClock clock = (VectorClock) value.getVersion();
-            outputStream.writeInt(value.getValue().length + clock.sizeInBytes());
-            outputStream.write(clock.toBytes());
-            outputStream.write(value.getValue());
-            outputStream.flush();
-            DataInputStream inputStream = sands.getInputStream();
-            checkException(inputStream);
+            requestFormat.writePutRequest(sands.getOutputStream(),
+                                       name,
+                                       key,
+                                       versioned.getValue(),
+                                       (VectorClock) versioned.getVersion(),
+                                       reroute);
+            sands.getOutputStream().flush();
+            requestFormat.readPutResponse(sands.getInputStream());
         } catch(IOException e) {
             close(sands.getSocket());
             throw new UnreachableStoreException(&quot;Failure in put on &quot; + destination + &quot;: &quot;
@@ -169,14 +158,6 @@ public class SocketStore implements Store&lt;ByteArray, byte[]&gt; {
         return name;
     }
 
-    private void checkException(DataInputStream inputStream) throws IOException {
-        short retCode = inputStream.readShort();
-        if(retCode != 0) {
-            String error = inputStream.readUTF();
-            throw errorCodeMapper.getError(retCode, error);
-        }
-    }
-
     private void close(Socket socket) {
         try {
             socket.close();</diff>
      <filename>src/java/voldemort/store/socket/SocketStore.java</filename>
    </modified>
    <modified>
      <diff>@@ -121,7 +121,7 @@ public class StatTrackingStore&lt;K, V&gt; extends DelegatingStore&lt;K, V&gt; {
         return stats.getAvgTimeInMs(Tracked.GET);
     }
 
-    @JmxGetter(name = &quot;averageGetCompletionTimeInMs&quot;, description = &quot;The avg. time in ms for GET calls to complete.&quot;)
+    @JmxGetter(name = &quot;averageGetCompletionTimeInMs&quot;, description = &quot;The avg. time in ms for GET_ALL calls to complete.&quot;)
     public double getAverageGetAllCompletionTimeInMs() {
         return stats.getAvgTimeInMs(Tracked.GET_ALL);
     }</diff>
      <filename>src/java/voldemort/store/stats/StatTrackingStore.java</filename>
    </modified>
    <modified>
      <diff>@@ -117,6 +117,26 @@ public class Props implements Map&lt;String, String&gt; {
         return props.put(key, value.toString());
     }
 
+    public Props with(String key, String value) {
+        put(key, value);
+        return this;
+    }
+
+    public Props with(String key, Integer value) {
+        put(key, value);
+        return this;
+    }
+
+    public Props with(String key, Double value) {
+        put(key, value);
+        return this;
+    }
+
+    public Props with(String key, Long value) {
+        put(key, value);
+        return this;
+    }
+
     public void putAll(Map&lt;? extends String, ? extends String&gt; m) {
         props.putAll(m);
     }</diff>
      <filename>src/java/voldemort/utils/Props.java</filename>
    </modified>
    <modified>
      <diff>@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.Comparator;
 import java.util.List;
 
 import com.google.common.base.Nullable;
@@ -408,4 +409,44 @@ public class Utils {
             throw new IllegalArgumentException(e.getCause());
     }
 
+    /**
+     * Return a copy of the list sorted according to the given comparator
+     * 
+     * @param &lt;T&gt; The type of the elements in the list
+     * @param l The list to sort
+     * @param comparator The comparator to use for sorting
+     * @return A sorted copy of the list
+     */
+    public static &lt;T&gt; List&lt;T&gt; sorted(List&lt;T&gt; l, Comparator&lt;T&gt; comparator) {
+        List&lt;T&gt; copy = new ArrayList&lt;T&gt;(l);
+        Collections.sort(copy, comparator);
+        return copy;
+    }
+
+    /**
+     * Return a copy of the list sorted according to the natural order
+     * 
+     * @param &lt;T&gt; The type of the elements in the list
+     * @param l The list to sort
+     * @return A sorted copy of the list
+     */
+    public static &lt;T extends Comparable&lt;T&gt;&gt; List&lt;T&gt; sorted(List&lt;T&gt; l) {
+        List&lt;T&gt; copy = new ArrayList&lt;T&gt;(l);
+        Collections.sort(copy);
+        return copy;
+    }
+
+    /**
+     * A reversed copy of the given list
+     * 
+     * @param &lt;T&gt; The type of the items in the list
+     * @param l The list to reverse
+     * @return The list, reversed
+     */
+    public static &lt;T&gt; List&lt;T&gt; reversed(List&lt;T&gt; l) {
+        List&lt;T&gt; copy = new ArrayList&lt;T&gt;(l);
+        Collections.reverse(copy);
+        return copy;
+    }
+
 }</diff>
      <filename>src/java/voldemort/utils/Utils.java</filename>
    </modified>
    <modified>
      <diff>@@ -342,4 +342,8 @@ public class VectorClock implements Version, Serializable {
         return this.timestamp;
     }
 
+    public List&lt;ClockEntry&gt; getEntries() {
+        return this.versions;
+    }
+
 }</diff>
      <filename>src/java/voldemort/versioning/VectorClock.java</filename>
    </modified>
    <modified>
      <diff>@@ -36,10 +36,6 @@ public final class Versioned&lt;T&gt; implements Serializable {
     private VectorClock version;
     private volatile T object;
 
-    public final static &lt;T&gt; Versioned&lt;T&gt; of(T object) {
-        return new Versioned&lt;T&gt;(object);
-    }
-
     public Versioned(T object) {
         this(object, new VectorClock());
     }
@@ -91,6 +87,14 @@ public final class Versioned&lt;T&gt; implements Serializable {
         return new Versioned&lt;T&gt;(this.getValue(), this.version.clone());
     }
 
+    public static &lt;S&gt; Versioned&lt;S&gt; value(S s) {
+        return new Versioned&lt;S&gt;(s, new VectorClock());
+    }
+
+    public static &lt;S&gt; Versioned&lt;S&gt; value(S s, Version v) {
+        return new Versioned&lt;S&gt;(s, v);
+    }
+
     public static final class HappenedBeforeComparator&lt;S&gt; implements Comparator&lt;Versioned&lt;S&gt;&gt; {
 
         public int compare(Versioned&lt;S&gt; v1, Versioned&lt;S&gt; v2) {</diff>
      <filename>src/java/voldemort/versioning/Versioned.java</filename>
    </modified>
    <modified>
      <diff>@@ -16,6 +16,8 @@
 
 package voldemort.xml;
 
+import java.io.File;
+import java.io.FileReader;
 import java.io.IOException;
 import java.io.Reader;
 import java.util.ArrayList;
@@ -74,6 +76,17 @@ public class ClusterMapper {
         }
     }
 
+    public Cluster readCluster(File f) throws IOException {
+        FileReader reader = null;
+        try {
+            reader = new FileReader(f);
+            return readCluster(reader);
+        } finally {
+            if(reader != null)
+                reader.close();
+        }
+    }
+
     @SuppressWarnings(&quot;unchecked&quot;)
     public Cluster readCluster(Reader input) {
         try {</diff>
      <filename>src/java/voldemort/xml/ClusterMapper.java</filename>
    </modified>
    <modified>
      <diff>@@ -16,6 +16,8 @@
 
 package voldemort.xml;
 
+import java.io.File;
+import java.io.FileReader;
 import java.io.IOException;
 import java.io.Reader;
 import java.util.ArrayList;
@@ -80,6 +82,17 @@ public class StoreDefinitionsMapper {
         }
     }
 
+    public List&lt;StoreDefinition&gt; readStoreList(File f) throws IOException {
+        FileReader reader = null;
+        try {
+            reader = new FileReader(f);
+            return readStoreList(reader);
+        } finally {
+            if(reader != null)
+                reader.close();
+        }
+    }
+
     @SuppressWarnings(&quot;unchecked&quot;)
     public List&lt;StoreDefinition&gt; readStoreList(Reader input) {
         try {
@@ -203,10 +216,10 @@ public class StoreDefinitionsMapper {
         store.addContent(new Element(STORE_ROUTING_TIER_ELMT).setText(storeDefinition.getRoutingPolicy()
                                                                                      .toDisplay()));
         store.addContent(new Element(STORE_REPLICATION_FACTOR_ELMT).setText(Integer.toString(storeDefinition.getReplicationFactor())));
-        if(storeDefinition.getPreferredReads() != null)
+        if(storeDefinition.hasPreferredReads())
             store.addContent(new Element(STORE_PREFERRED_READS_ELMT).setText(Integer.toString(storeDefinition.getPreferredReads())));
         store.addContent(new Element(STORE_REQUIRED_READS_ELMT).setText(Integer.toString(storeDefinition.getRequiredReads())));
-        if(storeDefinition.getPreferredWrites() != null)
+        if(storeDefinition.hasPreferredWrites())
             store.addContent(new Element(STORE_PREFERRED_WRITES_ELMT).setText(Integer.toString(storeDefinition.getPreferredWrites())));
         store.addContent(new Element(STORE_REQUIRED_WRITES_ELMT).setText(Integer.toString(storeDefinition.getRequiredWrites())));
 </diff>
      <filename>src/java/voldemort/xml/StoreDefinitionsMapper.java</filename>
    </modified>
    <modified>
      <diff>@@ -18,11 +18,10 @@ package voldemort;
 
 import java.io.File;
 import java.io.IOException;
+import java.io.StringReader;
 import java.net.ServerSocket;
 import java.util.ArrayList;
 import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
 
 import org.apache.commons.httpclient.HttpClient;
 import org.apache.commons.io.FileUtils;
@@ -30,19 +29,30 @@ import org.mortbay.jetty.Server;
 import org.mortbay.jetty.servlet.Context;
 import org.mortbay.jetty.servlet.ServletHolder;
 
+import voldemort.client.RoutingTier;
+import voldemort.client.protocol.RequestFormatFactory;
+import voldemort.client.protocol.RequestFormatType;
 import voldemort.cluster.Cluster;
 import voldemort.cluster.Node;
+import voldemort.serialization.SerializerDefinition;
+import voldemort.server.StoreRepository;
 import voldemort.server.VoldemortConfig;
 import voldemort.server.http.StoreServlet;
+import voldemort.server.protocol.RequestHandler;
+import voldemort.server.protocol.RequestHandlerFactory;
 import voldemort.server.socket.SocketServer;
 import voldemort.store.Store;
+import voldemort.store.StoreDefinition;
 import voldemort.store.http.HttpStore;
+import voldemort.store.memory.InMemoryStorageConfiguration;
 import voldemort.store.memory.InMemoryStorageEngine;
 import voldemort.store.metadata.MetadataStore;
 import voldemort.store.socket.SocketPool;
 import voldemort.store.socket.SocketStore;
 import voldemort.utils.ByteArray;
 import voldemort.utils.Props;
+import voldemort.xml.ClusterMapper;
+import voldemort.xml.StoreDefinitionsMapper;
 
 import com.google.common.collect.ImmutableList;
 
@@ -54,64 +64,70 @@ import com.google.common.collect.ImmutableList;
  */
 public class ServerTestUtils {
 
-    public static ConcurrentMap&lt;String, Store&lt;ByteArray, byte[]&gt;&gt; getStores(String storeName,
-                                                                            String clusterXml,
-                                                                            String storesXml) {
-        ConcurrentMap&lt;String, Store&lt;ByteArray, byte[]&gt;&gt; stores = new ConcurrentHashMap&lt;String, Store&lt;ByteArray, byte[]&gt;&gt;(1);
-        stores.put(storeName, new InMemoryStorageEngine&lt;ByteArray, byte[]&gt;(storeName));
-        // create metadata dir
-        File metadataDir = TestUtils.createTempDir();
-        try {
-            FileUtils.writeStringToFile(new File(metadataDir, &quot;cluster.xml&quot;), clusterXml);
-            FileUtils.writeStringToFile(new File(metadataDir, &quot;stores.xml&quot;), storesXml);
-            MetadataStore metadata = new MetadataStore(metadataDir);
-            stores.put(MetadataStore.METADATA_STORE_NAME, metadata);
-            return stores;
-        } catch(IOException e) {
-            throw new VoldemortException(&quot;Error creating metadata directory:&quot;, e);
-        }
+    public static StoreRepository getStores(String storeName, String clusterXml, String storesXml) {
+        StoreRepository repository = new StoreRepository();
+        Store&lt;ByteArray, byte[]&gt; store = new InMemoryStorageEngine&lt;ByteArray, byte[]&gt;(storeName);
+        repository.addLocalStore(store);
+        repository.addRoutedStore(store);
+        MetadataStore metadata = new MetadataStore(new ClusterMapper().readCluster(new StringReader(clusterXml)),
+                                                   new StoreDefinitionsMapper().readStoreList(new StringReader(storesXml)));
+        repository.addLocalStore(metadata);
+        return repository;
     }
 
     public static SocketServer getSocketServer(String clusterXml,
                                                String storesXml,
                                                String storeName,
-                                               int port) {
-
-        SocketServer socketServer = new SocketServer(getStores(storeName, clusterXml, storesXml),
-                                                     port,
+                                               int port,
+                                               RequestFormatType type) {
+        RequestHandlerFactory factory = new RequestHandlerFactory(getStores(storeName,
+                                                                            clusterXml,
+                                                                            storesXml));
+        SocketServer socketServer = new SocketServer(port,
                                                      5,
                                                      10,
-                                                     10000);
+                                                     10000,
+                                                     factory.getRequestHandler(type));
         socketServer.start();
         socketServer.awaitStartupCompletion();
         return socketServer;
     }
 
     public static SocketStore getSocketStore(String storeName, int port) {
-        SocketPool socketPool = new SocketPool(1, 2, 1000, 32 * 1024);
-        return new SocketStore(storeName, &quot;localhost&quot;, port, socketPool);
+        SocketPool socketPool = new SocketPool(1, 2, 10000, 1000, 32 * 1024);
+        return new SocketStore(storeName,
+                               &quot;localhost&quot;,
+                               port,
+                               socketPool,
+                               RequestFormatType.VOLDEMORT,
+                               false);
     }
 
     public static Context getJettyServer(String clusterXml,
                                          String storesXml,
                                          String storeName,
+                                         RequestFormatType requestFormat,
                                          int port) throws Exception {
-        ConcurrentMap&lt;String, Store&lt;ByteArray, byte[]&gt;&gt; stores = getStores(storeName,
-                                                                           clusterXml,
-                                                                           storesXml);
+        StoreRepository repository = getStores(storeName, clusterXml, storesXml);
 
         // initialize servlet
         Server server = new Server(port);
         server.setSendServerVersion(false);
         Context context = new Context(server, &quot;/&quot;, Context.NO_SESSIONS);
 
-        context.addServlet(new ServletHolder(new StoreServlet(stores)), &quot;/*&quot;);
+        RequestHandler handler = new RequestHandlerFactory(repository).getRequestHandler(requestFormat);
+        context.addServlet(new ServletHolder(new StoreServlet(handler)), &quot;/stores&quot;);
         server.start();
         return context;
     }
 
-    public static HttpStore getHttpStore(String storeName, int port) {
-        return new HttpStore(storeName, &quot;localhost&quot;, port, new HttpClient());
+    public static HttpStore getHttpStore(String storeName, RequestFormatType format, int port) {
+        return new HttpStore(storeName,
+                             &quot;localhost&quot;,
+                             port,
+                             new HttpClient(),
+                             new RequestFormatFactory().getRequestFormat(format),
+                             false);
     }
 
     /**
@@ -159,6 +175,24 @@ public class ServerTestUtils {
         return new Cluster(&quot;test-cluster&quot;, nodes);
     }
 
+    public static List&lt;StoreDefinition&gt; getStoreDefs(int numStores) {
+        List&lt;StoreDefinition&gt; defs = new ArrayList&lt;StoreDefinition&gt;();
+        SerializerDefinition serDef = new SerializerDefinition(&quot;string&quot;);
+        for(int i = 0; i &lt; numStores; i++)
+            defs.add(new StoreDefinition(&quot;test&quot; + i,
+                                         InMemoryStorageConfiguration.TYPE_NAME,
+                                         serDef,
+                                         serDef,
+                                         RoutingTier.SERVER,
+                                         2,
+                                         1,
+                                         1,
+                                         1,
+                                         1,
+                                         1));
+        return defs;
+    }
+
     public static VoldemortConfig createServerConfig(int nodeId,
                                                      String baseDir,
                                                      String clusterFile,</diff>
      <filename>test/common/voldemort/ServerTestUtils.java</filename>
    </modified>
    <modified>
      <diff>@@ -32,6 +32,7 @@ import java.util.Random;
 import java.util.SortedSet;
 import java.util.TreeSet;
 
+import junit.framework.AssertionFailedError;
 import voldemort.client.RoutingTier;
 import voldemort.cluster.Cluster;
 import voldemort.cluster.Node;
@@ -39,11 +40,14 @@ import voldemort.routing.ConsistentRoutingStrategy;
 import voldemort.routing.RoutingStrategy;
 import voldemort.serialization.SerializerDefinition;
 import voldemort.serialization.json.JsonReader;
+import voldemort.store.Store;
 import voldemort.store.StoreDefinition;
 import voldemort.store.readonly.JsonStoreBuilder;
 import voldemort.store.readonly.RandomAccessFileStorageConfiguration;
 import voldemort.utils.ByteArray;
+import voldemort.utils.Utils;
 import voldemort.versioning.VectorClock;
+import voldemort.versioning.Versioned;
 
 /**
  * Helper utilities for tests
@@ -147,6 +151,22 @@ public class TestUtils {
         return bytes;
     }
 
+    public static &lt;K, V&gt; void assertContains(Store&lt;K, V&gt; store, K key, V... values) {
+        List&lt;Versioned&lt;V&gt;&gt; found = store.get(key);
+        if(found.size() != values.length)
+            throw new AssertionFailedError(&quot;Expected to find &quot; + values.length
+                                           + &quot; values in store, but found only &quot; + found.size()
+                                           + &quot;.&quot;);
+        for(V v: values) {
+            boolean isFound = false;
+            for(Versioned&lt;V&gt; f: found)
+                if(Utils.deepEquals(f.getValue(), v))
+                    isFound = true;
+            if(!isFound)
+                throw new AssertionFailedError(&quot;Could not find value &quot; + v + &quot; in results.&quot;);
+        }
+    }
+
     /**
      * Return an array of length count containing random integers in the range
      * (0, max) generated off the test rng.</diff>
      <filename>test/common/voldemort/TestUtils.java</filename>
    </modified>
    <modified>
      <diff>@@ -16,6 +16,7 @@
 
 package voldemort;
 
+import voldemort.client.ClientConfig;
 import voldemort.client.DefaultStoreClient;
 import voldemort.client.SocketStoreClientFactory;
 import voldemort.client.StoreClientFactory;
@@ -37,14 +38,7 @@ public class TestClientShutdown {
         String storeName = args[0];
         String bootstrapUrl = args[1];
 
-        StoreClientFactory factory = new SocketStoreClientFactory(2,
-                                                                  2,
-                                                                  3,
-                                                                  10,
-                                                                  2000,
-                                                                  2000,
-                                                                  2000,
-                                                                  bootstrapUrl);
+        StoreClientFactory factory = new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrl));
         DefaultStoreClient&lt;Object, Object&gt; client = null;
         try {
             client = (DefaultStoreClient&lt;Object, Object&gt;) factory.getStoreClient(storeName);</diff>
      <filename>test/integration/voldemort/TestClientShutdown.java</filename>
    </modified>
    <modified>
      <diff>@@ -16,6 +16,7 @@
 
 package voldemort.performance;
 
+import voldemort.client.ClientConfig;
 import voldemort.client.HttpStoreClientFactory;
 import voldemort.client.StoreClient;
 import voldemort.server.VoldemortConfig;
@@ -35,18 +36,9 @@ public class RemoteHttpStoreLoadTest extends AbstractLoadTestHarness {
         serverA.start();
         serverB.start();
 
-        HttpStoreClientFactory factory = new HttpStoreClientFactory(5,
-                                                                    2000,
-                                                                    2000,
-                                                                    0,
-                                                                    2000,
-                                                                    2000,
-                                                                    10000,
-                                                                    10,
-                                                                    5,
-                                                                    serverA.getIdentityNode()
-                                                                           .getHttpUrl()
-                                                                           .toString());
+        HttpStoreClientFactory factory = new HttpStoreClientFactory(new ClientConfig().setBootstrapUrls(serverA.getIdentityNode()
+                                                                                                              .getHttpUrl()
+                                                                                                              .toString()));
         return factory.getStoreClient(&quot;users&quot;);
     }
 </diff>
      <filename>test/integration/voldemort/performance/RemoteHttpStoreLoadTest.java</filename>
    </modified>
    <modified>
      <diff>@@ -16,9 +16,6 @@
 
 package voldemort.performance;
 
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
 import org.apache.commons.httpclient.DefaultHttpMethodRetryHandler;
 import org.apache.commons.httpclient.HostConfiguration;
 import org.apache.commons.httpclient.HttpClient;
@@ -30,7 +27,11 @@ import org.apache.commons.httpclient.params.HttpConnectionManagerParams;
 import org.apache.commons.httpclient.params.HttpMethodParams;
 
 import voldemort.TestUtils;
+import voldemort.client.protocol.RequestFormatFactory;
+import voldemort.client.protocol.RequestFormatType;
+import voldemort.server.StoreRepository;
 import voldemort.server.http.HttpService;
+import voldemort.server.protocol.RequestHandlerFactory;
 import voldemort.server.socket.SocketServer;
 import voldemort.store.Store;
 import voldemort.store.http.HttpStore;
@@ -87,11 +88,21 @@ public class RemoteStoreComparisonTest {
 
         /*** Do Socket tests ***/
         String storeName = &quot;test&quot;;
-        ConcurrentMap&lt;String, Store&lt;ByteArray, byte[]&gt;&gt; stores = new ConcurrentHashMap&lt;String, Store&lt;ByteArray, byte[]&gt;&gt;(1);
-        stores.put(storeName, new InMemoryStorageEngine&lt;ByteArray, byte[]&gt;(storeName));
-        SocketPool socketPool = new SocketPool(10, 10, 1000, 32 * 1024);
-        final SocketStore socketStore = new SocketStore(storeName, &quot;localhost&quot;, 6666, socketPool);
-        SocketServer socketServer = new SocketServer(stores, 6666, 50, 50, 1000);
+        StoreRepository repository = new StoreRepository();
+        repository.addLocalStore(new InMemoryStorageEngine&lt;ByteArray, byte[]&gt;(storeName));
+        SocketPool socketPool = new SocketPool(10, 10, 1000, 1000, 32 * 1024);
+        final SocketStore socketStore = new SocketStore(storeName,
+                                                        &quot;localhost&quot;,
+                                                        6666,
+                                                        socketPool,
+                                                        RequestFormatType.VOLDEMORT,
+                                                        false);
+        RequestHandlerFactory factory = new RequestHandlerFactory(repository);
+        SocketServer socketServer = new SocketServer(6666,
+                                                     50,
+                                                     50,
+                                                     1000,
+                                                     factory.getRequestHandler(RequestFormatType.VOLDEMORT));
         socketServer.start();
         socketServer.awaitStartupCompletion();
 
@@ -133,8 +144,12 @@ public class RemoteStoreComparisonTest {
         socketServer.shutdown();
 
         /*** Do HTTP tests ***/
-        stores.put(storeName, new InMemoryStorageEngine&lt;ByteArray, byte[]&gt;(storeName));
-        HttpService httpService = new HttpService(storeName, null, numThreads, 8080);
+        repository.addLocalStore(new InMemoryStorageEngine&lt;ByteArray, byte[]&gt;(storeName));
+        HttpService httpService = new HttpService(null,
+                                                  repository,
+                                                  RequestFormatType.VOLDEMORT,
+                                                  numThreads,
+                                                  8080);
         httpService.start();
         HttpClient httpClient = new HttpClient(new MultiThreadedHttpConnectionManager());
         HttpClientParams clientParams = httpClient.getParams();
@@ -151,7 +166,12 @@ public class RemoteStoreComparisonTest {
         managerParams.setMaxTotalConnections(numThreads);
         managerParams.setStaleCheckingEnabled(false);
         managerParams.setMaxConnectionsPerHost(httpClient.getHostConfiguration(), numThreads);
-        final HttpStore httpStore = new HttpStore(&quot;test&quot;, &quot;localhost&quot;, 8080, httpClient);
+        final HttpStore httpStore = new HttpStore(&quot;test&quot;,
+                                                  &quot;localhost&quot;,
+                                                  8080,
+                                                  httpClient,
+                                                  new RequestFormatFactory().getRequestFormat(RequestFormatType.VOLDEMORT),
+                                                  false);
         Thread.sleep(400);
 
         PerformanceTest httpWriteTest = new PerformanceTest() {</diff>
      <filename>test/integration/voldemort/performance/RemoteStoreComparisonTest.java</filename>
    </modified>
    <modified>
      <diff>@@ -24,10 +24,10 @@ import java.util.concurrent.Executors;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import voldemort.TestUtils;
+import voldemort.client.ClientConfig;
 import voldemort.client.SocketStoreClientFactory;
 import voldemort.client.StoreClient;
 import voldemort.client.StoreClientFactory;
-import voldemort.serialization.DefaultSerializerFactory;
 import voldemort.versioning.Versioned;
 
 public class RemoteTest {
@@ -46,15 +46,8 @@ public class RemoteTest {
         if(args.length &gt; 4)
             ops = args[4];
 
-        StoreClientFactory factory = new SocketStoreClientFactory(Executors.newFixedThreadPool(20),
-                                                                  6,
-                                                                  200,
-                                                                  20000,
-                                                                  20000,
-                                                                  20000,
-                                                                  64000,
-                                                                  new DefaultSerializerFactory(),
-                                                                  url);
+        StoreClientFactory factory = new SocketStoreClientFactory(new ClientConfig().setMaxThreads(20)
+                                                                                    .setBootstrapUrls(url));
         final StoreClient&lt;String, String&gt; store = factory.getStoreClient(&quot;test&quot;);
 
         final String value = new String(TestUtils.randomBytes(valueSize));</diff>
      <filename>test/integration/voldemort/performance/RemoteTest.java</filename>
    </modified>
    <modified>
      <diff>@@ -68,14 +68,14 @@ public class DefaultStoreClientTest extends TestCase {
     }
 
     public void testPutVersioned() {
-        client.put(&quot;k&quot;, Versioned.of(&quot;v&quot;));
+        client.put(&quot;k&quot;, Versioned.value(&quot;v&quot;));
         Versioned&lt;String&gt; v = client.get(&quot;k&quot;);
         assertEquals(&quot;GET should return the version set by PUT.&quot;, &quot;v&quot;, v.getValue());
         VectorClock expected = new VectorClock();
         expected.incrementVersion(nodeId, time.getMilliseconds());
         assertEquals(&quot;The version should be incremented after a put.&quot;, expected, v.getVersion());
         try {
-            client.put(&quot;k&quot;, Versioned.of(&quot;v&quot;));
+            client.put(&quot;k&quot;, Versioned.value(&quot;v&quot;));
             fail(&quot;Put of obsolete version should throw exception.&quot;);
         } catch(ObsoleteVersionException e) {
             // this is good</diff>
      <filename>test/unit/voldemort/client/DefaultStoreClientTest.java</filename>
    </modified>
    <modified>
      <diff>@@ -20,6 +20,7 @@ import org.mortbay.jetty.Server;
 import org.mortbay.jetty.servlet.Context;
 
 import voldemort.ServerTestUtils;
+import voldemort.client.protocol.RequestFormatType;
 import voldemort.serialization.SerializerFactory;
 import voldemort.store.http.HttpStore;
 
@@ -40,9 +41,12 @@ public class HttpStoreClientFactoryTest extends AbstractStoreClientFactoryTest {
         context = ServerTestUtils.getJettyServer(getClusterXml(),
                                                  getStoreDefXml(),
                                                  getValidStoreName(),
+                                                 RequestFormatType.VOLDEMORT,
                                                  getLocalNode().getHttpPort());
         server = context.getServer();
-        httpStore = ServerTestUtils.getHttpStore(getValidStoreName(), getLocalNode().getHttpPort());
+        httpStore = ServerTestUtils.getHttpStore(getValidStoreName(),
+                                                 RequestFormatType.VOLDEMORT,
+                                                 getLocalNode().getHttpPort());
         url = getLocalNode().getHttpUrl().toString();
     }
 
@@ -55,23 +59,14 @@ public class HttpStoreClientFactoryTest extends AbstractStoreClientFactoryTest {
 
     @Override
     protected StoreClientFactory getFactory(String... bootstrapUrls) {
-        return new HttpStoreClientFactory(4, bootstrapUrls);
+        return new HttpStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrls));
     }
 
     @Override
     protected StoreClientFactory getFactoryWithSerializer(SerializerFactory factory,
                                                           String... bootstrapUrls) {
-        return new HttpStoreClientFactory(3,
-                                          1000,
-                                          1000,
-                                          0,
-                                          1000,
-                                          1000,
-                                          10000,
-                                          10,
-                                          10,
-                                          factory,
-                                          bootstrapUrls);
+        return new HttpStoreClientFactory(new ClientConfig().setSerializerFactory(factory)
+                                                            .setBootstrapUrls(bootstrapUrls));
     }
 
     @Override</diff>
      <filename>test/unit/voldemort/client/HttpStoreClientFactoryTest.java</filename>
    </modified>
    <modified>
      <diff>@@ -17,9 +17,9 @@
 package voldemort.client;
 
 import java.net.URISyntaxException;
-import java.util.concurrent.Executors;
 
 import voldemort.ServerTestUtils;
+import voldemort.client.protocol.RequestFormatType;
 import voldemort.serialization.SerializerFactory;
 import voldemort.server.socket.SocketServer;
 
@@ -41,7 +41,8 @@ public class SocketStoreClientFactoryTest extends AbstractStoreClientFactoryTest
         server = ServerTestUtils.getSocketServer(getClusterXml(),
                                                  getStoreDefXml(),
                                                  getValidStoreName(),
-                                                 getLocalNode().getSocketPort());
+                                                 getLocalNode().getSocketPort(),
+                                                 RequestFormatType.VOLDEMORT);
     }
 
     @Override
@@ -52,26 +53,14 @@ public class SocketStoreClientFactoryTest extends AbstractStoreClientFactoryTest
 
     @Override
     protected StoreClientFactory getFactory(String... bootstrapUrls) {
-        return new SocketStoreClientFactory(Executors.newCachedThreadPool(),
-                                            5,
-                                            10,
-                                            1000,
-                                            1000,
-                                            10000,
-                                            bootstrapUrls);
+        return new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrls));
     }
 
     @Override
     protected StoreClientFactory getFactoryWithSerializer(SerializerFactory factory,
                                                           String... bootstrapUrls) {
-        return new SocketStoreClientFactory(Executors.newCachedThreadPool(),
-                                            5,
-                                            10,
-                                            1000,
-                                            1000,
-                                            1000,
-                                            factory,
-                                            bootstrapUrls);
+        return new SocketStoreClientFactory(new ClientConfig().setBootstrapUrls(bootstrapUrls)
+                                                              .setSerializerFactory(factory));
     }
 
     @Override</diff>
      <filename>test/unit/voldemort/client/SocketStoreClientFactoryTest.java</filename>
    </modified>
    <modified>
      <diff>@@ -19,14 +19,12 @@ package voldemort.scheduled;
 import static voldemort.TestUtils.bytesEqual;
 
 import java.util.Date;
-import java.util.HashMap;
-import java.util.Map;
 
 import junit.framework.TestCase;
 import voldemort.TestUtils;
+import voldemort.server.StoreRepository;
 import voldemort.server.scheduler.SlopPusherJob;
-import voldemort.store.StorageEngine;
-import voldemort.store.Store;
+import voldemort.store.FailingStore;
 import voldemort.store.memory.InMemoryStorageEngine;
 import voldemort.store.slop.Slop;
 import voldemort.store.slop.Slop.Operation;
@@ -35,9 +33,11 @@ import voldemort.versioning.Versioned;
 
 public class SlopPusherTest extends TestCase {
 
-    private StorageEngine&lt;ByteArray, Slop&gt; slopStore;
-    private Map&lt;Integer, Store&lt;ByteArray, byte[]&gt;&gt; stores;
+    private final static String STORE_NAME = &quot;test&quot;;
+
     private SlopPusherJob pusher;
+    private StoreRepository repo;
+    private int failingNodeId;
 
     public SlopPusherTest(String name) {
         super(name);
@@ -45,48 +45,73 @@ public class SlopPusherTest extends TestCase {
 
     @Override
     protected void setUp() throws Exception {
-        slopStore = new InMemoryStorageEngine&lt;ByteArray, Slop&gt;(&quot;slop&quot;);
-        stores = new HashMap&lt;Integer, Store&lt;ByteArray, byte[]&gt;&gt;();
-        stores.put(0, new InMemoryStorageEngine&lt;ByteArray, byte[]&gt;(&quot;0&quot;));
-        stores.put(1, new InMemoryStorageEngine&lt;ByteArray, byte[]&gt;(&quot;1&quot;));
-        stores.put(2, new InMemoryStorageEngine&lt;ByteArray, byte[]&gt;(&quot;2&quot;));
-        pusher = new SlopPusherJob(slopStore, stores);
+        repo = new StoreRepository();
+        repo.setSlopStore(new InMemoryStorageEngine&lt;ByteArray, Slop&gt;(&quot;slop&quot;));
+        repo.addNodeStore(0, new InMemoryStorageEngine&lt;ByteArray, byte[]&gt;(STORE_NAME));
+        repo.addNodeStore(1, new InMemoryStorageEngine&lt;ByteArray, byte[]&gt;(STORE_NAME));
+        this.failingNodeId = 2;
+        repo.addNodeStore(failingNodeId, new FailingStore&lt;ByteArray, byte[]&gt;(STORE_NAME));
+        pusher = new SlopPusherJob(repo);
     }
 
-    private Slop randomSlop(String name, int nodeId) {
-        return new Slop(name,
-                        Operation.PUT,
-                        TestUtils.randomBytes(10),
-                        TestUtils.randomBytes(10),
-                        nodeId,
-                        new Date());
+    private Versioned&lt;Slop&gt; randomSlop(String name, int nodeId) {
+        return Versioned.value(new Slop(name,
+                                        Operation.PUT,
+                                        TestUtils.randomBytes(10),
+                                        TestUtils.randomBytes(10),
+                                        nodeId,
+                                        new Date()));
     }
 
-    private void testPush(Versioned&lt;Slop&gt;... slops) {
+    private void pushSlop(Versioned&lt;Slop&gt;... slops) {
         // put all the slop in the slop store
         for(Versioned&lt;Slop&gt; s: slops)
-            slopStore.put(s.getValue().makeKey(), s);
+            repo.getSlopStore().put(s.getValue().makeKey(), s);
 
         // run the pusher
         pusher.run();
+    }
 
-        // now all the slop should be gone and the various stores should have
+    private void checkPush(Versioned&lt;Slop&gt;[] delivered, Versioned&lt;Slop&gt;[] undelivered) {
+        // now all the delivered slop should be gone and the various stores
+        // should have
         // those items
-        for(Versioned&lt;Slop&gt; vs: slops) {
+        for(Versioned&lt;Slop&gt; vs: delivered) {
             // check that all the slops are in the stores
             // and no new slops have appeared
             // and the SloppyStore is now empty
             Slop slop = vs.getValue();
-            assertEquals(&quot;Slop remains.&quot;, 0, slopStore.get(slop.makeKey()).size());
-            assertTrue(bytesEqual(slop.getValue(), stores.get(slop.getNodeId())
-                                                         .get(slop.makeKey())
-                                                         .get(0)
-                                                         .getValue()));
+            assertEquals(&quot;Slop remains.&quot;, 0, repo.getSlopStore().get(slop.makeKey()).size());
+            assertTrue(bytesEqual(slop.getValue(), repo.getNodeStore(STORE_NAME, slop.getNodeId())
+                                                       .get(slop.makeKey())
+                                                       .get(0)
+                                                       .getValue()));
         }
+        // check that all undelivered slop is undelivered
+        for(Versioned&lt;Slop&gt; vs: undelivered) {
+            Slop slop = vs.getValue();
+            assertEquals(&quot;Slop is gone!&quot;, 1, repo.getSlopStore().get(slop.makeKey()).size());
+        }
+    }
+
+    public void testPushNoSlop() {
+        pusher.run();
+    }
+
+    @SuppressWarnings(&quot;unchecked&quot;)
+    public void testPushSomeSlop() {
+        Versioned&lt;Slop&gt;[] values = new Versioned[] { randomSlop(STORE_NAME, 0),
+                randomSlop(STORE_NAME, 1), randomSlop(STORE_NAME, 0) };
+        pushSlop(values);
+        checkPush(values, new Versioned[] {});
     }
 
     @SuppressWarnings(&quot;unchecked&quot;)
-    public void testPushSingleSlop() {
-        testPush(new Versioned&lt;Slop&gt;(randomSlop(&quot;0&quot;, 0)));
+    public void testSlopWithFailingStore() {
+        Versioned&lt;Slop&gt; good1 = randomSlop(STORE_NAME, 0);
+        Versioned&lt;Slop&gt; good2 = randomSlop(STORE_NAME, 1);
+        Versioned&lt;Slop&gt; bad = randomSlop(STORE_NAME, this.failingNodeId);
+        pushSlop(good1, bad, good2);
+        checkPush(new Versioned[] { good1, good2 }, new Versioned[] { bad });
     }
 }</diff>
      <filename>test/unit/voldemort/scheduled/SlopPusherTest.java</filename>
    </modified>
    <modified>
      <diff>@@ -46,11 +46,11 @@ public class ServiceTest extends TestCase {
         private RuntimeException stopException = null;
 
         public FakeService() {
-            super(&quot;do-nothing&quot;);
+            super(ServiceType.SCHEDULER);
         }
 
         public FakeService(RuntimeException startException, RuntimeException stopException) {
-            super(&quot;do-nothing&quot;);
+            super(ServiceType.SCHEDULER);
             this.startException = startException;
             this.stopException = stopException;
         }</diff>
      <filename>test/unit/voldemort/server/ServiceTest.java</filename>
    </modified>
    <modified>
      <diff>@@ -16,65 +16,54 @@
 
 package voldemort.server.socket;
 
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.atomic.AtomicInteger;
-
 import junit.framework.TestCase;
 import voldemort.ServerTestUtils;
-import voldemort.store.Store;
+import voldemort.server.StoreRepository;
+import voldemort.server.protocol.vold.VoldemortNativeRequestHandler;
+import voldemort.store.ErrorCodeMapper;
 import voldemort.store.socket.SocketAndStreams;
 import voldemort.store.socket.SocketDestination;
 import voldemort.store.socket.SocketPool;
-import voldemort.utils.ByteArray;
 
 /**
+ * Tests for the socket pooling
+ * 
  * @author jay
  * 
  */
 public class SocketPoolTest extends TestCase {
 
-    private int port1;
-    private int port2;
+    private int port;
     private int maxConnectionsPerNode = 3;
     private int maxTotalConnections = 2 * maxConnectionsPerNode + 1;
     private SocketPool pool;
     private SocketDestination dest1;
-    private SocketDestination dest2;
-    private SocketServer server1;
-    private SocketServer server2;
+    private SocketServer server;
 
     @Override
     public void setUp() {
-        int[] ports = ServerTestUtils.findFreePorts(2);
-        this.port1 = ports[0];
-        this.port2 = ports[1];
-        this.pool = new SocketPool(maxConnectionsPerNode, maxTotalConnections, 1000, 32 * 1024);
-        this.dest1 = new SocketDestination(&quot;localhost&quot;, port1);
-        this.dest2 = new SocketDestination(&quot;localhost&quot;, port2);
-        this.server1 = new SocketServer(new ConcurrentHashMap&lt;String, Store&lt;ByteArray, byte[]&gt;&gt;(),
-                                        port1,
-                                        maxTotalConnections,
-                                        maxTotalConnections + 3,
-                                        10000);
-        this.server2 = new SocketServer(new ConcurrentHashMap&lt;String, Store&lt;ByteArray, byte[]&gt;&gt;(),
-                                        port2,
-                                        maxTotalConnections,
-                                        maxTotalConnections + 3,
-                                        10000);
-        this.server1.start();
-        this.server1.awaitStartupCompletion();
-        this.server2.start();
-        this.server2.awaitStartupCompletion();
+        this.port = ServerTestUtils.findFreePort();
+        this.pool = new SocketPool(maxConnectionsPerNode,
+                                   maxTotalConnections,
+                                   1000,
+                                   1000,
+                                   32 * 1024);
+        this.dest1 = new SocketDestination(&quot;localhost&quot;, port);
+        VoldemortNativeRequestHandler requestHandler = new VoldemortNativeRequestHandler(new ErrorCodeMapper(),
+                                                                                         new StoreRepository());
+        this.server = new SocketServer(port,
+                                       maxTotalConnections,
+                                       maxTotalConnections + 3,
+                                       10000,
+                                       requestHandler);
+        this.server.start();
+        this.server.awaitStartupCompletion();
     }
 
     @Override
     public void tearDown() {
         this.pool.close();
-        this.server1.shutdown();
-        this.server2.shutdown();
+        this.server.shutdown();
     }
 
     public void testTwoCheckoutsGetTheSameSocket() throws Exception {
@@ -100,26 +89,4 @@ public class SocketPoolTest extends TestCase {
         assertTrue(sas1 != sas2);
     }
 
-    public void testNoChurn() throws Exception {
-        ExecutorService service = Executors.newFixedThreadPool(10);
-        int numRequests = 100;
-        final AtomicInteger curr = new AtomicInteger(0);
-        final CountDownLatch latch = new CountDownLatch(numRequests);
-        for(int i = 0; i &lt; numRequests; i++) {
-            service.execute(new Runnable() {
-
-                public void run() {
-                    SocketDestination dest = curr.getAndIncrement() % 2 == 0 ? dest1 : dest2;
-                    SocketAndStreams sas = pool.checkout(dest);
-                    pool.checkin(dest, sas);
-                    latch.countDown();
-                }
-            });
-        }
-        latch.await();
-        assertTrue(&quot;Created more sockets than expected (created = &quot;
-                           + pool.getNumberSocketsCreated() + &quot;, expected = &quot; + maxTotalConnections
-                           + &quot;.&quot;,
-                   maxTotalConnections &gt;= pool.getNumberSocketsCreated());
-    }
 }</diff>
      <filename>test/unit/voldemort/server/socket/SocketPoolTest.java</filename>
    </modified>
    <modified>
      <diff>@@ -272,6 +272,13 @@ public abstract class AbstractStoreTest&lt;K, V&gt; extends TestCase {
         assertEquals(0, result.size());
     }
 
+    public void testCloseIsIdempotent() {
+        Store&lt;K, V&gt; store = getStore();
+        store.close();
+        // second close is okay, should not throw an exception
+        store.close();
+    }
+
     protected void assertGetAllValues(V expectedValue, List&lt;Versioned&lt;V&gt;&gt; versioneds) {
         assertEquals(1, versioneds.size());
         valuesEqual(expectedValue, versioneds.get(0).getValue());</diff>
      <filename>test/unit/voldemort/store/AbstractStoreTest.java</filename>
    </modified>
    <modified>
      <diff>@@ -18,6 +18,11 @@ package voldemort.store.bdb;
 
 import java.io.File;
 import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
 
 import org.apache.commons.io.FileDeleteStrategy;
 
@@ -25,6 +30,8 @@ import voldemort.TestUtils;
 import voldemort.store.AbstractStorageEngineTest;
 import voldemort.store.StorageEngine;
 import voldemort.utils.ByteArray;
+import voldemort.utils.ClosableIterator;
+import voldemort.utils.Pair;
 import voldemort.versioning.VectorClock;
 import voldemort.versioning.Versioned;
 
@@ -36,6 +43,7 @@ import com.sleepycat.je.EnvironmentConfig;
 public class BdbStorageEngineTest extends AbstractStorageEngineTest {
 
     private Environment environment;
+    private EnvironmentConfig envConfig;
     private Database database;
     private File tempDir;
     private BdbStorageEngine store;
@@ -44,13 +52,12 @@ public class BdbStorageEngineTest extends AbstractStorageEngineTest {
     @Override
     protected void setUp() throws Exception {
         super.setUp();
-        EnvironmentConfig environmentConfig = new EnvironmentConfig();
-        environmentConfig = new EnvironmentConfig();
-        environmentConfig.setTxnNoSync(true);
-        environmentConfig.setAllowCreate(true);
-        environmentConfig.setTransactional(true);
+        this.envConfig = new EnvironmentConfig();
+        this.envConfig.setTxnNoSync(true);
+        this.envConfig.setAllowCreate(true);
+        this.envConfig.setTransactional(true);
         this.tempDir = TestUtils.createTempDir();
-        this.environment = new Environment(this.tempDir, environmentConfig);
+        this.environment = new Environment(this.tempDir, envConfig);
         this.databaseConfig = new DatabaseConfig();
         databaseConfig.setAllowCreate(true);
         databaseConfig.setTransactional(true);
@@ -63,7 +70,7 @@ public class BdbStorageEngineTest extends AbstractStorageEngineTest {
     protected void tearDown() throws Exception {
         super.tearDown();
         try {
-            database.close();
+            store.close();
             environment.close();
         } finally {
             FileDeleteStrategy.FORCE.delete(tempDir);
@@ -89,12 +96,13 @@ public class BdbStorageEngineTest extends AbstractStorageEngineTest {
     }
 
     public void testPersistence() throws Exception {
-        StorageEngine&lt;ByteArray, byte[]&gt; eng = getStorageEngine();
-        eng.put(new ByteArray(&quot;abc&quot;.getBytes()), new Versioned&lt;byte[]&gt;(&quot;cdef&quot;.getBytes()));
-        eng.close();
+        this.store.put(new ByteArray(&quot;abc&quot;.getBytes()), new Versioned&lt;byte[]&gt;(&quot;cdef&quot;.getBytes()));
+        this.store.close();
+        this.environment.close();
+        this.environment = new Environment(this.tempDir, envConfig);
         this.database = environment.openDatabase(null, &quot;test&quot;, databaseConfig);
-        eng = new BdbStorageEngine(&quot;test&quot;, this.environment, this.database);
-        List&lt;Versioned&lt;byte[]&gt;&gt; vals = eng.get(new ByteArray(&quot;abc&quot;.getBytes()));
+        this.store = new BdbStorageEngine(&quot;test&quot;, this.environment, this.database);
+        List&lt;Versioned&lt;byte[]&gt;&gt; vals = store.get(new ByteArray(&quot;abc&quot;.getBytes()));
         assertEquals(1, vals.size());
         TestUtils.bytesEqual(&quot;cdef&quot;.getBytes(), vals.get(0).getValue());
     }
@@ -126,4 +134,42 @@ public class BdbStorageEngineTest extends AbstractStorageEngineTest {
         fail(&quot;No exception thrown for null database.&quot;);
     }
 
+    public void testSimultaneousIterationAndModification() throws Exception {
+        // start a thread to do modifications
+        ExecutorService executor = Executors.newFixedThreadPool(2);
+        final Random rand = new Random();
+        final AtomicInteger count = new AtomicInteger(0);
+        executor.execute(new Runnable() {
+
+            public void run() {
+                while(!Thread.interrupted()) {
+                    byte[] bytes = Integer.toString(count.getAndIncrement()).getBytes();
+                    store.put(new ByteArray(bytes), Versioned.value(bytes));
+                    count.incrementAndGet();
+                }
+            }
+        });
+        executor.execute(new Runnable() {
+
+            public void run() {
+                while(!Thread.interrupted()) {
+                    byte[] bytes = Integer.toString(rand.nextInt(count.get())).getBytes();
+                    store.delete(new ByteArray(bytes), new VectorClock());
+                    count.incrementAndGet();
+                }
+            }
+        });
+
+        // wait a bit
+        while(count.get() &lt; 300)
+            continue;
+
+        // now simultaneously do iteration
+        ClosableIterator&lt;Pair&lt;ByteArray, Versioned&lt;byte[]&gt;&gt;&gt; iter = this.store.entries();
+        while(iter.hasNext())
+            iter.next();
+        iter.close();
+        executor.shutdownNow();
+        assertTrue(executor.awaitTermination(1, TimeUnit.SECONDS));
+    }
 }</diff>
      <filename>test/unit/voldemort/store/bdb/BdbStorageEngineTest.java</filename>
    </modified>
    <modified>
      <diff>@@ -22,6 +22,9 @@ import org.mortbay.jetty.servlet.Context;
 
 import voldemort.ServerTestUtils;
 import voldemort.VoldemortTestConstants;
+import voldemort.client.protocol.RequestFormat;
+import voldemort.client.protocol.RequestFormatFactory;
+import voldemort.client.protocol.RequestFormatType;
 import voldemort.cluster.Cluster;
 import voldemort.cluster.Node;
 import voldemort.store.AbstractByteArrayStoreTest;
@@ -33,6 +36,8 @@ import voldemort.versioning.Versioned;
 import voldemort.xml.ClusterMapper;
 
 /**
+ * Tests of HTTP store against the HTTP server
+ * 
  * @author jay
  * 
  */
@@ -50,14 +55,23 @@ public class HttpStoreTest extends AbstractByteArrayStoreTest {
         context = ServerTestUtils.getJettyServer(new ClusterMapper().writeCluster(cluster),
                                                  VoldemortTestConstants.getSimpleStoreDefinitionsXml(),
                                                  &quot;users&quot;,
+                                                 RequestFormatType.VOLDEMORT,
                                                  node.getHttpPort());
         server = context.getServer();
-        httpStore = ServerTestUtils.getHttpStore(&quot;users&quot;, node.getHttpPort());
+        httpStore = ServerTestUtils.getHttpStore(&quot;users&quot;,
+                                                 RequestFormatType.VOLDEMORT,
+                                                 node.getHttpPort());
     }
 
     public &lt;T extends Exception&gt; void testBadUrlOrPort(String url, int port, Class&lt;T&gt; expected) {
         ByteArray key = new ByteArray(&quot;test&quot;.getBytes());
-        HttpStore badUrlHttpStore = new HttpStore(&quot;test&quot;, url, port, new HttpClient());
+        RequestFormat requestFormat = new RequestFormatFactory().getRequestFormat(RequestFormatType.VOLDEMORT);
+        HttpStore badUrlHttpStore = new HttpStore(&quot;test&quot;,
+                                                  url,
+                                                  port,
+                                                  new HttpClient(),
+                                                  requestFormat,
+                                                  false);
         try {
             badUrlHttpStore.put(key, new Versioned&lt;byte[]&gt;(&quot;value&quot;.getBytes(), new VectorClock()));
         } catch(Exception e) {</diff>
      <filename>test/unit/voldemort/store/http/HttpStoreTest.java</filename>
    </modified>
    <modified>
      <diff>@@ -28,6 +28,7 @@ import org.apache.log4j.Logger;
 import voldemort.ServerTestUtils;
 import voldemort.TestUtils;
 import voldemort.VoldemortTestConstants;
+import voldemort.client.protocol.RequestFormatType;
 import voldemort.server.socket.SocketServer;
 import voldemort.store.AbstractByteArrayStoreTest;
 import voldemort.store.Store;
@@ -46,10 +47,12 @@ public class SocketStoreTest extends AbstractByteArrayStoreTest {
     protected void setUp() throws Exception {
         super.setUp();
         socketPort = ServerTestUtils.findFreePort();
+        /* TODO: parameterize to test all wire formats */
         socketServer = ServerTestUtils.getSocketServer(VoldemortTestConstants.getOneNodeClusterXml(),
                                                        VoldemortTestConstants.getSimpleStoreDefinitionsXml(),
                                                        &quot;test&quot;,
-                                                       socketPort);
+                                                       socketPort,
+                                                       RequestFormatType.VOLDEMORT);
         socketStore = ServerTestUtils.getSocketStore(&quot;test&quot;, socketPort);
     }
 </diff>
      <filename>test/unit/voldemort/store/socket/SocketStoreTest.java</filename>
    </modified>
  </modified>
  <removed type="array">
    <removed>
      <filename>src/java/voldemort/client/DaemonThreadFactory.java</filename>
    </removed>
    <removed>
      <filename>src/java/voldemort/server/socket/StreamStoreRequestHandler.java</filename>
    </removed>
    <removed>
      <filename>test/unit/voldemort/store/metadata/MetadataStoreTest.java</filename>
    </removed>
  </removed>
  <parents type="array">
    <parent>
      <id>5ba2b99ccf8a7bae212ed64ac71130313783a69b</id>
    </parent>
  </parents>
  <author>
    <name>Jay Kreps</name>
    <email>jay.kreps@gmail.com</email>
  </author>
  <url>http://github.com/voldemort/voldemort/commit/c49df1b09aa617de5f0180ae2da943313c4d7390</url>
  <id>c49df1b09aa617de5f0180ae2da943313c4d7390</id>
  <committed-date>2009-04-26T21:25:34-07:00</committed-date>
  <authored-date>2009-04-26T21:25:34-07:00</authored-date>
  <message>Add support for protocol buffers based network format. Add new client config to avoid crazy constructors on the client. Add alpha support for server-side routing (still needs support on the client, though).</message>
  <tree>c024d54092ae0ac59d4d0464d6b5c5fefb840c9c</tree>
  <committer>
    <name>Jay Kreps</name>
    <email>jay.kreps@gmail.com</email>
  </committer>
</commit>
