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/03/12 14:40:10 UTC

lucene-solr:branch_7_3: SOLR-11617: rename alias metadata to properties

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7_3 d77f618a6 -> 4a0d96974


SOLR-11617: rename alias metadata to properties

(cherry picked from commit 9957e0e)


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

Branch: refs/heads/branch_7_3
Commit: 4a0d96974b4d5ee1e68036c6b3782e4f3f2136b8
Parents: d77f618
Author: David Smiley <ds...@apache.org>
Authored: Mon Mar 12 10:36:37 2018 -0400
Committer: David Smiley <ds...@apache.org>
Committed: Mon Mar 12 10:40:04 2018 -0400

----------------------------------------------------------------------
 solr/CHANGES.txt                                |  4 +-
 .../cloud/api/collections/CreateAliasCmd.java   | 12 +--
 .../api/collections/MaintainRoutedAliasCmd.java |  2 +-
 .../cloud/api/collections/ModifyAliasCmd.java   | 81 ------------------
 .../OverseerCollectionMessageHandler.java       |  2 +-
 .../cloud/api/collections/SetAliasPropCmd.java  | 81 ++++++++++++++++++
 .../cloud/api/collections/TimeRoutedAlias.java  |  2 +-
 .../solr/handler/admin/CollectionsHandler.java  | 20 ++---
 .../TimeRoutedAliasUpdateProcessor.java         | 12 +--
 .../apache/solr/cloud/AliasIntegrationTest.java | 86 ++++++++++----------
 .../solr/cloud/CreateRoutedAliasTest.java       |  4 +-
 .../TimeRoutedAliasUpdateProcessorTest.java     |  4 +-
 solr/solr-ref-guide/src/collections-api.adoc    | 38 +++++----
 .../solrj/request/CollectionAdminRequest.java   | 22 ++---
 .../solrj/request/CollectionApiMapping.java     |  9 +-
 .../org/apache/solr/common/cloud/Aliases.java   | 72 ++++++++--------
 .../solr/common/params/CollectionParams.java    |  2 +-
 .../resources/apispec/collections.Commands.json | 10 +--
 18 files changed, 237 insertions(+), 226 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4a0d9697/solr/CHANGES.txt
----------------------------------------------------------------------
diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index 0667615..088ce69 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -111,8 +111,8 @@ 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)
+* SOLR-11617: Alias properties (formerly "metadata") are now mutable via a new ALIASPROP command.
+  These properties are returned from LISTALIASES. (Gus Heck via David Smiley)
 
 * SOLR-11702: Redesign current LIR implementation (Cao Manh Dat, shalin)
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4a0d9697/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java
index b89d752..7117019 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/CreateAliasCmd.java
@@ -108,12 +108,12 @@ public class CreateAliasCmd implements OverseerCollectionMessageHandler.Cmd {
       + " plus some create-collection prefixed ones.");
     }
 
-    Map<String, String> aliasMetadata = new LinkedHashMap<>();
+    Map<String, String> aliasProperties = new LinkedHashMap<>();
     message.getProperties().entrySet().stream()
-        .filter(entry -> TimeRoutedAlias.PARAM_IS_METADATA.test(entry.getKey()))
-        .forEach(entry -> aliasMetadata.put(entry.getKey(), (String) entry.getValue())); // way easier than .collect
+        .filter(entry -> TimeRoutedAlias.PARAM_IS_PROP.test(entry.getKey()))
+        .forEach(entry -> aliasProperties.put(entry.getKey(), (String) entry.getValue())); // way easier than .collect
 
-    TimeRoutedAlias timeRoutedAlias = new TimeRoutedAlias(aliasName, aliasMetadata); // validates as well
+    TimeRoutedAlias timeRoutedAlias = new TimeRoutedAlias(aliasName, aliasProperties); // validates as well
 
     String start = message.getStr(TimeRoutedAlias.ROUTER_START);
     Instant startTime = parseStart(start, timeRoutedAlias.getTimeZone());
@@ -121,13 +121,13 @@ public class CreateAliasCmd implements OverseerCollectionMessageHandler.Cmd {
     String initialCollectionName = TimeRoutedAlias.formatCollectionNameFromInstant(aliasName, startTime);
 
     // Create the collection
-    MaintainRoutedAliasCmd.createCollectionAndWait(state, aliasName, aliasMetadata, initialCollectionName, ocmh);
+    MaintainRoutedAliasCmd.createCollectionAndWait(state, aliasName, aliasProperties, initialCollectionName, ocmh);
     validateAllCollectionsExistAndNoDups(Collections.singletonList(initialCollectionName), zkStateReader);
 
     // Create/update the alias
     zkStateReader.aliasesManager.applyModificationAndExportToZk(aliases -> aliases
         .cloneWithCollectionAlias(aliasName, initialCollectionName)
-        .cloneWithCollectionAliasMetadata(aliasName, aliasMetadata));
+        .cloneWithCollectionAliasProperties(aliasName, aliasProperties));
   }
 
   private Instant parseStart(String str, TimeZone zone) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4a0d9697/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainRoutedAliasCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainRoutedAliasCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainRoutedAliasCmd.java
index d41a205..d5edfa6 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainRoutedAliasCmd.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/MaintainRoutedAliasCmd.java
@@ -103,7 +103,7 @@ public class MaintainRoutedAliasCmd implements OverseerCollectionMessageHandler.
     //---- PARSE ALIAS INFO FROM ZK
     final ZkStateReader.AliasesManager aliasesManager = ocmh.zkStateReader.aliasesManager;
     final Aliases aliases = aliasesManager.getAliases();
-    final Map<String, String> aliasMetadata = aliases.getCollectionAliasMetadata(aliasName);
+    final Map<String, String> aliasMetadata = aliases.getCollectionAliasProperties(aliasName);
     if (aliasMetadata == null) {
       throw newAliasMustExistException(aliasName); // if it did exist, we'd have a non-null map
     }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4a0d9697/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
deleted file mode 100644
index fa6a25a..0000000
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/ModifyAliasCmd.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * 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.aliasesManager.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/4a0d9697/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 d1ef0bd..8c512e0 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
@@ -227,7 +227,7 @@ public class OverseerCollectionMessageHandler implements OverseerMessageHandler,
         .put(DELETE, new DeleteCollectionCmd(this))
         .put(CREATEALIAS, new CreateAliasCmd(this))
         .put(DELETEALIAS, new DeleteAliasCmd(this))
-        .put(MODIFYALIAS, new ModifyAliasCmd(this))
+        .put(ALIASPROP, new SetAliasPropCmd(this))
         .put(MAINTAINROUTEDALIAS, new MaintainRoutedAliasCmd(this))
         .put(OVERSEERSTATUS, new OverseerStatusCmd(this))
         .put(DELETESHARD, new DeleteShardCmd(this))

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4a0d9697/solr/core/src/java/org/apache/solr/cloud/api/collections/SetAliasPropCmd.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/SetAliasPropCmd.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/SetAliasPropCmd.java
new file mode 100644
index 0000000..618b72d
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/SetAliasPropCmd.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 SetAliasPropCmd implements Cmd {
+
+  public static final String PROPERTIES = "property";
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final OverseerCollectionMessageHandler messageHandler;
+
+  SetAliasPropCmd(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> properties = (Map<String, String>) message.get(PROPERTIES);
+
+    zkStateReader.aliasesManager.applyModificationAndExportToZk(aliases1 -> {
+      for (Map.Entry<String, String> entry : properties.entrySet()) {
+        String key = entry.getKey();
+        if ("".equals(key.trim())) {
+          throw new SolrException(BAD_REQUEST, "property keys must not be pure whitespace");
+        }
+        if (!key.equals(key.trim())) {
+          throw new SolrException(BAD_REQUEST, "property keys should not begin or end with whitespace");
+        }
+        String value = entry.getValue();
+        if ("".equals(value)) {
+          value = null;
+        }
+        aliases1 = aliases1.cloneWithCollectionAliasProperties(aliasName, key, value);
+      }
+      return aliases1;
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4a0d9697/solr/core/src/java/org/apache/solr/cloud/api/collections/TimeRoutedAlias.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/cloud/api/collections/TimeRoutedAlias.java b/solr/core/src/java/org/apache/solr/cloud/api/collections/TimeRoutedAlias.java
index feb0334..5cbc625 100644
--- a/solr/core/src/java/org/apache/solr/cloud/api/collections/TimeRoutedAlias.java
+++ b/solr/core/src/java/org/apache/solr/cloud/api/collections/TimeRoutedAlias.java
@@ -84,7 +84,7 @@ public class TimeRoutedAlias {
       ROUTER_MAX_FUTURE,
       TZ)); // kinda special
 
-  static Predicate<String> PARAM_IS_METADATA =
+  static Predicate<String> PARAM_IS_PROP =
       key -> key.equals(TZ) ||
           (key.startsWith(ROUTER_PREFIX) && !key.equals(ROUTER_START)) || //TODO reconsider START special case
           key.startsWith(CREATE_COLLECTION_PREFIX);

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4a0d9697/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 b7d7b28..fa90b8f 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
@@ -571,18 +571,18 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
     DELETEALIAS_OP(DELETEALIAS, (req, rsp, h) -> req.getParams().required().getAll(null, NAME)),
 
     /**
-     * Change metadata for an alias (use CREATEALIAS_OP to change the actual value of the alias)
+     * Change properties for an alias (use CREATEALIAS_OP to change the actual value of the alias)
      */
-    MODIFYALIAS_OP(MODIFYALIAS, (req, rsp, h) -> {
+    ALIASPROP_OP(ALIASPROP, (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
+      // Note: success/no-op in the event of no properties supplied is intentional. Keeps code simple and one less case
       // for api-callers to check for.
-      return convertPrefixToMap(req.getParams(), params, "metadata");
+      return convertPrefixToMap(req.getParams(), params, "property");
     }),
 
     /**
-     * List the aliases and associated metadata.
+     * List the aliases and associated properties.
      */
     LISTALIASES_OP(LISTALIASES, (req, rsp, h) -> {
       ZkStateReader zkStateReader = h.coreContainer.getZkController().getZkStateReader();
@@ -590,15 +590,15 @@ public class CollectionsHandler extends RequestHandlerBase implements Permission
       if (aliases != null) {
         // the aliases themselves...
         rsp.getValues().add("aliases", aliases.getCollectionAliasMap());
-        // Any metadata for the above aliases.
+        // Any properties 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);
+          Map<String, String> collectionAliasProperties = aliases.getCollectionAliasProperties(alias);
+          if (collectionAliasProperties != null) {
+            meta.put(alias, collectionAliasProperties);
           }
         }
-        rsp.getValues().add("metadata", meta);
+        rsp.getValues().add("properties", meta);
       }
       return null;
     }),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4a0d9697/solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
----------------------------------------------------------------------
diff --git a/solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java b/solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
index b8ccbd3..224bd4b 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessor.java
@@ -63,7 +63,7 @@ import static org.apache.solr.update.processor.DistributingUpdateProcessorFactor
  * requests to create new collections on-demand.
  *
  * Depends on this core having a special core property that points to the alias name that this collection is a part of.
- * And further requires certain metadata on the Alias. Collections pointed to by the alias must be named for the alias
+ * And further requires certain properties on the Alias. Collections pointed to by the alias must be named for the alias
  * plus underscored ('_') and a time stamp of ISO_DATE plus optionally _HH_mm_ss. These collections should not be
  * created by the user, but are created automatically by the time partitioning system.
  *
@@ -124,14 +124,14 @@ public class TimeRoutedAliasUpdateProcessor extends UpdateRequestProcessor {
     cmdDistrib = new SolrCmdDistributor(cc.getUpdateShardHandler());
     collHandler = cc.getCollectionsHandler();
 
-    final Map<String, String> aliasMetadata = zkController.getZkStateReader().getAliases().getCollectionAliasMetadata(aliasName);
-    if (aliasMetadata == null) {
+    final Map<String, String> aliasProperties = zkController.getZkStateReader().getAliases().getCollectionAliasProperties(aliasName);
+    if (aliasProperties == null) {
       throw newAliasMustExistException(); // if it did exist, we'd have a non-null map
     }
     try {
-      this.timeRoutedAlias = new TimeRoutedAlias(aliasName, aliasMetadata);
+      this.timeRoutedAlias = new TimeRoutedAlias(aliasName, aliasProperties);
     } catch (Exception e) { // ensure we throw SERVER_ERROR not BAD_REQUEST at this stage
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Routed alias has invalid metadata: " + e, e);
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Routed alias has invalid properties: " + e, e);
     }
 
     ModifiableSolrParams outParams = new ModifiableSolrParams(req.getParams());
@@ -237,7 +237,7 @@ public class TimeRoutedAliasUpdateProcessor extends UpdateRequestProcessor {
 
   /**
    * Ensure {@link #parsedCollectionsAliases} is up to date. If it was modified, return true.
-   * Note that this will return true if some other alias was modified or if metadata was modified. These
+   * Note that this will return true if some other alias was modified or if properties were modified. These
    * are spurious and the caller should be written to be tolerant of no material changes.
    */
   private boolean updateParsedCollectionAliases() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4a0d9697/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 75d0591..2944a25 100644
--- a/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/AliasIntegrationTest.java
@@ -94,7 +94,7 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
 
   @Test
   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
-  public void testMetadata() throws Exception {
+  public void testProperties() throws Exception {
     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));
@@ -117,32 +117,32 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
     //noinspection unchecked
     assertTrue(((Map<String,Map<String,?>>)Utils.fromJSON(rawBytes)).get("collection").get("meta1") instanceof String);
 
-    // set metadata
-    UnaryOperator<Aliases> op5 = a -> a.cloneWithCollectionAliasMetadata("meta1", "foo", "bar");
+    // set properties
+    UnaryOperator<Aliases> op5 = a -> a.cloneWithCollectionAliasProperties("meta1", "foo", "bar");
     aliasesManager.applyModificationAndExportToZk(op5);
-    Map<String, String> meta = zkStateReader.getAliases().getCollectionAliasMetadata("meta1");
+    Map<String, String> meta = zkStateReader.getAliases().getCollectionAliasProperties("meta1");
     assertNotNull(meta);
     assertTrue(meta.containsKey("foo"));
     assertEquals("bar", meta.get("foo"));
 
-    // set more metadata
-    UnaryOperator<Aliases> op4 = a -> a.cloneWithCollectionAliasMetadata("meta1", "foobar", "bazbam");
+    // set more properties
+    UnaryOperator<Aliases> op4 = a -> a.cloneWithCollectionAliasProperties("meta1", "foobar", "bazbam");
     aliasesManager.applyModificationAndExportToZk(op4);
-    meta = zkStateReader.getAliases().getCollectionAliasMetadata("meta1");
+    meta = zkStateReader.getAliases().getCollectionAliasProperties("meta1");
     assertNotNull(meta);
 
-    // old metadata still there
+    // old properties still there
     assertTrue(meta.containsKey("foo"));
     assertEquals("bar", meta.get("foo"));
 
-    // new metadata added
+    // new properties added
     assertTrue(meta.containsKey("foobar"));
     assertEquals("bazbam", meta.get("foobar"));
 
-    // remove metadata
-    UnaryOperator<Aliases> op3 = a -> a.cloneWithCollectionAliasMetadata("meta1", "foo", null);
+    // remove properties
+    UnaryOperator<Aliases> op3 = a -> a.cloneWithCollectionAliasProperties("meta1", "foo", null);
     aliasesManager.applyModificationAndExportToZk(op3);
-    meta = zkStateReader.getAliases().getCollectionAliasMetadata("meta1");
+    meta = zkStateReader.getAliases().getCollectionAliasProperties("meta1");
     assertNotNull(meta);
 
     // verify key was removed
@@ -153,13 +153,13 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
     assertEquals("bazbam", meta.get("foobar"));
 
     // removal of non existent key should succeed.
-    UnaryOperator<Aliases> op2 = a -> a.cloneWithCollectionAliasMetadata("meta1", "foo", null);
+    UnaryOperator<Aliases> op2 = a -> a.cloneWithCollectionAliasProperties("meta1", "foo", null);
     aliasesManager.applyModificationAndExportToZk(op2);
 
     // chained invocations
     UnaryOperator<Aliases> op1 = a ->
-        a.cloneWithCollectionAliasMetadata("meta1", "foo2", "bazbam")
-        .cloneWithCollectionAliasMetadata("meta1", "foo3", "bazbam2");
+        a.cloneWithCollectionAliasProperties("meta1", "foo2", "bazbam")
+        .cloneWithCollectionAliasProperties("meta1", "foo3", "bazbam2");
     aliasesManager.applyModificationAndExportToZk(op1);
 
     // some other independent update (not overwritten)
@@ -169,17 +169,17 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
     // competing went through
     assertEquals("collection1meta,collection2meta", zkStateReader.getAliases().getCollectionAliasMap().get("meta3"));
 
-    meta = zkStateReader.getAliases().getCollectionAliasMetadata("meta1");
+    meta = zkStateReader.getAliases().getCollectionAliasProperties("meta1");
     assertNotNull(meta);
 
-    // old metadata still there
+    // old properties still there
     assertTrue(meta.containsKey("foobar"));
     assertEquals("bazbam", meta.get("foobar"));
 
     // competing update not overwritten
     assertEquals("collection1meta,collection2meta", zkStateReader.getAliases().getCollectionAliasMap().get("meta3"));
 
-    // new metadata added
+    // new properties added
     assertTrue(meta.containsKey("foo2"));
     assertEquals("bazbam", meta.get("foo2"));
     assertTrue(meta.containsKey("foo3"));
@@ -197,7 +197,7 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
       createdZKSR = true;
       zkStateReader.createClusterStateWatchersAndUpdate();
 
-      meta = zkStateReader.getAliases().getCollectionAliasMetadata("meta1");
+      meta = zkStateReader.getAliases().getCollectionAliasProperties("meta1");
       assertNotNull(meta);
 
       // verify key was removed in independent view
@@ -209,7 +209,7 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
 
       Aliases a = zkStateReader.getAliases();
       Aliases clone = a.cloneWithCollectionAlias("meta1", null);
-      meta = clone.getCollectionAliasMetadata("meta1");
+      meta = clone.getCollectionAliasProperties("meta1");
       assertEquals(0,meta.size());
     } finally {
       if (createdZKSR) {
@@ -218,16 +218,17 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
     }
   }
 
-  public void testModifyMetadataV2() throws Exception {
+  @Test
+  public void testModifyPropertiesV2() 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" +
+        "\"set-alias-property\" : {\n" +
         "  \"name\": \"" + aliasName + "\",\n" +
-        "  \"metadata\" : {\n" +
+        "  \"properties\" : {\n" +
         "    \"foo\": \"baz\",\n" +
         "    \"bar\": \"bam\"\n" +
         "    }\n" +
@@ -238,51 +239,52 @@ public class AliasIntegrationTest extends SolrCloudTestCase {
     checkFooAndBarMeta(aliasName, zkStateReader);
   }
 
-  public void testModifyMetadataV1() throws Exception {
+  @Test
+  public void testModifyPropertiesV1() 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" +
+    HttpGet get = new HttpGet(baseUrl + "/admin/collections?action=ALIASPROP" +
         "&wt=xml" +
         "&name=" + aliasName +
-        "&metadata.foo=baz" +
-        "&metadata.bar=bam");
+        "&property.foo=baz" +
+        "&property.bar=bam");
     assertSuccess(get);
     checkFooAndBarMeta(aliasName, zkStateReader);
   }
 
   @Test
   @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
-  public void testModifyMetadataCAR() throws Exception {
+  public void testModifyPropertiesCAR() 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());
+    CollectionAdminRequest.SetAliasProperty setAliasProperty = CollectionAdminRequest.setAliasProperty(aliasName);
+    setAliasProperty.addProperty("foo","baz");
+    setAliasProperty.addProperty("bar","bam");
+    setAliasProperty.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);
+    setAliasProperty = CollectionAdminRequest.setAliasProperty(aliasName);
+    setAliasProperty.addProperty("foo","");
+    setAliasProperty.process(cluster.getSolrClient());
+    setAliasProperty = CollectionAdminRequest.setAliasProperty(aliasName);
+    setAliasProperty.addProperty("bar",null);
+    setAliasProperty.process(cluster.getSolrClient());
+    setAliasProperty = CollectionAdminRequest.setAliasProperty(aliasName);
 
     // whitespace value
-    modifyAlias.addMetadata("foo"," ");
-    modifyAlias.process(cluster.getSolrClient());
+    setAliasProperty.addProperty("foo"," ");
+    setAliasProperty.process(cluster.getSolrClient());
 
 
   }
 
   private void checkFooAndBarMeta(String aliasName, ZkStateReader zkStateReader) throws Exception {
     zkStateReader.aliasesManager.update(); // ensure our view is up to date
-    Map<String, String> meta = zkStateReader.getAliases().getCollectionAliasMetadata(aliasName);
+    Map<String, String> meta = zkStateReader.getAliases().getCollectionAliasProperties(aliasName);
     assertNotNull(meta);
     assertTrue(meta.containsKey("foo"));
     assertEquals("baz", meta.get("foo"));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4a0d9697/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java b/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
index 78dc476..73be71e 100644
--- a/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/CreateRoutedAliasTest.java
@@ -169,7 +169,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
     Aliases aliases = cluster.getSolrClient().getZkStateReader().getAliases();
     Map<String, String> collectionAliasMap = aliases.getCollectionAliasMap();
     assertEquals(initialCollectionName, collectionAliasMap.get(aliasName));
-    Map<String, String> meta = aliases.getCollectionAliasMetadata(aliasName);
+    Map<String, String> meta = aliases.getCollectionAliasProperties(aliasName);
     //System.err.println(new TreeMap(meta));
     assertEquals("evt_dt",meta.get("router.field"));
     assertEquals("_default",meta.get("create-collection.collection.configName"));
@@ -214,7 +214,7 @@ public class CreateRoutedAliasTest extends SolrCloudTestCase {
     Map<String, String> collectionAliasMap = aliases.getCollectionAliasMap();
     String alias = collectionAliasMap.get(aliasName);
     assertNotNull(alias);
-    Map<String, String> meta = aliases.getCollectionAliasMetadata(aliasName);
+    Map<String, String> meta = aliases.getCollectionAliasProperties(aliasName);
     assertNotNull(meta);
     assertEquals("evt_dt",meta.get("router.field"));
     assertEquals("_default",meta.get("create-collection.collection.configName"));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4a0d9697/solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java b/solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
index 5df0437..ef8e1a5 100644
--- a/solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
+++ b/solr/core/src/test/org/apache/solr/update/processor/TimeRoutedAliasUpdateProcessorTest.java
@@ -202,8 +202,8 @@ public class TimeRoutedAliasUpdateProcessorTest extends SolrCloudTestCase {
 
 
     // update metadata to auto-delete oldest collections
-    CollectionAdminRequest.modifyAlias(alias)
-        .addMetadata(TimeRoutedAlias.ROUTER_AUTO_DELETE_AGE, "-1DAY")  // thus usually keep 2 collections of a day size
+    CollectionAdminRequest.setAliasProperty(alias)
+        .addProperty(TimeRoutedAlias.ROUTER_AUTO_DELETE_AGE, "-1DAY")  // thus usually keep 2 collections of a day size
         .process(solrClient);
 
     // add more docs, creating one new collection, but trigger ones prior to

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4a0d9697/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 022efe4..b2424c8 100644
--- a/solr/solr-ref-guide/src/collections-api.adoc
+++ b/solr/solr-ref-guide/src/collections-api.adoc
@@ -521,7 +521,7 @@ the data to shards based on customer id. Such shards can be of any type (NRT, PU
 placement strategies may also be used.
 
 The values supplied in this command for collection creation will be retained
-in alias metadata, and can be verified by inspecting `aliases.json` in ZooKeeper.
+in alias properties, and can be verified by inspecting `aliases.json` in ZooKeeper.
 
 *NOTE:* Presently only updates are routed and queries are distributed to all collections in the alias, but future
 features may enable routing of the query to the single appropriate collection based on a special parameter or perhaps
@@ -559,7 +559,7 @@ just fine. This parameter is required.
 `TZ`::
 The timezone to be used when evaluating any date math in router.start or router.interval. This is equivalent to the
 same parameter supplied to search queries, but understand in this case it's persisted with most of the other parameters
-as alias metadata.
+as an alias property.
 +
 If GMT-4 is supplied for this value then a document dated 2018-01-14T21:00:00:01.2345Z would be stored in the
 myAlias_2018-01-15_01 collection (assuming an interval of +1HOUR).
@@ -740,7 +740,7 @@ http://localhost:8983/solr/admin/collections?action=LISTALIASES&wt=xml
         <str name="testalias1">collection1</str>
         <str name="testalias2">collection1,collection2</str>
     </lst>
-    <lst name="metadata">
+    <lst name="properties">
         <lst name="testalias1"/>
         <lst name="testalias2">
             <str name="someKey">someValue</str>
@@ -749,37 +749,39 @@ http://localhost:8983/solr/admin/collections?action=LISTALIASES&wt=xml
 </response>
 ----
 
-[[modifyalias]]
-== MODIFYALIAS: Modify Alias Metadata for a Collection
+[[aliasprop]]
+== ALIASPROP: Modify Alias Properties 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.
+The `ALIASPROP` action modifies the properties (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`
+`/admin/collections?action=ALIASPROP&name=_name_&property.someKey=somevalue`
 
-=== MODIFYALIAS Parameters
+=== ALIASPROP Parameters
 
 `name`::
-The alias name on which to set metadata. This parameter is required.
+The alias name on which to set properties. 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.
+`property.*`::
+The name of the property to be modified replaces '*', the value for the parameter is passed as the value for the property.
 
 `async`::
 Request ID to track this action which will be <<Asynchronous Calls,processed asynchronously>>.
 
-=== MODIFYALIAS Response
+=== ALIASPROP 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.
+The output will simply be a responseHeader with details of the time it took to process the request.
+To confirm the creation of the property or properties, 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
+=== Examples using ALIASPROP
 
 *Input*
 
-For an alias named "testalias2" and set the value "someValue" for a metadata key of "someKey" and "otherValue" for "otherKey".
+For an alias named "testalias2" and set the value "someValue" for a property 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
+http://localhost:8983/solr/admin/collections?action=ALIASPROP&name=testalias2&property.someKey=someValue&property.otherKey=otherValue&wt=xml
 ----
 
 *Output*
@@ -809,7 +811,9 @@ Request ID to track this action which will be <<Asynchronous Calls,processed asy
 
 === DELETEALIAS Response
 
-The output will simply be a responseHeader with details of the time it took to process the request. To confirm the removal of the alias, you can look in the Solr Admin UI, under the Cloud section, and find the `aliases.json` file.
+The output will simply be a responseHeader with details of the time it took to process the request.
+To confirm the removal of the alias, you can look in the Solr Admin UI, under the Cloud section, and
+find the `aliases.json` file.
 
 === Examples using DELETEALIAS
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4a0d9697/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 1c4750e..a6a4f87 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
@@ -1320,29 +1320,29 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
 
   }
 
-  // MODIFYALIAS request
+  // ALIASPROP request
 
   /**
-   * Returns a SolrRequest to add or remove metadata from a request
+   * Returns a SolrRequest to add or remove properties from an alias
    * @param aliasName         the alias to modify
    */
 
-  public static ModifyAlias modifyAlias(String aliasName) {
-    return new ModifyAlias(aliasName);
+  public static SetAliasProperty setAliasProperty(String aliasName) {
+    return new SetAliasProperty(aliasName);
   }
 
-  public static class ModifyAlias extends AsyncCollectionAdminRequest {
+  public static class SetAliasProperty extends AsyncCollectionAdminRequest {
 
     private final String aliasName;
-    private Map<String,String> metadata = new HashMap<>();
+    private Map<String,String> properties = new HashMap<>();
 
-    public ModifyAlias(String aliasName) {
-      super(CollectionAction.MODIFYALIAS);
+    public SetAliasProperty(String aliasName) {
+      super(CollectionAction.ALIASPROP);
       this.aliasName = SolrIdentifierValidator.validateAliasName(aliasName);
     }
 
-    public ModifyAlias addMetadata(String key, String value) {
-      metadata.put(key,value);
+    public SetAliasProperty addProperty(String key, String value) {
+      properties.put(key,value);
       return this;
     }
 
@@ -1350,7 +1350,7 @@ public abstract class CollectionAdminRequest<T extends CollectionAdminResponse>
     public SolrParams getParams() {
       ModifiableSolrParams params = (ModifiableSolrParams) super.getParams();
       params.set(CoreAdminParams.NAME, aliasName);
-      metadata.forEach((key, value) ->  params.set("metadata." + key, value));
+      properties.forEach((key, value) ->  params.set("property." + key, value));
       return params;
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4a0d9697/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 f3d5ca5..7fd4efe 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
@@ -129,11 +129,12 @@ public class CollectionApiMapping {
         DELETEALIAS,
         "delete-alias",
         null),
-    MODIFY_ALIAS(COLLECTIONS_COMMANDS,
+    ALIAS_PROP(COLLECTIONS_COMMANDS,
         POST,
-        MODIFYALIAS,
-        "modify-alias",
-        null),
+        ALIASPROP,
+        "set-alias-property",
+        null,
+        Utils.makeMap("property.", "properties.")),
     CREATE_SHARD(PER_COLLECTION_SHARDS_COMMANDS,
         POST,
         CREATESHARD,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4a0d9697/solr/solrj/src/java/org/apache/solr/common/cloud/Aliases.java
----------------------------------------------------------------------
diff --git a/solr/solrj/src/java/org/apache/solr/common/cloud/Aliases.java b/solr/solrj/src/java/org/apache/solr/common/cloud/Aliases.java
index 14ccacc..671c27b 100644
--- a/solr/solrj/src/java/org/apache/solr/common/cloud/Aliases.java
+++ b/solr/solrj/src/java/org/apache/solr/common/cloud/Aliases.java
@@ -44,24 +44,28 @@ public class Aliases {
    */
   public static final Aliases EMPTY = new Aliases(Collections.emptyMap(), Collections.emptyMap(), 0);
 
+  // These two constants correspond to the top level elements in aliases.json. The first one denotes
+  // a section containing a list of aliases and their attendant collections, the second contains a list of
+  // aliases and their attendant properties (metadata) They probably should be
+  // named "aliases" and "alias_properties" but for back compat reasons, we cannot change them
   private static final String COLLECTION = "collection";
   private static final String COLLECTION_METADATA = "collection_metadata";
 
   // aliasName -> list of collections.  (note: the Lists here should be unmodifiable)
   private final Map<String, List<String>> collectionAliases; // not null
 
-  // aliasName --> metadataKey --> metadataValue (note: the inner Map here should be unmodifiable)
-  private final Map<String, Map<String, String>> collectionAliasMetadata; // notnull
+  // aliasName --> propertiesKey --> propertiesValue (note: the inner Map here should be unmodifiable)
+  private final Map<String, Map<String, String>> collectionAliasProperties; // notnull
 
   private final int zNodeVersion;
 
   /** Construct aliases directly with this information -- caller should not retain.
    * Any deeply nested collections are assumed to already be unmodifiable. */
   private Aliases(Map<String, List<String>> collectionAliases,
-                 Map<String, Map<String, String>> collectionAliasMetadata,
+                 Map<String, Map<String, String>> collectionAliasProperties,
                  int zNodeVersion) {
     this.collectionAliases = Objects.requireNonNull(collectionAliases);
-    this.collectionAliasMetadata = Objects.requireNonNull(collectionAliasMetadata);
+    this.collectionAliasProperties = Objects.requireNonNull(collectionAliasProperties);
     this.zNodeVersion = zNodeVersion;
   }
 
@@ -96,13 +100,13 @@ public class Aliases {
    */
   public byte[] toJSON() {
     if (collectionAliases.isEmpty()) {
-      assert collectionAliasMetadata.isEmpty();
+      assert collectionAliasProperties.isEmpty();
       return null;
     } else {
       Map<String,Map> tmp = new LinkedHashMap<>();
       tmp.put(COLLECTION, convertMapOfListToMapOfCommaDelimited(collectionAliases));
-      if (!collectionAliasMetadata.isEmpty()) {
-        tmp.put(COLLECTION_METADATA, collectionAliasMetadata);
+      if (!collectionAliasProperties.isEmpty()) {
+        tmp.put(COLLECTION_METADATA, collectionAliasProperties);
       }
       return Utils.toJSON(tmp);
     }
@@ -154,15 +158,15 @@ public class Aliases {
   }
 
   /**
-   * Returns an unmodifiable Map of metadata for a given alias. If an alias by the given name
+   * Returns an unmodifiable Map of properties for a given alias. If an alias by the given name
    * exists, this method will never return null.
    *
    * @param alias the name of an alias also found as a key in {@link #getCollectionAliasListMap()}
-   * @return The metadata for the alias (possibly empty) or null if the alias does not exist.
+   * @return The properties for the alias (possibly empty) or null if the alias does not exist.
    */
-  public Map<String,String> getCollectionAliasMetadata(String alias) {
+  public Map<String,String> getCollectionAliasProperties(String alias) {
     // Note: map is already unmodifiable; it can be shared safely
-    return collectionAliasMetadata.getOrDefault(alias, Collections.emptyMap());
+    return collectionAliasProperties.getOrDefault(alias, Collections.emptyMap());
   }
 
   /**
@@ -212,72 +216,72 @@ public class Aliases {
     if (alias == null) {
       throw new NullPointerException("Alias name cannot be null");
     }
-    Map<String, Map<String, String>> newColMetadata;
+    Map<String, Map<String, String>> newColProperties;
     Map<String, List<String>> newColAliases = new LinkedHashMap<>(this.collectionAliases);//clone to modify
     if (collections == null) { // REMOVE:
-      newColMetadata = new LinkedHashMap<>(this.collectionAliasMetadata);//clone to modify
-      newColMetadata.remove(alias);
+      newColProperties = new LinkedHashMap<>(this.collectionAliasProperties);//clone to modify
+      newColProperties.remove(alias);
       newColAliases.remove(alias);
     } else {
-      newColMetadata = this.collectionAliasMetadata;// no changes
+      newColProperties = this.collectionAliasProperties;// no changes
       // java representation is a list, so split before adding to maintain consistency
       newColAliases.put(alias, splitCollections(collections)); // note: unmodifiableList
     }
-    return new Aliases(newColAliases, newColMetadata, zNodeVersion);
+    return new Aliases(newColAliases, newColProperties, zNodeVersion);
   }
 
   /**
-   * Set the value for some metadata on a collection alias. This is done by creating a new Aliases instance
+   * Set the value for some properties on a collection alias. This is done by creating a new Aliases instance
    * with the same data as the current one but with a modification based on the parameters.
    * <p>
    * Note that the state in zookeeper is unaffected by this method and the change must still be persisted via
    * {@link ZkStateReader.AliasesManager#applyModificationAndExportToZk(UnaryOperator)}
    *
    * @param alias the alias to update
-   * @param metadataKey the key for the metadata
-   * @param metadataValue the metadata to add/replace, null to remove the key.
-   *                      @return An immutable copy of the aliases with the new metadata.
+   * @param propertiesKey the key for the properties
+   * @param propertiesValue the properties to add/replace, null to remove the key.
+   * @return An immutable copy of the aliases with the new properties.
    */
-  public Aliases cloneWithCollectionAliasMetadata(String alias, String metadataKey, String metadataValue) {
-    return cloneWithCollectionAliasMetadata(alias, Collections.singletonMap(metadataKey,metadataValue));
+  public Aliases cloneWithCollectionAliasProperties(String alias, String propertiesKey, String propertiesValue) {
+    return cloneWithCollectionAliasProperties(alias, Collections.singletonMap(propertiesKey,propertiesValue));
   }
 
   /**
-   * Set the values for some metadata keys on a collection alias. This is done by creating a new Aliases instance
+   * Set the values for some properties keys on a collection alias. This is done by creating a new Aliases instance
    * with the same data as the current one but with a modification based on the parameters.
    * <p>
    * Note that the state in zookeeper is unaffected by this method and the change must still be persisted via
    * {@link ZkStateReader.AliasesManager#applyModificationAndExportToZk(UnaryOperator)}
    *
    * @param alias the alias to update
-   * @param metadata the metadata to add/replace, null values in the map will remove the key.
-   * @return An immutable copy of the aliases with the new metadata.
+   * @param properties the properties to add/replace, null values in the map will remove the key.
+   * @return An immutable copy of the aliases with the new properties.
    */
-  public Aliases cloneWithCollectionAliasMetadata(String alias, Map<String,String> metadata) {
+  public Aliases cloneWithCollectionAliasProperties(String alias, Map<String,String> properties) {
     if (!collectionAliases.containsKey(alias)) {
       throw new IllegalArgumentException(alias + " is not a valid alias");
     }
-    if (metadata == null) {
-      throw new IllegalArgumentException("Null is not a valid metadata map");
+    if (properties == null) {
+      throw new IllegalArgumentException("Null is not a valid properties map");
     }
-    Map<String,Map<String,String>> newColMetadata = new LinkedHashMap<>(this.collectionAliasMetadata);//clone to modify
-    Map<String, String> newMetaMap = new LinkedHashMap<>(newColMetadata.getOrDefault(alias, Collections.emptyMap()));
-    for (Map.Entry<String, String> metaEntry : metadata.entrySet()) {
+    Map<String,Map<String,String>> newColProperties = new LinkedHashMap<>(this.collectionAliasProperties);//clone to modify
+    Map<String, String> newMetaMap = new LinkedHashMap<>(newColProperties.getOrDefault(alias, Collections.emptyMap()));
+    for (Map.Entry<String, String> metaEntry : properties.entrySet()) {
       if (metaEntry.getValue() != null) {
         newMetaMap.put(metaEntry.getKey(), metaEntry.getValue());
       } else {
         newMetaMap.remove(metaEntry.getKey());
       }
     }
-    newColMetadata.put(alias, Collections.unmodifiableMap(newMetaMap));
-    return new Aliases(collectionAliases, newColMetadata, zNodeVersion);
+    newColProperties.put(alias, Collections.unmodifiableMap(newMetaMap));
+    return new Aliases(collectionAliases, newColProperties, zNodeVersion);
   }
 
   @Override
   public String toString() {
     return "Aliases{" +
         "collectionAliases=" + collectionAliases +
-        ", collectionAliasMetadata=" + collectionAliasMetadata +
+        ", collectionAliasProperties=" + collectionAliasProperties +
         ", zNodeVersion=" + zNodeVersion +
         '}';
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4a0d9697/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 f9f3df8..2f3672d 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
@@ -77,7 +77,7 @@ public interface CollectionParams {
     SYNCSHARD(true, LockLevel.SHARD),
     CREATEALIAS(true, LockLevel.COLLECTION),
     DELETEALIAS(true, LockLevel.COLLECTION),
-    MODIFYALIAS(true, LockLevel.COLLECTION),
+    ALIASPROP(true, LockLevel.COLLECTION),
     LISTALIASES(false, LockLevel.NONE),
     MAINTAINROUTEDALIAS(true, LockLevel.COLLECTION),
     DELETEROUTEDALIASCOLLECTIONS(true, LockLevel.COLLECTION),

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4a0d9697/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 c845ee4..0268227 100644
--- a/solr/solrj/src/resources/apispec/collections.Commands.json
+++ b/solr/solrj/src/resources/apispec/collections.Commands.json
@@ -210,18 +210,18 @@
         "name"
       ]
     },
-    "modify-alias": {
+    "set-alias-property": {
       "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",
+      "description": "Allows changing the properties 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."
+          "description": "The alias name on which to set properties."
         },
-        "metadata" : {
+        "properties" : {
           "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.",
+          "description": "A map of key/value pairs that will be associated with the alias as alias properties (metadata). An empty value will delete any existing value for a given key.",
           "additionalProperties": true
         },
         "async": {