You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ds...@apache.org on 2018/01/28 21:56:27 UTC

lucene-solr:master: SOLR-11617: Alias metadata API; returned from LISTALIASES, set via MODIFYALIAS

Repository: lucene-solr
Updated Branches:
  refs/heads/master 56f3f6d94 -> 154bdeb7d


SOLR-11617: Alias metadata API; returned from LISTALIASES, set via MODIFYALIAS


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

Branch: refs/heads/master
Commit: 154bdeb7db1794c4019ceb1c27b47ff6159a08e8
Parents: 56f3f6d
Author: David Smiley <ds...@apache.org>
Authored: Sun Jan 28 16:56:19 2018 -0500
Committer: David Smiley <ds...@apache.org>
Committed: Sun Jan 28 16:56:19 2018 -0500

----------------------------------------------------------------------
 solr/CHANGES.txt                                |   3 +
 .../cloud/api/collections/ModifyAliasCmd.java   |  81 ++++++++++
 .../OverseerCollectionMessageHandler.java       |   1 +
 .../solr/handler/admin/CollectionsHandler.java  |  49 +++++-
 .../apache/solr/cloud/AliasIntegrationTest.java | 161 +++++++++++++++++--
 solr/solr-ref-guide/src/collections-api.adoc    |  61 ++++++-
 .../solrj/request/CollectionAdminRequest.java   |  35 ++++
 .../solrj/request/CollectionApiMapping.java     |  11 +-
 .../solr/common/params/CollectionParams.java    |   1 +
 .../resources/apispec/collections.Commands.json |  23 +++
 10 files changed, 404 insertions(+), 22 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/154bdeb7/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index f622374..c7c83db 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -110,6 +110,9 @@ New Features
 
 * SOLR-11782: Refactor LatchWatcher.await to protect against spurious wakeup (Tomás Fernández Löbbe, David Smiley, Dawid Weiss)
 
+* SOLR-11617: Alias metadata is now mutable via a new MODIFYALIAS command.  Metadata is returned from LISTALIASES.
+  (Gus Heck via David Smiley)
+
 Bug Fixes
 ----------------------
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/154bdeb7/solr/core/src/java/org/apache/solr/cloud/api/collections/ModifyAliasCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/ModifyAliasCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/ModifyAliasCmd.java
new file mode 100644
index 0000000..992f20d
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/ModifyAliasCmd.java
@@ -0,0 +1,81 @@
+/*
+ * 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.cloud.api.collections;
+
+import java.lang.invoke.MethodHandles;
+import java.util.Locale;
+import java.util.Map;
+
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.ZkNodeProps;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.NamedList;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.cloud.api.collections.OverseerCollectionMessageHandler.*;
+import static org.apache.solr.common.SolrException.ErrorCode.BAD_REQUEST;
+import static org.apache.solr.common.params.CommonParams.NAME;
+
+public class ModifyAliasCmd implements Cmd {
+
+  public static final String META_DATA = "metadata";
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final OverseerCollectionMessageHandler messageHandler;
+
+  ModifyAliasCmd(OverseerCollectionMessageHandler messageHandler) {
+    this.messageHandler = messageHandler;
+  }
+
+  @Override
+  public void call(ClusterState state, ZkNodeProps message, NamedList results) throws Exception {
+    String aliasName = message.getStr(NAME);
+
+
+    ZkStateReader zkStateReader = messageHandler.zkStateReader;
+    if (zkStateReader.getAliases().getCollectionAliasMap().get(aliasName) == null) {
+      // nicer than letting aliases object throw later on...
+      throw new SolrException(BAD_REQUEST,
+          String.format(Locale.ROOT,  "Can't modify non-existent alias %s", aliasName));
+    }
+
+    @SuppressWarnings("unchecked")
+    Map<String, String> metadata = (Map<String, String>) message.get(META_DATA);
+
+    zkStateReader.aliasesHolder.applyModificationAndExportToZk(aliases1 -> {
+      for (Map.Entry<String, String> entry : metadata.entrySet()) {
+        String key = entry.getKey();
+        if ("".equals(key.trim())) {
+          throw new SolrException(BAD_REQUEST, "metadata keys must not be pure whitespace");
+        }
+        if (!key.equals(key.trim())) {
+          throw new SolrException(BAD_REQUEST, "metadata keys should not begin or end with whitespace");
+        }
+        String value = entry.getValue();
+        if ("".equals(value)) {
+          value = null;
+        }
+        aliases1 = aliases1.cloneWithCollectionAliasMetadata(aliasName, key, value);
+      }
+      return aliases1;
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/154bdeb7/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
index 3df8cde..ba27fe4 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/OverseerCollectionMessageHandler.java
@@ -228,6 +228,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
         .put(CREATEALIAS, new CreateAliasCmd(this))
         .put(CREATEROUTEDALIAS, new CreateAliasCmd(this))
         .put(DELETEALIAS, new DeleteAliasCmd(this))
+        .put(MODIFYALIAS, new ModifyAliasCmd(this))
         .put(ROUTEDALIAS_CREATECOLL, new RoutedAliasCreateCollectionCmd(this))
         .put(OVERSEERSTATUS, new OverseerStatusCmd(this))
         .put(DELETESHARD, new DeleteShardCmd(this))

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/154bdeb7/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
index d4598e9..cebb2d0 100644
--- a/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
+++ b/solr/core/src/java/org/apache/solr/handler/admin/CollectionsHandler.java
@@ -537,14 +537,34 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
     DELETEALIAS_OP(DELETEALIAS, (req, rsp, h) -> req.getParams().required().getAll(null, NAME)),
 
     /**
-     * Handle cluster status request.
-     * Can return status per specific collection/shard or per all collections.
+     * Change metadata for an alias (use CREATEALIAS_OP to change the actual value of the alias)
+     */
+    MODIFYALIAS_OP(MODIFYALIAS, (req, rsp, h) -> {
+      Map<String, Object> params = req.getParams().required().getAll(null, NAME);
+
+      // Note: success/no-op in the event of no metadata supplied is intentional. Keeps code simple and one less case
+      // for api-callers to check for.
+      return convertPrefixToMap(req.getParams(), params, "metadata");
+    }),
+
+    /**
+     * List the aliases and associated metadata.
      */
     LISTALIASES_OP(LISTALIASES, (req, rsp, h) -> {
       ZkStateReader zkStateReader = h.coreContainer.getZkController().getZkStateReader();
       Aliases aliases = zkStateReader.getAliases();
       if (aliases != null) {
+        // the aliases themselves...
         rsp.getValues().add("aliases", aliases.getCollectionAliasMap());
+        // Any metadata for the above aliases.
+        Map<String,Map<String,String>> meta = new LinkedHashMap<>();
+        for (String alias : aliases.getCollectionAliasListMap().keySet()) {
+          Map<String, String> collectionAliasMetadata = aliases.getCollectionAliasMetadata(alias);
+          if (collectionAliasMetadata != null) {
+            meta.put(alias, collectionAliasMetadata);
+          }
+        }
+        rsp.getValues().add("metadata", meta);
       }
       return null;
     }),
@@ -990,6 +1010,31 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
     }),
     DELETENODE_OP(DELETENODE, (req, rsp, h) -> req.getParams().required().getAll(null, "node"));
 
+    /**
+     * Places all prefixed properties in the sink map (or a new map) using the prefix as the key and a map of
+     * all prefixed properties as the value. The sub-map keys have the prefix removed.
+     *
+     * @param params The solr params from which to extract prefixed properties.
+     * @param sink The map to add the properties too.
+     * @param prefix The prefix to identify properties to be extracted
+     * @return The sink map, or a new map if the sink map was null
+     */
+    private static Map<String, Object> convertPrefixToMap(SolrParams params, Map<String, Object> sink, String prefix) {
+      Map<String,Object> result = new LinkedHashMap<>();
+      Iterator<String> iter =  params.getParameterNamesIterator();
+      while (iter.hasNext()) {
+        String param = iter.next();
+        if (param.startsWith(prefix)) {
+          result.put(param.substring(prefix.length()+1), params.get(param));
+        }
+      }
+      if (sink == null) {
+        sink = new LinkedHashMap<>();
+      }
+      sink.put(prefix, result);
+      return sink;
+    }
+
     public final CollectionOp fun;
     CollectionAction action;
     long timeOut;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/154bdeb7/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java b/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java
index 1c20b30..e2e155f 100644
--- a/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java
@@ -22,6 +22,15 @@ import java.util.Map;
 import java.util.function.Consumer;
 import java.util.function.UnaryOperator;
 
+import org.apache.http.client.methods.CloseableHttpResponse;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.methods.HttpUriRequest;
+import org.apache.http.entity.ContentType;
+import org.apache.http.entity.StringEntity;
+import org.apache.http.impl.client.CloseableHttpClient;
+import org.apache.http.util.EntityUtils;
+import org.apache.lucene.util.IOUtils;
 import org.apache.solr.client.solrj.SolrQuery;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrServerException;
@@ -40,6 +49,9 @@ import org.apache.solr.common.cloud.ZkStateReader;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.common.util.Utils;
+import org.apache.zookeeper.KeeperException;
+import org.junit.After;
+import org.junit.Before;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -47,6 +59,9 @@ import static org.apache.solr.common.cloud.ZkStateReader.ALIASES;
 
 public class AliasIntegrationTest extends SolrCloudTestCase {
 
+  private CloseableHttpClient httpClient;
+  private CloudSolrClient solrClient;
+
   @BeforeClass
   public static void setupCluster() throws Exception {
     configureCluster(2)
@@ -54,6 +69,33 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
         .configure();
   }
 
+  @Before
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    solrClient = getCloudSolrClient(cluster);
+    httpClient = (CloseableHttpClient) solrClient.getHttpClient();
+  }
+
+  @After
+  @Override
+  public void tearDown() throws Exception {
+    super.tearDown();
+    IOUtils.close(solrClient, httpClient);
+
+    // make sure all aliases created are removed for the next test method
+    Map<String, String> aliases = new CollectionAdminRequest.ListAliases().process(cluster.getSolrClient()).getAliases();
+    for (String alias : aliases.keySet()) {
+      CollectionAdminRequest.deleteAlias(alias).processAsync(cluster.getSolrClient());
+    }
+
+    // make sure all collections are removed for the next test method
+    List<String> collections = CollectionAdminRequest.listCollections(cluster.getSolrClient());
+    for (String collection : collections) {
+      CollectionAdminRequest.deleteCollection(collection).process(cluster.getSolrClient());
+    }
+  }
+
   @Test
   public void testMetadata() throws Exception {
     CollectionAdminRequest.createCollection("collection1meta", "conf", 2, 1).process(cluster.getSolrClient());
@@ -75,6 +117,7 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
     assertEquals("collection2meta", aliases.get(1));
     //ensure we have the back-compat format in ZK:
     final byte[] rawBytes = zkStateReader.getZkClient().getData(ALIASES, null, null, true);
+    //noinspection unchecked
     assertTrue(((Map<String,Map<String,?>>)Utils.fromJSON(rawBytes)).get("collection").get("meta1") instanceof String);
 
     // set metadata
@@ -178,6 +221,104 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
     }
   }
 
+  public void testModifyMetadataV2() throws Exception {
+    final String aliasName = getTestName();
+    ZkStateReader zkStateReader = createColectionsAndAlias(aliasName);
+    final String baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
+    //TODO fix Solr test infra so that this /____v2/ becomes /api/
+    HttpPost post = new HttpPost(baseUrl + "/____v2/c");
+    post.setEntity(new StringEntity("{\n" +
+        "\"modify-alias\" : {\n" +
+        "  \"name\": \"" + aliasName + "\",\n" +
+        "  \"metadata\" : {\n" +
+        "    \"foo\": \"baz\",\n" +
+        "    \"bar\": \"bam\"\n" +
+        "    }\n" +
+        //TODO should we use "NOW=" param?  Won't work with v2 and is kinda a hack any way since intended for distrib
+        "  }\n" +
+        "}", ContentType.APPLICATION_JSON));
+    assertSuccess(post);
+    checkFooAndBarMeta(aliasName, zkStateReader);
+  }
+
+  public void testModifyMetadataV1() throws Exception {
+    // note we don't use TZ in this test, thus it's UTC
+    final String aliasName = getTestName();
+    ZkStateReader zkStateReader = createColectionsAndAlias(aliasName);
+    final String baseUrl = cluster.getRandomJetty(random()).getBaseUrl().toString();
+    HttpGet get = new HttpGet(baseUrl + "/admin/collections?action=MODIFYALIAS" +
+        "&wt=xml" +
+        "&name=" + aliasName +
+        "&metadata.foo=baz" +
+        "&metadata.bar=bam");
+    assertSuccess(get);
+    checkFooAndBarMeta(aliasName, zkStateReader);
+  }
+
+  public void testModifyMetadataCAR() throws Exception {
+    // note we don't use TZ in this test, thus it's UTC
+    final String aliasName = getTestName();
+    ZkStateReader zkStateReader = createColectionsAndAlias(aliasName);
+    CollectionAdminRequest.ModifyAlias modifyAlias = CollectionAdminRequest.modifyAlias(aliasName);
+    modifyAlias.addMetadata("foo","baz");
+    modifyAlias.addMetadata("bar","bam");
+    modifyAlias.process(cluster.getSolrClient());
+    checkFooAndBarMeta(aliasName, zkStateReader);
+
+    // now verify we can delete
+    modifyAlias = CollectionAdminRequest.modifyAlias(aliasName);
+    modifyAlias.addMetadata("foo","");
+    modifyAlias.process(cluster.getSolrClient());
+    modifyAlias = CollectionAdminRequest.modifyAlias(aliasName);
+    modifyAlias.addMetadata("bar",null);
+    modifyAlias.process(cluster.getSolrClient());
+    modifyAlias = CollectionAdminRequest.modifyAlias(aliasName);
+
+    // whitespace value
+    modifyAlias.addMetadata("foo"," ");
+    modifyAlias.process(cluster.getSolrClient());
+
+
+  }
+
+  private void checkFooAndBarMeta(String aliasName, ZkStateReader zkStateReader) {
+    Map<String, String> meta = zkStateReader.getAliases().getCollectionAliasMetadata(aliasName);
+    assertNotNull(meta);
+    assertTrue(meta.containsKey("foo"));
+    assertEquals("baz", meta.get("foo"));
+    assertTrue(meta.containsKey("bar"));
+    assertEquals("bam", meta.get("bar"));
+  }
+
+  private ZkStateReader createColectionsAndAlias(String aliasName) throws SolrServerException, IOException, KeeperException, InterruptedException {
+    CollectionAdminRequest.createCollection("collection1meta", "conf", 2, 1).process(cluster.getSolrClient());
+    CollectionAdminRequest.createCollection("collection2meta", "conf", 1, 1).process(cluster.getSolrClient());
+    waitForState("Expected collection1 to be created with 2 shards and 1 replica", "collection1meta", clusterShape(2, 1));
+    waitForState("Expected collection2 to be created with 1 shard and 1 replica", "collection2meta", clusterShape(1, 1));
+    ZkStateReader zkStateReader = cluster.getSolrClient().getZkStateReader();
+    zkStateReader.createClusterStateWatchersAndUpdate();
+    List<String> aliases = zkStateReader.getAliases().resolveAliases(aliasName);
+    assertEquals(1, aliases.size());
+    assertEquals(aliasName, aliases.get(0));
+    UnaryOperator<Aliases> op6 = a -> a.cloneWithCollectionAlias(aliasName, "collection1meta,collection2meta");
+    final ZkStateReader.AliasesManager aliasesHolder = zkStateReader.aliasesHolder;
+
+    aliasesHolder.applyModificationAndExportToZk(op6);
+    aliases = zkStateReader.getAliases().resolveAliases(aliasName);
+    assertEquals(2, aliases.size());
+    assertEquals("collection1meta", aliases.get(0));
+    assertEquals("collection2meta", aliases.get(1));
+    return zkStateReader;
+  }
+
+  private void assertSuccess(HttpUriRequest msg) throws IOException {
+    try (CloseableHttpResponse response = httpClient.execute(msg)) {
+      if (200 != response.getStatusLine().getStatusCode()) {
+        System.err.println(EntityUtils.toString(response.getEntity()));
+        fail("Unexpected status: " + response.getStatusLine());
+      }
+    }
+  }
   // Rather a long title, but it's common to recommend when people need to re-index for any reason that they:
   // 1> create a new collection
   // 2> index the corpus to the new collection and verify it
@@ -497,22 +638,19 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
     ignoreException(".");
     
     // Invalid Alias name
-    SolrException e = expectThrows(SolrException.class, () -> {
-      CollectionAdminRequest.createAlias("test:alias", "testErrorChecks-collection").process(cluster.getSolrClient());
-    });
+    SolrException e = expectThrows(SolrException.class, () ->
+        CollectionAdminRequest.createAlias("test:alias", "testErrorChecks-collection").process(cluster.getSolrClient()));
     assertEquals(SolrException.ErrorCode.BAD_REQUEST, SolrException.ErrorCode.getErrorCode(e.code()));
 
     // Target collection doesn't exists
-    e = expectThrows(SolrException.class, () -> {
-      CollectionAdminRequest.createAlias("testalias", "doesnotexist").process(cluster.getSolrClient());
-    });
+    e = expectThrows(SolrException.class, () ->
+        CollectionAdminRequest.createAlias("testalias", "doesnotexist").process(cluster.getSolrClient()));
     assertEquals(SolrException.ErrorCode.BAD_REQUEST, SolrException.ErrorCode.getErrorCode(e.code()));
     assertTrue(e.getMessage().contains("Can't create collection alias for collections='doesnotexist', 'doesnotexist' is not an existing collection or alias"));
 
     // One of the target collections doesn't exist
-    e = expectThrows(SolrException.class, () -> {
-      CollectionAdminRequest.createAlias("testalias", "testErrorChecks-collection,doesnotexist").process(cluster.getSolrClient());
-    });
+    e = expectThrows(SolrException.class, () ->
+        CollectionAdminRequest.createAlias("testalias", "testErrorChecks-collection,doesnotexist").process(cluster.getSolrClient()));
     assertEquals(SolrException.ErrorCode.BAD_REQUEST, SolrException.ErrorCode.getErrorCode(e.code()));
     assertTrue(e.getMessage().contains("Can't create collection alias for collections='testErrorChecks-collection,doesnotexist', 'doesnotexist' is not an existing collection or alias"));
 
@@ -522,9 +660,8 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
     CollectionAdminRequest.createAlias("testalias2", "testalias").process(cluster.getSolrClient());
 
     // Alias + invalid
-    e = expectThrows(SolrException.class, () -> {
-      CollectionAdminRequest.createAlias("testalias3", "testalias2,doesnotexist").process(cluster.getSolrClient());
-    });
+    e = expectThrows(SolrException.class, () ->
+        CollectionAdminRequest.createAlias("testalias3", "testalias2,doesnotexist").process(cluster.getSolrClient()));
     assertEquals(SolrException.ErrorCode.BAD_REQUEST, SolrException.ErrorCode.getErrorCode(e.code()));
     unIgnoreException(".");
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/154bdeb7/solr/solr-ref-guide/src/collections-api.adoc
----------------------------------------------------------------------
diff --git a/solr/solr-ref-guide/src/collections-api.adoc b/solr/solr-ref-guide/src/collections-api.adoc
index b840e97..946a6ef 100644
--- a/solr/solr-ref-guide/src/collections-api.adoc
+++ b/solr/solr-ref-guide/src/collections-api.adoc
@@ -708,13 +708,64 @@ http://localhost:8983/solr/admin/collections?action=LISTALIASES&wt=xml
 [source,xml]
 ----
 <response>
+    <lst name="responseHeader">
+        <int name="status">0</int>
+        <int name="QTime">0</int>
+    </lst>
+    <lst name="aliases">
+        <str name="testalias1">collection1</str>
+        <str name="testalias2">collection1,collection2</str>
+    </lst>
+    <lst name="metadata">
+        <lst name="testalias1"/>
+        <lst name="testalias2">
+            <str name="someKey">someValue</str>
+        </lst>
+    </lst>
+</response>
+----
+
+[[modifyalias]]
+== MODIFYALIAS: Modify Alias Metadata for a Collection
+
+The `MODIFYALIAS` action modifies the metadata on an alias. If a key is set with a value that is empty it will be removed.
+
+`/admin/collections?action=MODIFYALIAS&name=_name_&metadata.someKey=somevalue`
+
+=== MODIFYALIAS Parameters
+
+`name`::
+The alias name on which to set metadata. This parameter is required.
+
+`metadata.*`::
+The name of the key for the metadata element to be modified replaces '*', the value for the parameter is passed as the value for the metadata.
+
+`async`::
+Request ID to track this action which will be <<Asynchronous Calls,processed asynchronously>>.
+
+=== MODIFYALIAS Response
+
+The output will simply be a responseHeader with details of the time it took to process the request. To confirm the creation of the metadata, you can look in the Solr Admin UI, under the Cloud section and find the `aliases.json` file or use the LISTALIASES api command.
+
+=== Examples using MODIFYALIAS
+
+*Input*
+
+For an alias named "testalias2" and set the value "someValue" for a metadata key of "someKey" and "otherValue" for "otherKey".
+
+[source,text]
+----
+http://localhost:8983/solr/admin/collections?action=MODIFYALIAS&name=testalias2&metadata.someKey=someValue&metadata.otherKey=otherValue&wt=xml
+----
+
+*Output*
+
+[source,xml]
+----
+<response>
   <lst name="responseHeader">
     <int name="status">0</int>
-    <int name="QTime">0</int>
-  </lst>
-  <lst name="aliases">
-    <str name="testalias1">collection1</str>
-    <str name="testalias2">collection2</str>
+    <int name="QTime">122</int>
   </lst>
 </response>
 ----

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/154bdeb7/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 e2e3540..3f2d800 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
@@ -17,6 +17,7 @@
 package org.apache.solr.client.solrj.request;
 
 import java.io.IOException;
+import java.util.HashMap;
 import java.util.Iterator;
 import java.util.Map;
 import java.util.Optional;
@@ -1319,6 +1320,40 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
 
   }
 
+  // MODIFYALIAS request
+
+  /**
+   * Returns a SolrRequest to add or remove metadata from a request
+   * @param aliasName         the alias to modify
+   */
+
+  public static ModifyAlias modifyAlias(String aliasName) {
+    return new ModifyAlias(aliasName);
+  }
+
+  public static class ModifyAlias extends AsyncCollectionAdminRequest {
+
+    private final String aliasName;
+    private Map<String,String> metadata = new HashMap<>();
+
+    public ModifyAlias(String aliasName) {
+      super(CollectionAction.MODIFYALIAS);
+      this.aliasName = SolrIdentifierValidator.validateAliasName(aliasName);
+    }
+
+    public void addMetadata(String key, String value) {
+      metadata.put(key,value);
+    }
+
+    @Override
+    public SolrParams getParams() {
+      ModifiableSolrParams params = (ModifiableSolrParams) super.getParams();
+      params.set(CoreAdminParams.NAME, aliasName);
+      metadata.forEach((key, value) ->  params.set("metadata." + key, value));
+      return params;
+    }
+  }
+
   /**
    * Returns a SolrRequest to create a new alias
    * @param aliasName           the alias name

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/154bdeb7/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 dc00119..56fe930 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
@@ -58,9 +58,9 @@ import static org.apache.solr.client.solrj.request.CollectionApiMapping.EndPoint
 import static org.apache.solr.common.params.CollectionParams.CollectionAction.*;
 import static org.apache.solr.common.params.CommonParams.NAME;
 
-/** stores the mapping of v1 API parameters to v2 API parameters
- * for collection API and configset API
- *
+/**
+ * Stores the mapping of v1 API parameters to v2 API parameters
+ * for the collection API and the configset API.
  */
 public class CollectionApiMapping {
 
@@ -135,6 +135,11 @@ public class CollectionApiMapping {
         DELETEALIAS,
         "delete-alias",
         null),
+    MODIFY_ALIAS(COLLECTIONS_COMMANDS,
+        POST,
+        MODIFYALIAS,
+        "modify-alias",
+        null),
     CREATE_SHARD(PER_COLLECTION_SHARDS_COMMANDS,
         POST,
         CREATESHARD,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/154bdeb7/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java b/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java
index 9f6a254..fe472e1 100644
--- a/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java
+++ b/solr/solrj/src/java/org/apache/solr/common/params/CollectionParams.java
@@ -78,6 +78,7 @@ public interface CollectionParams {
     CREATEALIAS(true, LockLevel.COLLECTION),
     CREATEROUTEDALIAS(true, LockLevel.COLLECTION),
     DELETEALIAS(true, LockLevel.COLLECTION),
+    MODIFYALIAS(true, LockLevel.COLLECTION),
     LISTALIASES(false, LockLevel.NONE),
     ROUTEDALIAS_CREATECOLL(true, LockLevel.COLLECTION),
     SPLITSHARD(true, LockLevel.SHARD),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/154bdeb7/solr/solrj/src/resources/apispec/collections.Commands.json
----------------------------------------------------------------------
diff --git a/solr/solrj/src/resources/apispec/collections.Commands.json b/solr/solrj/src/resources/apispec/collections.Commands.json
index 17172cc..0a88f6c 100644
--- a/solr/solrj/src/resources/apispec/collections.Commands.json
+++ b/solr/solrj/src/resources/apispec/collections.Commands.json
@@ -227,6 +227,29 @@
         "name"
       ]
     },
+    "modify-alias": {
+      "documentation": "https://lucene.apache.org/solr/guide/collections-api.html#modifyalias",
+      "description": "Allows changing the metadata on an alias. If a key is set with an empty string then it will be removed",
+      "type": "object",
+      "properties": {
+        "name": {
+          "type": "string",
+          "description": "The alias name on which to set metadata."
+        },
+        "metadata" : {
+          "type": "object",
+          "description": "A map of key/value pairs that will be associated with the alias as alias metadata. An empty value will delete any existing value for a given key.",
+          "additionalProperties": true
+        },
+        "async": {
+          "type": "string",
+          "description": "Defines a request ID that can be used to track this action after it's submitted. The action will be processed asynchronously."
+        }
+      },
+      "required": [
+        "name"
+      ]
+    },
     "backup-collection": {
       "documentation": "https://lucene.apache.org/solr/guide/collections-api.html#backup",
       "description": "Backup Solr indexes and configurations for a specific collection. One copy of the indexes will be taken from each shard, and the config set for the collection will also be copied.",