You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by no...@apache.org on 2017/06/16 04:36:02 UTC

lucene-solr:master: SOLR-10433: CollectionAdmin requests in SolrJ to support V2 calls

Repository: lucene-solr
Updated Branches:
  refs/heads/master a7245d5e7 -> f6f6f1132


SOLR-10433: CollectionAdmin requests in SolrJ to support V2 calls


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/f6f6f113
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/f6f6f113
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/f6f6f113

Branch: refs/heads/master
Commit: f6f6f113209a5766c837665a818a524d0613757e
Parents: a7245d5
Author: Noble Paul <no...@apache.org>
Authored: Fri Jun 16 14:05:51 2017 +0930
Committer: Noble Paul <no...@apache.org>
Committed: Fri Jun 16 14:05:51 2017 +0930

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   2 +
 .../src/java/org/apache/solr/api/ApiBag.java    |   9 +-
 .../solr/request/SolrQueryRequestBase.java      |  15 +-
 .../org/apache/solr/servlet/HttpSolrCall.java   |   9 +-
 .../apache/solr/cloud/TestCollectionAPI.java    |  30 +++-
 .../solr/handler/admin/TestCollectionAPIs.java  |  10 +-
 .../test/org/apache/solr/util/TestUtils.java    |  32 ++++-
 .../apache/solr/client/solrj/SolrRequest.java   |  18 +++
 .../solr/client/solrj/V2RequestSupport.java     |  30 ++++
 .../solr/client/solrj/impl/CloudSolrClient.java |   5 +
 .../solr/client/solrj/impl/HttpSolrClient.java  |   7 +-
 .../solrj/request/CollectionAdminRequest.java   |  10 +-
 .../solrj/request/CollectionApiMapping.java     |  15 ++
 .../client/solrj/request/V1toV2ApiMapper.java   | 144 +++++++++++++++++++
 .../solr/client/solrj/request/V2Request.java    |  47 ++++--
 .../solr/common/util/CommandOperation.java      |  48 ++++++-
 .../solr/common/util/ContentStreamBase.java     |   7 +-
 .../apache/solr/common/util/JavaBinCodec.java   |  12 +-
 .../solr/common/util/JsonSchemaValidator.java   |  28 +++-
 .../org/apache/solr/common/util/Template.java   |  66 +++++++++
 .../java/org/apache/solr/common/util/Utils.java |   9 ++
 .../solrj/request/TestV1toV2ApiMapper.java      |  55 +++++++
 22 files changed, 551 insertions(+), 57 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 3c27672..7c8f013 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -142,6 +142,8 @@ New Features
   This will correct counts (and other statistics) for those top buckets collected in the first
   phase. (yonik)
 
+* SOLR-10433: CollectionAdmin requests in SolrJ to support V2 calls (noble)
+
 
 Bug Fixes
 ----------------------

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/core/src/java/org/apache/solr/api/ApiBag.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/api/ApiBag.java b/solr/core/src/java/org/apache/solr/api/ApiBag.java
index 805c31e..b1ca9a9 100644
--- a/solr/core/src/java/org/apache/solr/api/ApiBag.java
+++ b/solr/core/src/java/org/apache/solr/api/ApiBag.java
@@ -35,6 +35,8 @@ import com.google.common.collect.ImmutableSet;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SpecProvider;
 import org.apache.solr.common.util.CommandOperation;
+import org.apache.solr.common.util.ContentStream;
+import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.common.util.ValidatingJsonMap;
 import org.apache.solr.core.PluginBag;
@@ -282,13 +284,14 @@ public class ApiBag {
     }
   }
 
-  public static List<CommandOperation> getCommandOperations(Reader reader, Map<String, JsonSchemaValidator> validators, boolean validate) {
+  public static List<CommandOperation> getCommandOperations(ContentStream stream, Map<String, JsonSchemaValidator> validators, boolean validate) {
     List<CommandOperation> parsedCommands = null;
     try {
-      parsedCommands = CommandOperation.parse(reader);
+      parsedCommands = CommandOperation.readCommands(Collections.singleton(stream), new NamedList());
     } catch (IOException e) {
-      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unable to parse commands");
     }
+
     if (validators == null || !validate) {    // no validation possible because we do not have a spec
       return parsedCommands;
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java b/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java
index 83e44dd..31b8605 100644
--- a/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java
+++ b/solr/core/src/java/org/apache/solr/request/SolrQueryRequestBase.java
@@ -203,8 +203,7 @@ public abstract class SolrQueryRequestBase implements SolrQueryRequest, Closeabl
       Iterable<ContentStream> contentStreams = getContentStreams();
       if (contentStreams == null) throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No content stream");
       for (ContentStream contentStream : contentStreams) {
-        parsedCommands = ApiBag.getCommandOperations(getInputStream(contentStream),
-            getValidators(), validateInput);
+        parsedCommands = ApiBag.getCommandOperations(contentStream, getValidators(), validateInput);
       }
 
     }
@@ -212,18 +211,6 @@ public abstract class SolrQueryRequestBase implements SolrQueryRequest, Closeabl
 
   }
 
-  private InputStreamReader getInputStream(ContentStream contentStream) {
-    if(contentStream instanceof InputStream) {
-      return new InputStreamReader((InputStream)contentStream, UTF_8);
-    } else {
-      try {
-        return new InputStreamReader(contentStream.getStream(), UTF_8);
-      } catch (IOException e) {
-        throw new RuntimeException(e);
-      }
-    }
-  }
-
   protected ValidatingJsonMap getSpec() {
     return null;
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
index 11b483a..bf6c553 100644
--- a/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
+++ b/solr/core/src/java/org/apache/solr/servlet/HttpSolrCall.java
@@ -1108,14 +1108,7 @@ public class HttpSolrCall {
       Iterable<ContentStream> contentStreams = solrReq.getContentStreams();
       if (contentStreams == null) parsedCommands = Collections.EMPTY_LIST;
       else {
-        for (ContentStream contentStream : contentStreams) {
-          try {
-            parsedCommands = ApiBag.getCommandOperations(contentStream.getReader(), getValidators(), validateInput);
-          } catch (IOException e) {
-            throw new SolrException(ErrorCode.BAD_REQUEST, "Error reading commands");
-          }
-          break;
-        }
+        parsedCommands = ApiBag.getCommandOperations(contentStreams.iterator().next(), getValidators(), validateInput);
       }
     }
     return CommandOperation.clone(parsedCommands);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java b/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
index 43b018e..9f81110 100644
--- a/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
+++ b/solr/core/src/test/org/apache/solr/cloud/TestCollectionAPI.java
@@ -22,15 +22,18 @@ import java.util.HashMap;
 import java.util.List;
 import java.util.Locale;
 import java.util.Map;
+import java.util.concurrent.atomic.AtomicLong;
 
 import com.google.common.collect.Lists;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.impl.HttpSolrClient.RemoteSolrException;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.QueryRequest;
+import org.apache.solr.client.solrj.request.V2Request;
 import org.apache.solr.client.solrj.response.QueryResponse;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
@@ -67,7 +70,9 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
         req = CollectionAdminRequest.createCollection(COLLECTION_NAME, "conf1",2, 1, 0, 1);
       }
       req.setMaxShardsPerNode(2);
+      setV2(req);
       client.request(req);
+      assertV2CallsCount();
       createCollection(null, COLLECTION_NAME1, 1, 1, 1, client, null, "conf1");
     }
 
@@ -184,9 +189,8 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
   private void clusterStatusZNodeVersion() throws Exception {
     String cname = "clusterStatusZNodeVersion";
     try (CloudSolrClient client = createCloudClient(null)) {
-
-      CollectionAdminRequest.createCollection(cname,"conf1",1,1).setMaxShardsPerNode(1).process(client);
-
+      setV2(CollectionAdminRequest.createCollection(cname, "conf1", 1, 1).setMaxShardsPerNode(1)).process(client);
+      assertV2CallsCount();
       waitForRecoveriesToFinish(cname, true);
 
       ModifiableSolrParams params = new ModifiableSolrParams();
@@ -208,8 +212,9 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
       assertNotNull(znodeVersion);
 
       CollectionAdminRequest.AddReplica addReplica = CollectionAdminRequest.addReplicaToShard(cname, "shard1");
+      setV2(addReplica);
       addReplica.process(client);
-
+      assertV2CallsCount();
       waitForRecoveriesToFinish(cname, true);
 
       rsp = client.request(request);
@@ -222,6 +227,23 @@ public class TestCollectionAPI extends ReplicaPropertiesBase {
     }
   }
 
+  private static long totalexpectedV2Calls;
+
+  public static SolrRequest setV2(SolrRequest req) {
+    if (V2Request.v2Calls.get() == null) V2Request.v2Calls.set(new AtomicLong());
+    totalexpectedV2Calls = V2Request.v2Calls.get().get();
+    if (random().nextBoolean()) {
+      req.setUseV2(true);
+      req.setUseBinaryV2(random().nextBoolean());
+      totalexpectedV2Calls++;
+    }
+    return req;
+  }
+
+  public static void assertV2CallsCount() {
+    assertEquals(totalexpectedV2Calls, V2Request.v2Calls.get().get());
+  }
+
   private void clusterStatusWithRouteKey() throws IOException, SolrServerException {
     try (CloudSolrClient client = createCloudClient(DEFAULT_COLLECTION)) {
       SolrInputDocument doc = new SolrInputDocument();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java b/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
index e7dbf3e..f72bea1 100644
--- a/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
+++ b/solr/core/src/test/org/apache/solr/handler/admin/TestCollectionAPIs.java
@@ -17,7 +17,6 @@
 
 package org.apache.solr.handler.admin;
 
-import java.io.StringReader;
 import java.lang.invoke.MethodHandles;
 import java.util.Arrays;
 import java.util.Collection;
@@ -27,11 +26,15 @@ import java.util.List;
 import java.util.Map;
 
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.api.Api;
+import org.apache.solr.api.ApiBag;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.common.cloud.ZkNodeProps;
 import org.apache.solr.common.params.CollectionParams;
 import org.apache.solr.common.params.MultiMapSolrParams;
 import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.CommandOperation;
+import org.apache.solr.common.util.ContentStreamBase;
 import org.apache.solr.common.util.Pair;
 import org.apache.solr.common.util.Utils;
 import org.apache.solr.core.CoreContainer;
@@ -39,9 +42,6 @@ import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.servlet.SolrRequestParsers;
-import org.apache.solr.common.util.CommandOperation;
-import org.apache.solr.api.Api;
-import org.apache.solr.api.ApiBag;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -202,7 +202,7 @@ public class TestCollectionAPIs extends SolrTestCaseJ4 {
       @Override
       public List<CommandOperation> getCommands(boolean validateInput) {
         if (payload == null) return Collections.emptyList();
-        return ApiBag.getCommandOperations(new StringReader(payload), api.getCommandSchema(), true);
+        return ApiBag.getCommandOperations(new ContentStreamBase.StringStream(payload), api.getCommandSchema(), true);
       }
 
       @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/core/src/test/org/apache/solr/util/TestUtils.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/util/TestUtils.java b/solr/core/src/test/org/apache/solr/util/TestUtils.java
index 4cf6f6b..5ec88d7 100644
--- a/solr/core/src/test/org/apache/solr/util/TestUtils.java
+++ b/solr/core/src/test/org/apache/solr/util/TestUtils.java
@@ -16,22 +16,35 @@
  */
 package org.apache.solr.util;
 
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
 import java.io.IOException;
+import java.io.InputStream;
+import java.io.Reader;
 import java.io.StringReader;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 
 import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.client.solrj.impl.BinaryRequestWriter;
+import org.apache.solr.client.solrj.impl.BinaryRequestWriter.BAOS;
+import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.util.CommandOperation;
+import org.apache.solr.common.util.ContentStream;
+import org.apache.solr.common.util.ContentStreamBase;
+import org.apache.solr.common.util.JavaBinCodec;
+import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
 import org.apache.solr.common.util.StrUtils;
 import org.apache.solr.common.util.Utils;
 import org.junit.Assert;
 
 import static java.nio.charset.StandardCharsets.UTF_8;
+import static org.apache.solr.common.util.Utils.fromJSONString;
 
 /**
  *
@@ -163,7 +176,7 @@ public class TestUtils extends SolrTestCaseJ4 {
   public void testNoggitFlags() throws IOException {
     String s = "a{b:c, d [{k1:v1}{k2:v2}]}";
     assertNoggitJsonValues((Map) Utils.fromJSON(s.getBytes(UTF_8)));
-    assertNoggitJsonValues((Map) Utils.fromJSONString(s));
+    assertNoggitJsonValues((Map) fromJSONString(s));
     List<CommandOperation> commands = CommandOperation.parse(new StringReader(s + s));
     assertEquals(2, commands.size());
     for (CommandOperation command : commands) {
@@ -173,6 +186,21 @@ public class TestUtils extends SolrTestCaseJ4 {
     }
   }
 
+  public void testBinaryCommands() throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    new JavaBinCodec().marshal((MapWriter) ew -> {
+      ew.put("set-user", fromJSONString("{x:y}"));
+      ew.put("set-user", fromJSONString("{x:y,x1:y1}"));
+      ew.put("single", Arrays.asList(fromJSONString("[{x:y,x1:y1},{x2:y2}]"), fromJSONString( "{x2:y2}")));
+      ew.put("multi", Arrays.asList(fromJSONString("{x:y,x1:y1}"), fromJSONString( "{x2:y2}")));
+    }, baos);
+
+    ContentStream stream = new ContentStreamBase.ByteArrayStream(baos.toByteArray(),null, "application/javabin");
+    List<CommandOperation> commands = CommandOperation.readCommands(Collections.singletonList(stream), new NamedList(), Collections.singleton("single"));
+
+    assertEquals(5, commands.size());
+  }
+
   private void assertNoggitJsonValues(Map m) {
     assertEquals( "c" ,Utils.getObjectByPath(m, true, "/a/b"));
     assertEquals( "v1" ,Utils.getObjectByPath(m, true, "/a/d[0]/k1"));
@@ -196,7 +224,7 @@ public class TestUtils extends SolrTestCaseJ4 {
         "        'path':'/update/*',\n" +
         "        'role':'dev'}],\n" +
         "    '':{'v':4}}}";
-    Map m = (Map) Utils.fromJSONString(json);
+    Map m = (Map) fromJSONString(json);
     assertEquals("x-update", Utils.getObjectByPath(m,false, "authorization/permissions[1]/name"));
     
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/solrj/src/java/org/apache/solr/client/solrj/SolrRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/SolrRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/SolrRequest.java
index 4dbba5b..fe5c4fc 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/SolrRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/SolrRequest.java
@@ -56,6 +56,24 @@ public abstract class SolrRequest<T extends SolrResponse> implements Serializabl
   private StreamingResponseCallback callback;
   private Set<String> queryParams;
 
+  protected boolean usev2;
+  protected boolean useBinaryV2;
+
+  /**If set to true, every request that implements {@link V2RequestSupport} will be converted
+   * to a V2 API call
+   */
+  public SolrRequest setUseV2(boolean flag){
+    this.usev2 = flag;
+    return this;
+  }
+
+  /**If set to true use javabin instead of json (default)
+   */
+  public SolrRequest setUseBinaryV2(boolean flag){
+    this.useBinaryV2 = flag;
+    return this;
+  }
+
   private String basicAuthUser, basicAuthPwd;
 
   public SolrRequest setBasicAuthCredentials(String user, String password) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/solrj/src/java/org/apache/solr/client/solrj/V2RequestSupport.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/V2RequestSupport.java b/solr/solrj/src/java/org/apache/solr/client/solrj/V2RequestSupport.java
new file mode 100644
index 0000000..3ff13a6
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/V2RequestSupport.java
@@ -0,0 +1,30 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj;
+
+/**A a request object is able to convert itself to V2 Request
+ * it should implement this interface
+ *
+ */
+public interface V2RequestSupport {
+  /**If usev2 flag is set to true, return V2Request, if not,
+   * return V1 request object
+   *
+   */
+  SolrRequest getV2Request();
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
index 10ea590..f0684f8 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/CloudSolrClient.java
@@ -48,6 +48,7 @@ import org.apache.solr.client.solrj.ResponseParser;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.V2RequestSupport;
 import org.apache.solr.client.solrj.request.AbstractUpdateRequest;
 import org.apache.solr.client.solrj.request.IsUpdateRequest;
 import org.apache.solr.client.solrj.request.RequestWriter;
@@ -813,6 +814,7 @@ public class CloudSolrClient extends SolrClient {
    */
   protected NamedList<Object> requestWithRetryOnStaleState(SolrRequest request, int retryCount, String collection)
       throws SolrServerException, IOException {
+    SolrRequest originalRequest = request;
 
     connect(); // important to call this before you start working with the ZkStateReader
 
@@ -823,6 +825,9 @@ public class CloudSolrClient extends SolrClient {
     String stateVerParam = null;
     List<DocCollection> requestedCollections = null;
     boolean isCollectionRequestOfV2 = false;
+    if (request instanceof V2RequestSupport) {
+      request = ((V2RequestSupport) request).getV2Request();
+    }
     if (request instanceof V2Request) {
       isCollectionRequestOfV2 = ((V2Request) request).isPerCollectionRequest();
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
index 1ba08bf..104ab1f 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrClient.java
@@ -65,6 +65,7 @@ import org.apache.solr.client.solrj.ResponseParser;
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.V2RequestSupport;
 import org.apache.solr.client.solrj.request.RequestWriter;
 import org.apache.solr.client.solrj.request.V2Request;
 import org.apache.solr.common.SolrException;
@@ -295,8 +296,10 @@ public class HttpSolrClient extends SolrClient {
     return queryModParams;
   }
 
-  protected HttpRequestBase createMethod(final SolrRequest request, String collection) throws IOException, SolrServerException {
-
+  protected HttpRequestBase createMethod(SolrRequest request, String collection) throws IOException, SolrServerException {
+    if (request instanceof V2RequestSupport) {
+      request = ((V2RequestSupport) request).getV2Request();
+    }
     SolrParams params = request.getParams();
     Collection<ContentStream> streams = requestWriter.getContentStreams(request);
     String path = requestWriter.getPath(request);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
index 9ce6664..ed5b622 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionAdminRequest.java
@@ -28,6 +28,7 @@ import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrResponse;
 import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.V2RequestSupport;
 import org.apache.solr.client.solrj.response.CollectionAdminResponse;
 import org.apache.solr.client.solrj.response.RequestStatusState;
 import org.apache.solr.client.solrj.util.SolrIdentifierValidator;
@@ -55,7 +56,7 @@ import static org.apache.solr.common.params.CollectionAdminParams.CREATE_NODE_SE
  *
  * @since solr 4.5
  */
-public abstract class CollectionAdminRequest<T extends CollectionAdminResponse> extends SolrRequest<T> {
+public abstract class CollectionAdminRequest<T extends CollectionAdminResponse> extends SolrRequest<T> implements V2RequestSupport {
 
   protected final CollectionAction action;
 
@@ -71,6 +72,13 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
   }
 
   @Override
+  public SolrRequest getV2Request() {
+    return usev2 ?
+        V1toV2ApiMapper.convert(this).useBinary(useBinaryV2).build() :
+        this;
+  }
+
+  @Override
   public SolrParams getParams() {
     ModifiableSolrParams params = new ModifiableSolrParams();
     params.set(CoreAdminParams.ACTION, action.toString());

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
index e622c4b..0b77c0e 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/CollectionApiMapping.java
@@ -23,11 +23,13 @@ import java.util.Collection;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.function.BiConsumer;
 
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.common.params.CollectionParams.CollectionAction;
 import org.apache.solr.common.params.ConfigSetParams.ConfigSetAction;
 import org.apache.solr.common.util.CommandOperation;
+import org.apache.solr.common.util.Pair;
 import org.apache.solr.common.util.Utils;
 
 import static org.apache.solr.client.solrj.SolrRequest.METHOD.DELETE;
@@ -274,6 +276,19 @@ public class CollectionApiMapping {
       }
       return s;
     }
+    public Object getReverseParamSubstitute(String param) {
+      String s = paramstoAttr.containsKey(param) ? paramstoAttr.get(param) : param;
+
+      if (prefixSubstitutes != null) {
+        for (Map.Entry<String, String> e : prefixSubstitutes.entrySet()) {
+          if(param.startsWith(e.getValue())){
+            return new Pair<>(e.getKey().substring(0, e.getKey().length() - 1), param.substring(e.getValue().length()));
+          }
+        }
+      }
+      return s;
+
+    }
 
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/solrj/src/java/org/apache/solr/client/solrj/request/V1toV2ApiMapper.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/V1toV2ApiMapper.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/V1toV2ApiMapper.java
new file mode 100644
index 0000000..c34f92c
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/V1toV2ApiMapper.java
@@ -0,0 +1,144 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj.request;
+
+import java.util.EnumMap;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrRequest;
+import org.apache.solr.common.MapWriter;
+import org.apache.solr.common.params.CollectionParams.CollectionAction;
+import org.apache.solr.common.params.CoreAdminParams;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.JsonSchemaValidator;
+import org.apache.solr.common.util.Pair;
+import org.apache.solr.common.util.Template;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.common.util.ValidatingJsonMap;
+
+import static org.apache.solr.common.util.ValidatingJsonMap.NOT_NULL;
+
+
+public class V1toV2ApiMapper {
+
+  private static EnumMap<CollectionAction, ActionInfo> mapping = new EnumMap<>(CollectionAction.class);
+
+  static {
+    for (CollectionApiMapping.Meta meta : CollectionApiMapping.Meta.values()) {
+      if (meta.action != null) mapping.put(meta.action, new ActionInfo(meta));
+    }
+  }
+
+  private static class ActionInfo {
+    CollectionApiMapping.Meta meta;
+    String path;
+    Template template;
+
+
+    JsonSchemaValidator validator;
+
+    ActionInfo(CollectionApiMapping.Meta meta) {
+      this.meta = meta;
+    }
+
+    //do this lazily because , it makes no sense if this is not used
+    synchronized void setPath() {
+      if (path == null) {
+        ValidatingJsonMap m = Utils.getSpec(meta.getEndPoint().getSpecName()).getSpec();
+        Object o = Utils.getObjectByPath(m, false, "url/paths");
+
+        String result = null;
+        if (o instanceof List) {//choose the shortest path
+          for (Object s : (List) o) {
+            if (result == null || s.toString().length() < result.length()) result = s.toString();
+          }
+        } else if (o instanceof String) {
+          result = (String) o;
+        }
+        path = result;
+        template = new Template(path, Template.BRACES_PLACEHOLDER_PATTERN);
+
+        validator = new JsonSchemaValidator(m.getMap("commands", NOT_NULL).getMap(meta.commandName, NOT_NULL));
+      }
+    }
+
+    public V2Request.Builder convert(SolrParams params) {
+      String[] list = new String[template.variables.size()];
+      MapWriter data = serializeToV2Format(params, list);
+      Map o = data.toMap(new LinkedHashMap<>());
+      return new V2Request.Builder(template.apply(s -> {
+        int idx = template.variables.indexOf(s);
+        return list[idx];
+      }))
+          .withMethod(meta.getHttpMethod())
+          .withPayload(o);
+
+    }
+
+    private MapWriter serializeToV2Format(SolrParams params, String[] list) {
+      return ew -> ew.put(meta.commandName, (MapWriter) ew1 -> {
+        Iterator<String> iter = params.getParameterNamesIterator();
+        Map<String, Map<String, String>> subProperties = null;
+        while (iter.hasNext()) {
+          String key = iter.next();
+          if (CoreAdminParams.ACTION.equals(key)) continue;
+          Object substitute = meta.getReverseParamSubstitute(key);
+          int idx = template.variables.indexOf(substitute);
+          if (idx > -1) {
+            String val = params.get(String.valueOf(substitute));
+            if (val == null) throw new RuntimeException("null value is not valid for " + key);
+            list[idx] = val;
+            continue;
+          }
+          if (substitute instanceof Pair) {//this is a nested object
+            Pair<String, String> p = (Pair<String, String>) substitute;
+            if (subProperties == null) subProperties = new HashMap<>();
+            subProperties.computeIfAbsent(p.first(), s -> new HashMap<>()).put(p.second(), params.get(key));
+          } else {
+            Object val = params.get(key);
+            ew1.put(substitute.toString(), val);
+          }
+        }
+        if (subProperties != null) {
+          for (Map.Entry<String, Map<String, String>> e : subProperties.entrySet()) {
+            ew1.put(e.getKey(), e.getValue());
+          }
+        }
+      });
+    }
+  }
+
+
+  public static V2Request.Builder convert(CollectionAdminRequest request) {
+    ActionInfo info = mapping.get(request.action);
+    if (info == null) throw new RuntimeException("Unsupported action :" + request.action);
+
+    if (info.meta.getHttpMethod() == SolrRequest.METHOD.POST) {
+      if (info.path == null) info.setPath();
+      return info.convert(request.getParams());
+    }
+
+    return null;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java b/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java
index 6cc2314..df186e8 100644
--- a/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java
+++ b/solr/solrj/src/java/org/apache/solr/client/solrj/request/V2Request.java
@@ -23,23 +23,29 @@ import java.io.InputStream;
 import java.nio.charset.StandardCharsets;
 import java.util.Collection;
 import java.util.Collections;
+import java.util.concurrent.atomic.AtomicLong;
 import java.util.regex.Pattern;
 
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.ContentStream;
 import org.apache.solr.common.util.ContentStreamBase;
+import org.apache.solr.common.util.Utils;
 
 public class V2Request extends SolrRequest {
+  //only for debugging purposes
+  public static final ThreadLocal<AtomicLong> v2Calls = new ThreadLocal<>();
   static final Pattern COLL_REQ_PATTERN = Pattern.compile("/(c|collections)/[^/]+/(?!shards)");
   private InputStream payload;
   private SolrParams solrParams;
+  public final boolean useBinary;
 
-  private V2Request(METHOD m, String resource, InputStream payload) {
+  private V2Request(METHOD m, String resource, boolean useBinary) {
     super(m, resource);
-    this.payload = payload;
+    this.useBinary = useBinary;
   }
 
   @Override
@@ -49,6 +55,7 @@ public class V2Request extends SolrRequest {
 
   @Override
   public Collection<ContentStream> getContentStreams() throws IOException {
+    if (v2Calls.get() != null) v2Calls.get().incrementAndGet();
     if (payload != null) {
       return Collections.singleton(new ContentStreamBase() {
         @Override
@@ -58,7 +65,7 @@ public class V2Request extends SolrRequest {
 
         @Override
         public String getContentType() {
-          return "application/json";
+          return useBinary ? "application/javabin" : "application/json";
         }
       });
     }
@@ -77,8 +84,9 @@ public class V2Request extends SolrRequest {
   public static class Builder {
     private String resource;
     private METHOD method = METHOD.GET;
-    private InputStream payload;
+    private Object payload;
     private SolrParams params;
+    private boolean useBinary = false;
 
     /**
      * Create a Builder object based on the provided resource.
@@ -108,7 +116,12 @@ public class V2Request extends SolrRequest {
       return this;
     }
 
-    public Builder withPayLoad(InputStream payload) {
+    public Builder withPayload(Object payload) {
+      this.payload = payload;
+      return this;
+    }
+
+    public Builder withPayload(InputStream payload) {
       this.payload = payload;
       return this;
     }
@@ -118,10 +131,28 @@ public class V2Request extends SolrRequest {
       return this;
     }
 
+    public Builder useBinary(boolean flag) {
+      this.useBinary = flag;
+      return this;
+    }
+
     public V2Request build() {
-      V2Request v2Request = new V2Request(method, resource, payload);
-      v2Request.solrParams = params;
-      return v2Request;
+      try {
+        V2Request v2Request = new V2Request(method, resource, useBinary);
+        v2Request.solrParams = params;
+        InputStream is = null;
+        if (payload != null) {
+          if (payload instanceof InputStream) is = (InputStream) payload;
+          else if (useBinary) is = Utils.toJavabin(payload);
+          else is = new ByteArrayInputStream(Utils.toJSON(payload));
+        }
+        v2Request.payload = is;
+        return v2Request;
+      } catch (IOException e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
+      }
+
+
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/solrj/src/java/org/apache/solr/common/util/CommandOperation.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/CommandOperation.java b/solr/solrj/src/java/org/apache/solr/common/util/CommandOperation.java
index c18a8e4..50002f6 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/CommandOperation.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/CommandOperation.java
@@ -17,10 +17,12 @@
 package org.apache.solr.common.util;
 
 import java.io.IOException;
+import java.io.InputStream;
 import java.io.Reader;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Collections;
+import java.util.HashMap;
 import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
@@ -221,7 +223,41 @@ public class CommandOperation {
   }
 
   /**
-   * Parse the command operations into command objects
+   * Parse the command operations into command objects from javabin payload
+   * * @param singletonCommands commands that cannot be repeated
+   */
+  public static List<CommandOperation> parse(InputStream in, Set<String> singletonCommands) throws IOException {
+    List<CommandOperation> operations = new ArrayList<>();
+
+    final HashMap map = new HashMap(0) {
+      @Override
+      public Object put(Object key, Object value) {
+        List vals = null;
+        if (value instanceof List && !singletonCommands.contains(key)) {
+          vals = (List) value;
+        } else {
+          vals = Collections.singletonList(value);
+        }
+        for (Object val : vals) {
+          operations.add(new CommandOperation(String.valueOf(key), val));
+        }
+        return null;
+      }
+    };
+
+    new JavaBinCodec() {
+      int level = 0;
+      @Override
+      protected Map<Object, Object> newMap(int size) {
+        level++;
+        return level == 1 ? map : super.newMap(size);
+      }
+    }.unmarshal(in);
+    return operations;
+  }
+
+  /**
+   * Parse the command operations into command objects from a json payload
    *
    * @param rdr               The payload
    * @param singletonCommands commands that cannot be repeated
@@ -304,9 +340,13 @@ public class CommandOperation {
       throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "missing content stream");
     }
     ArrayList<CommandOperation> ops = new ArrayList<>();
-
-    for (ContentStream stream : streams)
-      ops.addAll(parse(stream.getReader(), singletonCommands));
+    for (ContentStream stream : streams) {
+      if ("application/javabin".equals(stream.getContentType())) {
+        ops.addAll(parse(stream.getStream(), singletonCommands));
+      } else {
+        ops.addAll(parse(stream.getReader(), singletonCommands));
+      }
+    }
     List<Map> errList = CommandOperation.captureErrors(ops);
     if (!errList.isEmpty()) {
       resp.add(CommandOperation.ERR_MSGS, errList);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/solrj/src/java/org/apache/solr/common/util/ContentStreamBase.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/ContentStreamBase.java b/solr/solrj/src/java/org/apache/solr/common/util/ContentStreamBase.java
index bd1d63b..f9db69c 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/ContentStreamBase.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/ContentStreamBase.java
@@ -258,11 +258,14 @@ public abstract class ContentStreamBase implements ContentStream
   public static class ByteArrayStream extends ContentStreamBase
   {
     private final byte[] bytes;
-    
     public ByteArrayStream( byte[] bytes, String source ) {
+      this(bytes,source, null);
+    }
+    
+    public ByteArrayStream( byte[] bytes, String source, String contentType ) {
       this.bytes = bytes; 
       
-      this.contentType = null;
+      this.contentType = contentType;
       name = source;
       size = new Long(bytes.length);
       sourceInfo = source;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/solrj/src/java/org/apache/solr/common/util/JavaBinCodec.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/JavaBinCodec.java b/solr/solrj/src/java/org/apache/solr/common/util/JavaBinCodec.java
index d9843e1..a2400f3 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/JavaBinCodec.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/JavaBinCodec.java
@@ -628,7 +628,7 @@ public class JavaBinCodec implements PushWriter {
 
 
   public Map<Object, Object> readMapIter(DataInputInputStream dis) throws IOException {
-    Map<Object, Object> m = new LinkedHashMap<>();
+    Map<Object, Object> m = newMap(-1);
     for (; ; ) {
       Object key = readVal(dis);
       if (key == END_OBJ) break;
@@ -638,10 +638,18 @@ public class JavaBinCodec implements PushWriter {
     return m;
   }
 
+  /**
+   * create a new Map object
+   * @param size expected size, -1 means unknown size
+   */
+  protected Map<Object, Object> newMap(int size) {
+    return size < 0 ? new LinkedHashMap<>() : new LinkedHashMap<>(size);
+  }
+
   public Map<Object,Object> readMap(DataInputInputStream dis)
           throws IOException {
     int sz = readVInt(dis);
-    Map<Object,Object> m = new LinkedHashMap<>(sz);
+    Map<Object, Object> m = newMap(sz);
     for (int i = 0; i < sz; i++) {
       Object key = readVal(dis);
       Object val = readVal(dis);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaValidator.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaValidator.java b/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaValidator.java
index e7b725f..e5a1d44 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaValidator.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/JsonSchemaValidator.java
@@ -94,8 +94,32 @@ enum Type {
   STRING(String.class),
   ARRAY(List.class),
   NUMBER(Number.class),
-  INTEGER(Long.class),
-  BOOLEAN(Boolean.class),
+  INTEGER(Long.class){
+    @Override
+    boolean isValid(Object o) {
+      if(super.isValid(o)) return true;
+      try {
+        Long.parseLong(String.valueOf(o));
+        return true;
+      } catch (NumberFormatException e) {
+        return false;
+
+      }
+    }
+  },
+  BOOLEAN(Boolean.class){
+    @Override
+    boolean isValid(Object o) {
+      if(super.isValid(o)) return true;
+      try {
+        Boolean.parseBoolean (String.valueOf(o));
+        return true;
+      } catch (NumberFormatException e) {
+        return false;
+      }
+
+    }
+  },
   ENUM(List.class),
   OBJECT(Map.class),
   NULL(null),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/solrj/src/java/org/apache/solr/common/util/Template.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/Template.java b/solr/solrj/src/java/org/apache/solr/common/util/Template.java
new file mode 100644
index 0000000..478addc
--- /dev/null
+++ b/solr/solrj/src/java/org/apache/solr/common/util/Template.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.common.util;
+
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.function.Function;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+public class Template {
+  public final String template;
+  public static final Pattern DOLLAR_BRACES_PLACEHOLDER_PATTERN = Pattern
+      .compile("[$][{](.*?)[}]");
+  public static final Pattern BRACES_PLACEHOLDER_PATTERN = Pattern
+      .compile("[{](.*?)[}]");
+
+
+  public Template(String template, Pattern pattern) {
+    this.template = template;
+    List<String> variables = new ArrayList<>(2);
+    Matcher m = pattern.matcher(template);
+    while (m.find()) {
+      String variable = m.group(1);
+      startIndexes.add(m.start(0));
+      endOffsets.add(m.end(0));
+      variables.add(variable);
+    }
+    this.variables = Collections.unmodifiableList(variables);
+
+  }
+
+  public String apply(Function<String, Object> valueSupplier) {
+    if (startIndexes != null) {
+      StringBuilder sb = new StringBuilder(template);
+      for (int i = startIndexes.size() - 1; i >= 0; i--) {
+        String replacement = valueSupplier.apply(variables.get(i)).toString();
+        sb.replace(startIndexes.get(i), endOffsets.get(i), replacement);
+      }
+      return sb.toString();
+    } else {
+      return template;
+    }
+  }
+
+  private List<Integer> startIndexes = new ArrayList<>(2);
+  private List<Integer> endOffsets = new ArrayList<>(2);
+  public final List<String> variables ;
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/util/Utils.java b/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
index b64f971..66e21c0 100644
--- a/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
+++ b/solr/solrj/src/java/org/apache/solr/common/util/Utils.java
@@ -22,6 +22,7 @@ import java.io.InputStreamReader;
 import java.io.Reader;
 import java.io.StringReader;
 import java.lang.invoke.MethodHandles;
+import java.nio.ByteBuffer;
 import java.nio.charset.StandardCharsets;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -39,6 +40,7 @@ import java.util.regex.Pattern;
 
 import org.apache.http.HttpEntity;
 import org.apache.http.util.EntityUtils;
+import org.apache.solr.client.solrj.impl.BinaryRequestWriter;
 import org.apache.solr.common.IteratorWriter;
 import org.apache.solr.common.MapWriter;
 import org.apache.solr.common.SolrException;
@@ -47,6 +49,7 @@ import org.apache.solr.common.cloud.SolrZkClient;
 import org.apache.solr.common.cloud.ZkOperation;
 import org.apache.solr.common.params.CommonParams;
 import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.server.ByteBufferInputStream;
 import org.noggit.CharArr;
 import org.noggit.JSONParser;
 import org.noggit.JSONWriter;
@@ -102,6 +105,12 @@ public class Utils {
     return v;
   }
 
+  public static InputStream toJavabin(Object o) throws IOException {
+    BinaryRequestWriter.BAOS baos = new BinaryRequestWriter.BAOS();
+    new JavaBinCodec().marshal(o,baos);
+    return new ByteBufferInputStream(ByteBuffer.wrap(baos.getbuf(),0,baos.size()));
+  }
+
   public static Collection getDeepCopy(Collection c, int maxDepth, boolean mutable) {
     return getDeepCopy(c, maxDepth, mutable, false);
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/f6f6f113/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV1toV2ApiMapper.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV1toV2ApiMapper.java b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV1toV2ApiMapper.java
new file mode 100644
index 0000000..7daefb9
--- /dev/null
+++ b/solr/solrj/src/test/org/apache/solr/client/solrj/request/TestV1toV2ApiMapper.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.client.solrj.request;
+
+import java.io.IOException;
+import java.util.Map;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.lucene.util.LuceneTestCase;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest.Create;
+import org.apache.solr.common.util.Utils;
+
+public class TestV1toV2ApiMapper extends LuceneTestCase {
+
+  public void testCommands() throws IOException {
+    Create cmd = CollectionAdminRequest
+        .createCollection("mycoll", "conf1", 3, 2)
+        .setProperties(ImmutableMap.<String,String>builder()
+            .put("p1","v1")
+            .put("p2","v2")
+            .build());
+    V2Request v2r = V1toV2ApiMapper.convert(cmd).build();
+    Map m = (Map) Utils.fromJSON(v2r.getContentStreams().iterator().next().getStream());
+    assertEquals("/c", v2r.getPath());
+    assertEquals("v1", Utils.getObjectByPath(m,true,"/create/properties/p1"));
+    assertEquals("v2", Utils.getObjectByPath(m,true,"/create/properties/p2"));
+    assertEquals("3", Utils.getObjectByPath(m,true,"/create/numShards"));
+    assertEquals("2", Utils.getObjectByPath(m,true,"/create/nrtReplicas"));
+
+    CollectionAdminRequest.AddReplica addReplica = CollectionAdminRequest.addReplicaToShard("mycoll", "shard1");
+    v2r = V1toV2ApiMapper.convert(addReplica).build();
+    m = (Map) Utils.fromJSON(v2r.getContentStreams().iterator().next().getStream());
+    assertEquals("/c/mycoll/shards", v2r.getPath());
+    assertEquals("shard1", Utils.getObjectByPath(m,true,"/add-replica/shard"));
+    assertEquals("NRT", Utils.getObjectByPath(m,true,"/add-replica/type"));
+
+
+
+  }
+}