You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by th...@apache.org on 2021/07/30 16:35:13 UTC

[lucene-solr] branch branch_8x updated: SOLR-15277: Schema Designer screen for Admin UI and supporting backend API (#2538)

This is an automated email from the ASF dual-hosted git repository.

thelabdude pushed a commit to branch branch_8x
in repository https://gitbox.apache.org/repos/asf/lucene-solr.git


The following commit(s) were added to refs/heads/branch_8x by this push:
     new 5768e5a  SOLR-15277: Schema Designer screen for Admin UI and supporting backend API (#2538)
5768e5a is described below

commit 5768e5ab9afa0899725e0722c396b9b21acf0e16
Author: Timothy Potter <th...@gmail.com>
AuthorDate: Fri Jul 30 10:34:58 2021 -0600

    SOLR-15277: Schema Designer screen for Admin UI and supporting backend API (#2538)
---
 solr/CHANGES.txt                                   |    2 +
 .../java/org/apache/solr/core/CoreContainer.java   |    6 +
 .../designer/DefaultSampleDocumentsLoader.java     |  389 ++++
 .../handler/designer/DefaultSchemaSuggester.java   |  449 +++++
 .../solr/handler/designer/ManagedSchemaDiff.java   |  282 +++
 .../solr/handler/designer/SampleDocuments.java     |   68 +
 .../handler/designer/SampleDocumentsLoader.java    |   53 +
 .../solr/handler/designer/SchemaDesignerAPI.java   | 1224 ++++++++++++
 .../designer/SchemaDesignerConfigSetHelper.java    | 1078 +++++++++++
 .../handler/designer/SchemaDesignerConstants.java  |   50 +
 .../handler/designer/SchemaDesignerSettings.java   |  149 ++
 .../designer/SchemaDesignerSettingsDAO.java        |  175 ++
 .../solr/handler/designer/SchemaSuggester.java     |   42 +
 .../apache/solr/handler/designer/package-info.java |   26 +
 .../org/apache/solr/handler/loader/CSVLoader.java  |    2 +-
 .../apache/solr/handler/loader/CSVLoaderBase.java  |   12 +-
 .../org/apache/solr/handler/loader/JsonLoader.java |   70 +-
 .../ParseBooleanFieldUpdateProcessorFactory.java   |   39 +-
 .../ParseDateFieldUpdateProcessorFactory.java      |   63 +-
 .../ParseDoubleFieldUpdateProcessorFactory.java    |   42 +-
 .../ParseLongFieldUpdateProcessorFactory.java      |   39 +-
 .../test-files/schema-designer/add-new-field.json  |    9 +
 .../test-files/schema-designer/add-new-type.json   |   41 +
 .../schema-designer/update-author-field.json       |   11 +
 .../test-files/schema-designer/update-type.json    |   41 +
 .../solr/handler/TestSampleDocumentsLoader.java    |  123 ++
 .../handler/designer/ManagedSchemaDiffTest.java    |  117 ++
 .../handler/designer/TestSchemaDesignerAPI.java    |  874 +++++++++
 .../TestSchemaDesignerConfigSetHelper.java         |  360 ++++
 .../designer/TestSchemaDesignerSettingsDAO.java    |  126 ++
 .../src/documents-fields-and-schema-design.adoc    |    6 +-
 .../images/schema-designer/analyze-sample-docs.png |  Bin 0 -> 149120 bytes
 .../src/images/schema-designer/field-filters.png   |  Bin 0 -> 59884 bytes
 .../src/images/schema-designer/incompat-change.png |  Bin 0 -> 108134 bytes
 .../src/images/schema-designer/new-schema.png      |  Bin 0 -> 202830 bytes
 .../src/images/schema-designer/publish.png         |  Bin 0 -> 109754 bytes
 .../src/images/schema-designer/query-tester.png    |  Bin 0 -> 194928 bytes
 .../src/images/schema-designer/reload-schema.png   |  Bin 0 -> 37160 bytes
 .../src/images/schema-designer/schema-designer.png |  Bin 0 -> 892780 bytes
 .../schema-designer/schema-editor-fields.png       |  Bin 0 -> 203144 bytes
 .../images/schema-designer/schema-editor-root.png  |  Bin 0 -> 208229 bytes
 .../schema-designer/schema-editor-update.png       |  Bin 0 -> 264132 bytes
 .../src/images/schema-designer/show-changes.png    |  Bin 0 -> 153951 bytes
 .../src/images/schema-designer/text-analysis.png   |  Bin 0 -> 166359 bytes
 .../src/overview-of-the-solr-admin-ui.adoc         |   12 +
 solr/solr-ref-guide/src/schema-designer.adoc       |  236 +++
 solr/webapp/web/css/angular/menu.css               |    2 +
 solr/webapp/web/css/angular/schema-designer.css    | 1603 ++++++++++++++++
 solr/webapp/web/index.html                         |    4 +
 solr/webapp/web/js/angular/app.js                  |   31 +-
 solr/webapp/web/js/angular/controllers/cloud.js    |   10 +-
 .../web/js/angular/controllers/collections.js      |    3 +
 .../web/js/angular/controllers/schema-designer.js  | 1994 ++++++++++++++++++++
 solr/webapp/web/js/angular/services.js             |   11 +
 solr/webapp/web/partials/schema-designer.html      | 1033 ++++++++++
 55 files changed, 10773 insertions(+), 134 deletions(-)

diff --git a/solr/CHANGES.txt b/solr/CHANGES.txt
index dcd6dd3..75af006 100644
--- a/solr/CHANGES.txt
+++ b/solr/CHANGES.txt
@@ -15,6 +15,8 @@ New Features
 
 * SOLR-15208: Add the countDist aggregation to the stats, facet and timeseries Streaming Expressions (Joel Bernstein)
 
+* SOLR-15277: Schema Designer UI (Timothy Potter, Kiran Chitturi)
+
 Improvements
 ---------------------
 * SOLR-15460: Implement LIKE, IS NOT NULL, IS NULL, and support wildcard * in equals string literal for Parallel SQL (Timothy Potter, Houston Putman)
diff --git a/solr/core/src/java/org/apache/solr/core/CoreContainer.java b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
index 77c0346..1857e86 100644
--- a/solr/core/src/java/org/apache/solr/core/CoreContainer.java
+++ b/solr/core/src/java/org/apache/solr/core/CoreContainer.java
@@ -92,6 +92,7 @@ import org.apache.solr.handler.ClusterAPI;
 import org.apache.solr.handler.CollectionBackupsAPI;
 import org.apache.solr.handler.CollectionsAPI;
 import org.apache.solr.handler.RequestHandlerBase;
+import org.apache.solr.handler.designer.SchemaDesignerAPI;
 import org.apache.solr.handler.SnapShooter;
 import org.apache.solr.handler.admin.AutoscalingHistoryHandler;
 import org.apache.solr.handler.admin.CollectionsHandler;
@@ -743,6 +744,11 @@ public class CoreContainer {
     containerHandlers.getApiBag().registerObject(clusterAPI);
     containerHandlers.getApiBag().registerObject(clusterAPI.commands);
     containerHandlers.getApiBag().registerObject(clusterAPI.configSetCommands);
+
+    if (isZooKeeperAware()) {
+      containerHandlers.getApiBag().registerObject(new SchemaDesignerAPI(this));
+    } // else Schema Designer not available in standalone (non-cloud) mode
+
     /*
      * HealthCheckHandler needs to be initialized before InfoHandler, since the later one will call CoreContainer.getHealthCheckHandler().
      * We don't register the handler here because it'll be registered inside InfoHandler
diff --git a/solr/core/src/java/org/apache/solr/handler/designer/DefaultSampleDocumentsLoader.java b/solr/core/src/java/org/apache/solr/handler/designer/DefaultSampleDocumentsLoader.java
new file mode 100644
index 0000000..92cca70
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/designer/DefaultSampleDocumentsLoader.java
@@ -0,0 +1,389 @@
+/*
+ * 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.handler.designer;
+
+import javax.xml.stream.XMLInputFactory;
+import javax.xml.stream.XMLStreamConstants;
+import javax.xml.stream.XMLStreamException;
+import javax.xml.stream.XMLStreamReader;
+import java.io.BufferedReader;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Reader;
+import java.io.StringReader;
+import java.lang.invoke.MethodHandles;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.stream.Collectors;
+
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.SolrInputField;
+import org.apache.solr.common.params.ModifiableSolrParams;
+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.NamedList;
+import org.apache.solr.handler.loader.CSVLoaderBase;
+import org.apache.solr.handler.loader.JsonLoader;
+import org.apache.solr.handler.loader.XMLLoader;
+import org.apache.solr.request.SolrQueryRequestBase;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.update.processor.UpdateRequestProcessor;
+import org.apache.solr.util.SafeXMLParsing;
+import org.noggit.JSONParser;
+import org.noggit.ObjectBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.xml.sax.SAXException;
+
+import static org.apache.solr.common.params.CommonParams.JSON_MIME;
+import static org.apache.solr.handler.loader.CSVLoaderBase.SEPARATOR;
+
+public class DefaultSampleDocumentsLoader implements SampleDocumentsLoader {
+  public static final String CSV_MULTI_VALUE_DELIM_PARAM = "csvMultiValueDelimiter";
+  private static final int MAX_STREAM_SIZE = (5 * 1024 * 1024);
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  public static byte[] streamAsBytes(final InputStream in) throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    byte[] buf = new byte[1024];
+    int r;
+    try {
+      while ((r = in.read(buf)) != -1) baos.write(buf, 0, r);
+    } finally {
+      in.close();
+    }
+    return baos.toByteArray();
+  }
+
+  @Override
+  public SampleDocuments parseDocsFromStream(SolrParams params, ContentStream stream, final int maxDocsToLoad) throws IOException {
+    final String contentType = stream.getContentType();
+    if (contentType == null) {
+      return SampleDocuments.NONE;
+    }
+
+    if (params == null) {
+      params = new ModifiableSolrParams();
+    }
+
+    Long streamSize = stream.getSize();
+    if (streamSize != null && streamSize > MAX_STREAM_SIZE) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+          "Sample is too big! " + MAX_STREAM_SIZE + " bytes is the max upload size for sample documents.");
+    }
+
+    String fileSource = "paste";
+    if ("file".equals(stream.getName())) {
+      fileSource = stream.getSourceInfo() != null ? stream.getSourceInfo() : "file";
+    }
+
+    byte[] uploadedBytes = streamAsBytes(stream.getStream());
+    // recheck the upload size in case the stream returned null for getSize
+    if (uploadedBytes.length > MAX_STREAM_SIZE) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+          "Sample is too big! " + MAX_STREAM_SIZE + " bytes is the max upload size for sample documents.");
+    }
+    // use a byte stream for the parsers in case they need to re-parse using a different strategy
+    // e.g. JSON vs. JSON lines or different CSV strategies ...
+    ContentStreamBase.ByteArrayStream byteStream = new ContentStreamBase.ByteArrayStream(uploadedBytes, fileSource, contentType);
+    String charset = ContentStreamBase.getCharsetFromContentType(stream.getContentType());
+    if (charset == null) {
+      charset = ContentStreamBase.DEFAULT_CHARSET;
+    }
+
+    List<SolrInputDocument> docs = null;
+    if (stream.getSize() > 0) {
+      if (contentType.contains(JSON_MIME)) {
+        docs = loadJsonDocs(params, byteStream, maxDocsToLoad);
+      } else if (contentType.contains("text/xml") || contentType.contains("application/xml")) {
+        docs = loadXmlDocs(params, byteStream, maxDocsToLoad);
+      } else if (contentType.contains("text/csv") || contentType.contains("application/csv")) {
+        docs = loadCsvDocs(params, fileSource, uploadedBytes, charset, maxDocsToLoad);
+      } else if (contentType.contains("text/plain") || contentType.contains("application/octet-stream")) {
+        docs = loadJsonLines(params, byteStream, maxDocsToLoad);
+      } else {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, contentType + " not supported yet!");
+      }
+
+      if (docs != null && maxDocsToLoad > 0 && docs.size() > maxDocsToLoad) {
+        docs = docs.subList(0, maxDocsToLoad);
+      }
+    }
+
+    return new SampleDocuments(docs, contentType, fileSource);
+  }
+
+  protected List<SolrInputDocument> loadCsvDocs(SolrParams params, String source, byte[] streamBytes, String charset, final int maxDocsToLoad) throws IOException {
+    ContentStream stream;
+    if (params.get(SEPARATOR) == null) {
+      String csvStr = new String(streamBytes, charset);
+      char sep = detectTSV(csvStr);
+      ModifiableSolrParams modifiableSolrParams = new ModifiableSolrParams(params);
+      modifiableSolrParams.set(SEPARATOR, String.valueOf(sep));
+      params = modifiableSolrParams;
+      stream = new ContentStreamBase.StringStream(csvStr, "text/csv");
+    } else {
+      stream = new ContentStreamBase.ByteArrayStream(streamBytes, source, "text/csv");
+    }
+    return (new SampleCSVLoader(new CSVRequest(params), maxDocsToLoad)).loadDocs(stream);
+  }
+
+  @SuppressWarnings("unchecked")
+  protected List<SolrInputDocument> loadJsonLines(SolrParams params, ContentStreamBase.ByteArrayStream stream, final int maxDocsToLoad) throws IOException {
+    List<Map<String, Object>> docs = new LinkedList<>();
+    try (Reader r = stream.getReader()) {
+      BufferedReader br = new BufferedReader(r);
+      String line;
+      while ((line = br.readLine()) != null) {
+        line = line.trim();
+        if (!line.isEmpty() && line.startsWith("{") && line.endsWith("}")) {
+          Object jsonLine = ObjectBuilder.getVal(new JSONParser(line));
+          if (jsonLine instanceof Map) {
+            docs.add((Map<String, Object>) jsonLine);
+          }
+        }
+        if (maxDocsToLoad > 0 && docs.size() == maxDocsToLoad) {
+          break;
+        }
+      }
+    }
+
+    return docs.stream().map(JsonLoader::buildDoc).collect(Collectors.toList());
+  }
+
+  @SuppressWarnings("unchecked")
+  protected List<SolrInputDocument> loadJsonDocs(SolrParams params, ContentStreamBase.ByteArrayStream stream, final int maxDocsToLoad) throws IOException {
+    Object json;
+    try (Reader r = stream.getReader()) {
+      json = ObjectBuilder.getVal(new JSONParser(r));
+    }
+    if (json == null) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Expected at least 1 JSON doc in the request body!");
+    }
+
+    List<Map<String, Object>> docs;
+    if (json instanceof List) {
+      // list of docs
+      docs = (List<Map<String, Object>>) json;
+    } else if (json instanceof Map) {
+      // single doc ... see if this is a json lines file
+      boolean isJsonLines = false;
+      String charset = ContentStreamBase.getCharsetFromContentType(stream.getContentType());
+      String jsonStr = new String(streamAsBytes(stream.getStream()), charset != null ? charset : ContentStreamBase.DEFAULT_CHARSET);
+      String[] lines = jsonStr.split("\n");
+      if (lines.length > 1) {
+        for (String line : lines) {
+          line = line.trim();
+          if (!line.isEmpty() && line.startsWith("{") && line.endsWith("}")) {
+            isJsonLines = true;
+            break;
+          }
+        }
+      }
+      if (isJsonLines) {
+        docs = loadJsonLines(lines);
+      } else {
+        docs = Collections.singletonList((Map<String, Object>) json);
+      }
+    } else {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Expected one or more JSON docs in the request body!");
+    }
+    if (maxDocsToLoad > 0 && docs.size() > maxDocsToLoad) {
+      docs = docs.subList(0, maxDocsToLoad);
+    }
+    return docs.stream().map(JsonLoader::buildDoc).collect(Collectors.toList());
+  }
+
+  protected List<SolrInputDocument> loadXmlDocs(SolrParams params, ContentStreamBase.ByteArrayStream stream, final int maxDocsToLoad) throws IOException {
+    String xmlString = readInputAsString(stream.getStream()).trim();
+    List<SolrInputDocument> docs;
+    if (xmlString.contains("<add>") && xmlString.contains("<doc>")) {
+      XMLInputFactory inputFactory = XMLInputFactory.newInstance();
+      inputFactory.setProperty(XMLInputFactory.IS_SUPPORTING_EXTERNAL_ENTITIES, false);
+      inputFactory.setProperty(XMLInputFactory.SUPPORT_DTD, false);
+      XMLStreamReader parser = null;
+      try {
+        parser = inputFactory.createXMLStreamReader(new StringReader(xmlString));
+        docs = parseXmlDocs(parser, maxDocsToLoad);
+      } catch (XMLStreamException e) {
+        throw new IOException(e);
+      } finally {
+        if (parser != null) {
+          try {
+            parser.close();
+          } catch (XMLStreamException ignore) {
+          }
+        }
+      }
+    } else {
+      Document xmlDoc;
+      try {
+        xmlDoc = SafeXMLParsing.parseUntrustedXML(log, xmlString);
+      } catch (SAXException e) {
+        throw new IOException(e);
+      }
+      Element root = xmlDoc.getDocumentElement();
+      // TODO: support other types of XML here
+      throw new IOException("TODO: XML documents with root " + root.getTagName() + " not supported yet!");
+    }
+    return docs;
+  }
+
+  protected List<SolrInputDocument> parseXmlDocs(XMLStreamReader parser, final int maxDocsToLoad) throws XMLStreamException {
+    List<SolrInputDocument> docs = new LinkedList<>();
+    XMLLoader loader = new XMLLoader().init(null);
+    while (true) {
+      final int event;
+      try {
+        event = parser.next();
+      } catch (java.util.NoSuchElementException noSuchElementException) {
+        return docs;
+      }
+      switch (event) {
+        case XMLStreamConstants.END_DOCUMENT:
+          parser.close();
+          return docs;
+        case XMLStreamConstants.START_ELEMENT:
+          if ("doc".equals(parser.getLocalName())) {
+            SolrInputDocument doc = loader.readDoc(parser);
+            if (doc != null) {
+              docs.add(doc);
+
+              if (maxDocsToLoad > 0 && docs.size() >= maxDocsToLoad) {
+                parser.close();
+                return docs;
+              }
+            }
+          }
+      }
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  protected List<Map<String, Object>> loadJsonLines(String[] lines) throws IOException {
+    List<Map<String, Object>> docs = new ArrayList<>(lines.length);
+    for (String line : lines) {
+      line = line.trim();
+      if (!line.isEmpty() && line.startsWith("{") && line.endsWith("}")) {
+        Object jsonLine = ObjectBuilder.getVal(new JSONParser(line));
+        if (jsonLine instanceof Map) {
+          docs.add((Map<String, Object>) jsonLine);
+        }
+      }
+    }
+    return docs;
+  }
+
+  protected String readInputAsString(InputStream in) throws IOException {
+    return new String(streamAsBytes(in), StandardCharsets.UTF_8);
+  }
+
+  protected char detectTSV(String csvStr) {
+    char sep = ',';
+    int endOfFirstLine = csvStr.indexOf('\n');
+    if (endOfFirstLine != -1) {
+      int commas = 0;
+      int tabs = 0;
+      for (char value : csvStr.substring(0, endOfFirstLine).toCharArray()) {
+        if (value == ',') {
+          ++commas;
+        } else if (value == '\t') {
+          ++tabs;
+        }
+      }
+      if (tabs >= commas) {
+        sep = '\t';
+      }
+    }
+    return sep;
+  }
+
+  @Override
+  @SuppressWarnings({"rawtypes"})
+  public void init(NamedList args) {
+
+  }
+
+  private static class NoOpUpdateRequestProcessor extends UpdateRequestProcessor {
+    NoOpUpdateRequestProcessor() {
+      super(null);
+    }
+  }
+
+  private static class CSVRequest extends SolrQueryRequestBase {
+    CSVRequest(SolrParams params) {
+      super(null, params);
+    }
+  }
+
+  private static class SampleCSVLoader extends CSVLoaderBase {
+    List<SolrInputDocument> docs = new LinkedList<>();
+    CSVRequest req;
+    int maxDocsToLoad;
+    String multiValueDelimiter;
+
+    SampleCSVLoader(CSVRequest req, int maxDocsToLoad) {
+      super(req, new NoOpUpdateRequestProcessor());
+      this.req = req;
+      this.maxDocsToLoad = maxDocsToLoad;
+      this.multiValueDelimiter = req.getParams().get(CSV_MULTI_VALUE_DELIM_PARAM);
+    }
+
+    List<SolrInputDocument> loadDocs(ContentStream stream) throws IOException {
+      load(req, new SolrQueryResponse(), stream, processor);
+      return docs;
+    }
+
+    @Override
+    public void addDoc(int line, String[] vals) throws IOException {
+      if (maxDocsToLoad > 0 && docs.size() >= maxDocsToLoad) {
+        return; // just a short circuit, probably doesn't help that much
+      }
+
+      templateAdd.clear();
+      SolrInputDocument doc = new SolrInputDocument();
+      doAdd(line, vals, doc, templateAdd);
+      if (templateAdd.solrDoc != null) {
+        if (multiValueDelimiter != null) {
+          for (SolrInputField field : templateAdd.solrDoc.values()) {
+            if (field.getValueCount() == 1) {
+              Object value = field.getFirstValue();
+              if (value instanceof String) {
+                String[] splitValue = ((String) value).split(multiValueDelimiter);
+                if (splitValue.length > 1) {
+                  field.setValue(Arrays.asList(splitValue));
+                }
+              }
+            }
+          }
+        }
+        docs.add(templateAdd.solrDoc);
+      }
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/designer/DefaultSchemaSuggester.java b/solr/core/src/java/org/apache/solr/handler/designer/DefaultSchemaSuggester.java
new file mode 100644
index 0000000..704556e
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/designer/DefaultSchemaSuggester.java
@@ -0,0 +1,449 @@
+/*
+ * 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.handler.designer;
+
+import java.io.IOException;
+import java.math.RoundingMode;
+import java.text.NumberFormat;
+import java.text.ParsePosition;
+import java.time.ZoneId;
+import java.time.ZoneOffset;
+import java.time.format.DateTimeFormatter;
+import java.time.format.DateTimeFormatterBuilder;
+import java.time.format.ResolverStyle;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Set;
+import java.util.stream.Collectors;
+import java.util.stream.Stream;
+
+import com.google.common.collect.Sets;
+import org.apache.commons.lang3.LocaleUtils;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.ManagedIndexSchema;
+import org.apache.solr.schema.NumberType;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.schema.TextField;
+import org.apache.solr.update.processor.ParseBooleanFieldUpdateProcessorFactory;
+import org.apache.solr.update.processor.ParseDateFieldUpdateProcessorFactory;
+import org.apache.solr.update.processor.ParseDoubleFieldUpdateProcessorFactory;
+import org.apache.solr.update.processor.ParseLongFieldUpdateProcessorFactory;
+
+import static org.apache.solr.common.params.CommonParams.VERSION_FIELD;
+import static org.apache.solr.update.processor.ParseDateFieldUpdateProcessorFactory.validateFormatter;
+
+// Just a quick hack to flush out the design, more intelligence is needed
+public class DefaultSchemaSuggester implements SchemaSuggester {
+
+  private static final List<String> DEFAULT_DATE_TIME_PATTERNS =
+      Arrays.asList("yyyy-MM-dd['T'[HH:mm[:ss[.SSS]][z", "yyyy-MM-dd['T'[HH:mm[:ss[,SSS]][z", "yyyy-MM-dd HH:mm[:ss[.SSS]][z", "yyyy-MM-dd HH:mm[:ss[,SSS]][z", "[EEE, ]dd MMM yyyy HH:mm[:ss] z", "EEEE, dd-MMM-yy HH:mm:ss z", "EEE MMM ppd HH:mm:ss [z ]yyyy");
+
+  private static final String FORMATS_PARAM = "format";
+  private static final String DEFAULT_TIME_ZONE_PARAM = "defaultTimeZone";
+  private static final String LOCALE_PARAM = "locale";
+  private static final String TRUE_VALUES_PARAM = "trueValue";
+  private static final String FALSE_VALUES_PARAM = "falseValue";
+  private static final String CASE_SENSITIVE_PARAM = "caseSensitive";
+
+  private static final String TYPE_CHANGE_ERROR = "Failed to parse all sample values as %s for changing type for field %s to %s";
+
+  // boolean parsing
+  private final Set<String> trueValues = new HashSet<>(Arrays.asList("true"));
+  private final Set<String> falseValues = new HashSet<>(Arrays.asList("false"));
+  private final List<DateTimeFormatter> dateTimeFormatters = new LinkedList<>();
+  private boolean caseSensitive = false;
+
+  @Override
+  public void validateTypeChange(SchemaField field, FieldType toType, List<SolrInputDocument> docs) throws IOException {
+    final NumberType toNumType = toType.getNumberType();
+    if (toNumType != null) {
+      validateNumericTypeChange(field, toType, docs, toNumType);
+    }
+  }
+
+  protected void validateNumericTypeChange(SchemaField field, FieldType toType, List<SolrInputDocument> docs, final NumberType toNumType) {
+    // desired type is numeric, make sure all the sample values are numbers
+    List<Object> fieldValues = docs.stream()
+        .map(d -> d.getFieldValue(field.getName()))
+        .filter(Objects::nonNull)
+        .flatMap(c -> (c instanceof Collection) ? ((Collection<?>) c).stream() : Stream.of(c))
+        .collect(Collectors.toList());
+    switch (toNumType) {
+      case DOUBLE:
+      case FLOAT:
+        if (isFloatOrDouble(fieldValues, Locale.ROOT) == null) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+              String.format(Locale.ROOT, TYPE_CHANGE_ERROR, toNumType.name(), field.getName(), toType.getTypeName()));
+        }
+        break;
+      case LONG:
+      case INTEGER:
+        if (isIntOrLong(fieldValues, Locale.ROOT) == null) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+              String.format(Locale.ROOT, TYPE_CHANGE_ERROR, toNumType.name(), field.getName(), toType.getTypeName()));
+        }
+        break;
+      case DATE:
+        if (!isDateTime(fieldValues)) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+              String.format(Locale.ROOT, TYPE_CHANGE_ERROR, toNumType.name(), field.getName(), toType.getTypeName()));
+        }
+        break;
+    }
+  }
+
+  @Override
+  public Optional<SchemaField> suggestField(String fieldName, List<Object> sampleValues, IndexSchema schema, List<String> langs) {
+
+    // start by looking at the fieldName and seeing if there is a dynamic field in the schema that already applies
+    if (schema.isDynamicField(fieldName)) {
+      return Optional.of(schema.getFieldOrNull(fieldName));
+    }
+
+    // TODO: use passed in langs
+    Locale locale = Locale.ROOT;
+
+    boolean isMV = isMultiValued(sampleValues);
+    String fieldTypeName = guessFieldType(fieldName, sampleValues, schema, isMV, locale);
+    FieldType fieldType = schema.getFieldTypeByName(fieldTypeName);
+    if (fieldType == null) {
+      // TODO: construct this field type on-the-fly ...
+      throw new IllegalStateException("FieldType '" + fieldTypeName + "' not found in the schema!");
+    }
+
+    Map<String, String> fieldProps = guessFieldProps(fieldName, fieldType, sampleValues, isMV, schema);
+    SchemaField schemaField = schema.newField(fieldName, fieldTypeName, fieldProps);
+    return Optional.of(schemaField);
+  }
+
+  @Override
+  public ManagedIndexSchema adaptExistingFieldToData(SchemaField schemaField, List<Object> sampleValues, ManagedIndexSchema schema) {
+    // Promote a single-valued to multi-valued if needed
+    if (!schemaField.multiValued() && isMultiValued(sampleValues)) {
+      // this existing field needs to be promoted to multi-valued
+      SimpleOrderedMap<Object> fieldProps = schemaField.getNamedPropertyValues(false);
+      fieldProps.add("multiValued", true);
+      fieldProps.remove("name");
+      fieldProps.remove("type");
+      schema = schema.replaceField(schemaField.getName(), schemaField.getType(), fieldProps.asShallowMap());
+    }
+    // TODO: other "healing" type operations here ... but we have to be careful about overriding explicit user changes
+    // such as a user making a text field a string field, we wouldn't want to revert that field back to text
+    return schema;
+  }
+
+  @Override
+  public Map<String, List<Object>> transposeDocs(List<SolrInputDocument> docs) {
+    Map<String, List<Object>> mapByField = new HashMap<>();
+    docs.forEach(doc -> doc.getFieldNames().forEach(f -> {
+      // skip the version field on incoming docs
+      if (!VERSION_FIELD.equals(f)) {
+        List<Object> values = mapByField.computeIfAbsent(f, k -> new LinkedList<>());
+        Collection<Object> fieldValues = doc.getFieldValues(f);
+        if (fieldValues != null && !fieldValues.isEmpty()) {
+          if (fieldValues.size() == 1) {
+            // flatten so every field doesn't end up multi-valued
+            values.add(fieldValues.iterator().next());
+          } else {
+            // truly multi-valued
+            values.add(fieldValues);
+          }
+        }
+      }
+    }));
+    return mapByField;
+  }
+
+  protected String guessFieldType(String fieldName, final List<Object> sampleValues, IndexSchema schema, boolean isMV, Locale locale) {
+    String type = null;
+
+    // flatten values to a single stream for easier analysis; also remove nulls
+    List<Object> flattened = sampleValues.stream()
+        .flatMap(c -> (c instanceof Collection) ? ((Collection<?>) c).stream() : Stream.of(c))
+        .filter(Objects::nonNull)
+        .collect(Collectors.toList());
+
+    if (isBoolean(flattened)) {
+      type = isMV ? "booleans" : "boolean";
+    } else {
+      String intType = isIntOrLong(flattened, locale);
+      if (intType != null) {
+        type = isMV ? intType + "s" : intType;
+      } else {
+        String floatType = isFloatOrDouble(flattened, locale);
+        if (floatType != null) {
+          type = isMV ? floatType + "s" : floatType;
+        }
+      }
+    }
+
+    if (type == null) {
+      if (isDateTime(flattened)) {
+        type = isMV ? "pdates" : "pdate";
+      } else if (isText(flattened)) {
+        type = "en".equals(locale.getLanguage()) ? "text_en" : "text_general";
+      }
+    }
+
+    // if we get here and haven't made a decision, it's a string
+    if (type == null) {
+      type = isMV ? "strings" : "string";
+    }
+
+    return type;
+  }
+
+  protected boolean isText(List<Object> values) {
+    if (values == null || values.isEmpty()) {
+      return false;
+    }
+
+    int maxLength = -1;
+    int maxTerms = -1;
+    for (Object next : values) {
+      if (!(next instanceof String)) {
+        return false;
+      }
+
+      String cs = (String) next;
+      int len = cs.length();
+      if (len > maxLength) {
+        maxLength = len;
+      }
+
+      String[] terms = cs.split("\\s+");
+      if (terms.length > maxTerms) {
+        maxTerms = terms.length;
+      }
+    }
+
+    // don't want to choose text for fields where string will do
+    // if most of the sample values are unique but only a few terms, then it's likely a text field
+    return (maxLength > 60 || maxTerms > 12 || (maxTerms > 4 && values.size() >= 10 && ((float) Sets.newHashSet(values).size() / values.size()) > 0.9f));
+  }
+
+  protected String isFloatOrDouble(List<Object> values, Locale locale) {
+    NumberFormat format = NumberFormat.getInstance(locale);
+    format.setParseIntegerOnly(false);
+    format.setRoundingMode(RoundingMode.CEILING);
+    //boolean isFloat = true;
+    for (Object next : values) {
+      Object parsed = ParseDoubleFieldUpdateProcessorFactory.parsePossibleDouble(next, format);
+      if (parsed == null) {
+        // not a double ...
+        return null;
+      }
+
+      /*
+      Tried to be clever and pick pfloat if double precision is not needed, but the ParseDoubleFieldUpdateProcessorFactory
+      doesn't work with pfloat, so you don't get any locale sensitive parsing in the URP chain, so pdouble it is ...
+
+      Number num = (Number) parsed;
+      String str = num.toString();
+      int dotAt = str.indexOf('.');
+      if (dotAt != -1) {
+        String scalePart = str.substring(dotAt + 1);
+        if (scalePart.length() > 2) {
+          isFloat = false;
+        }
+      }
+       */
+    }
+
+    return "pdouble";
+  }
+
+  protected boolean isBoolean(List<Object> values) {
+    for (Object next : values) {
+      Object parsed = ParseBooleanFieldUpdateProcessorFactory.parsePossibleBoolean(next, caseSensitive, trueValues, falseValues);
+      if (parsed == null) {
+        return false;
+      }
+    }
+    // all values are booleans
+    return true;
+  }
+
+  protected String isIntOrLong(List<Object> values, Locale locale) {
+    NumberFormat format = NumberFormat.getInstance(locale);
+    format.setParseIntegerOnly(true);
+    long maxLong = Long.MIN_VALUE;
+    for (Object next : values) {
+      Object parsed = ParseLongFieldUpdateProcessorFactory.parsePossibleLong(next, format);
+      if (parsed == null) {
+        // not a long ...
+        return null;
+      } else {
+        long parsedLong = ((Number) parsed).longValue();
+        if (parsedLong > maxLong) {
+          maxLong = parsedLong;
+        }
+      }
+    }
+
+    // if all values are less than some smallish threshold, then it's likely this field holds small numbers
+    // but be very conservative here as it's simply an optimization and we can always fall back to long
+    return maxLong < 10000 ? "pint" : "plong";
+  }
+
+  protected boolean isDateTime(List<Object> values) {
+    if (dateTimeFormatters.isEmpty()) {
+      return false;
+    }
+
+    for (Object next : values) {
+      Object parsedDate = ParseDateFieldUpdateProcessorFactory.parsePossibleDate(next, dateTimeFormatters, new ParsePosition(0));
+      if (parsedDate == null) {
+        // not a date value
+        return false;
+      }
+    }
+    return true;
+  }
+
+  public boolean isMultiValued(String name, List<SolrInputDocument> docs) {
+    Map<String, List<Object>> transposed = transposeDocs(docs);
+    List<Object> sampleValues = transposed.get(name);
+    return sampleValues != null && isMultiValued(sampleValues);
+  }
+
+  protected boolean isMultiValued(final List<Object> sampleValues) {
+    for (Object next : sampleValues) {
+      if (next instanceof Collection) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  protected Map<String, String> guessFieldProps(String fieldName, FieldType fieldType, List<Object> sampleValues, boolean isMV, IndexSchema schema) {
+    Map<String, String> props = new HashMap<>();
+    props.put("indexed", "true");
+
+    if (isMV && !fieldType.isMultiValued()) {
+      props.put("multiValued", "true"); // override the mv setting on the type
+    }
+
+    boolean docValues = true;
+    if (fieldType instanceof TextField) {
+      docValues = false;
+    } else {
+      // Not sure if this field supports docValues, so try creating a SchemaField
+      Map<String, String> tmpProps = new HashMap<>(props);
+      tmpProps.put("docValues", "true"); // to test if docValues is supported
+      try {
+        fieldType.checkSchemaField(schema.newField(fieldName, fieldType.getTypeName(), tmpProps));
+      } catch (SolrException solrException) {
+        docValues = false;
+      }
+    }
+
+    props.put("docValues", String.valueOf(docValues));
+
+    if (!docValues) {
+      props.put("stored", "true");
+    } else {
+      props.put("stored", "false");
+      props.put("useDocValuesAsStored", "true");
+    }
+
+    return props;
+  }
+
+  @Override
+  @SuppressWarnings({"rawtypes"})
+  public void init(NamedList args) {
+    initDateTimeFormatters(args);
+    initBooleanParsing(args);
+  }
+
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  protected void initDateTimeFormatters(NamedList args) {
+
+    Locale locale = Locale.US;
+    String localeParam = (String) args.remove(LOCALE_PARAM);
+    if (null != localeParam) {
+      locale = LocaleUtils.toLocale(localeParam);
+    }
+
+    ZoneId defaultTimeZone = ZoneOffset.UTC;
+    Object defaultTimeZoneParam = args.remove(DEFAULT_TIME_ZONE_PARAM);
+    if (null != defaultTimeZoneParam) {
+      defaultTimeZone = ZoneId.of(defaultTimeZoneParam.toString());
+    }
+
+    Collection<String> dateTimePatterns = args.removeConfigArgs(FORMATS_PARAM);
+    if (dateTimePatterns == null || dateTimePatterns.isEmpty()) {
+      dateTimePatterns = DEFAULT_DATE_TIME_PATTERNS;
+    }
+
+    for (String pattern : dateTimePatterns) {
+      DateTimeFormatter formatter = new DateTimeFormatterBuilder().parseLenient().parseCaseInsensitive()
+          .appendPattern(pattern).toFormatter(locale).withResolverStyle(ResolverStyle.LENIENT).withZone(defaultTimeZone);
+      validateFormatter(formatter);
+      dateTimeFormatters.add(formatter);
+    }
+  }
+
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  protected void initBooleanParsing(NamedList args) {
+    Object caseSensitiveParam = args.remove(CASE_SENSITIVE_PARAM);
+    if (null != caseSensitiveParam) {
+      if (caseSensitiveParam instanceof Boolean) {
+        caseSensitive = (Boolean) caseSensitiveParam;
+      } else {
+        caseSensitive = Boolean.parseBoolean(caseSensitiveParam.toString());
+      }
+    }
+
+    Collection<String> trueValuesParam = args.removeConfigArgs(TRUE_VALUES_PARAM);
+    if (!trueValuesParam.isEmpty()) {
+      trueValues.clear();
+      for (String trueVal : trueValuesParam) {
+        trueValues.add(caseSensitive ? trueVal : trueVal.toLowerCase(Locale.ROOT));
+      }
+    }
+
+    Collection<String> falseValuesParam = args.removeConfigArgs(FALSE_VALUES_PARAM);
+    if (!falseValuesParam.isEmpty()) {
+      falseValues.clear();
+      for (String val : falseValuesParam) {
+        final String falseVal = caseSensitive ? val : val.toLowerCase(Locale.ROOT);
+        if (trueValues.contains(falseVal)) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+              "Param '" + FALSE_VALUES_PARAM + "' contains a value also in param '" + TRUE_VALUES_PARAM
+                  + "': '" + val + "'");
+        }
+        falseValues.add(falseVal);
+      }
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/designer/ManagedSchemaDiff.java b/solr/core/src/java/org/apache/solr/handler/designer/ManagedSchemaDiff.java
new file mode 100644
index 0000000..90b6823
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/designer/ManagedSchemaDiff.java
@@ -0,0 +1,282 @@
+/*
+ * 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.handler.designer;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import com.google.common.collect.MapDifference;
+import com.google.common.collect.Maps;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.ManagedIndexSchema;
+import org.apache.solr.schema.SchemaField;
+
+/**
+ * Utility methods for comparing managed index schemas
+ */
+public class ManagedSchemaDiff {
+
+  private static final String UPDATED_KEY_STRING = "updated";
+  private static final String ADDED_KEY_STRING = "added";
+  private static final String REMOVED_KEY_STRING = "removed";
+
+  private static final String FIELDS_KEY_STRING = "fields";
+  private static final String FIELD_TYPES_KEY_STRING = "fieldTypes";
+  private static final String DYNAMIC_FIELDS_KEY_STRING = "dynamicFields";
+  private static final String COPY_FIELDS_KEY_STRING = "copyFields";
+
+  /**
+   * Compute difference between two managed schemas. The returned map consists of changed, new, removed
+   * elements in fields, field types, dynamic fields and copy fields between input schemas.
+   *
+   * <pre> Output format when rendered to json will look like below:
+   *   {@code
+   *   {
+   *     "fields": {
+   *       "updated": {...},
+   *       "added": {...},
+   *       "removed": {...}
+   *     },
+   *     "fieldTypes": {
+   *       "updated": {...},
+   *       "added": {...},
+   *       "removed": {...}
+   *     },
+   *     "dynamicFields": {
+   *       "updated": {...},
+   *       "added": {...},
+   *       "removed": {...}
+   *     },
+   *     "copyFields: {
+   *       "new": [...],
+   *       "old": [...]
+   *     }
+   *   }
+   *   }
+   * </pre>
+   *
+   * @param oldSchema instance of {@link ManagedIndexSchema}
+   * @param newSchema instance of {@link ManagedIndexSchema}
+   * @return the difference between two schemas
+   */
+  public static Map<String, Object> diff(ManagedIndexSchema oldSchema, ManagedIndexSchema newSchema) {
+    Map<String, Object> diff = new HashMap<>();
+
+    Map<String, Object> fieldsDiff = diff(mapFieldsToPropertyValues(oldSchema.getFields()), mapFieldsToPropertyValues(newSchema.getFields()));
+    Map<String, Object> fieldTypesDiff = diff(mapFieldTypesToPropValues(oldSchema.getFieldTypes()), mapFieldTypesToPropValues(newSchema.getFieldTypes()));
+    Map<String, Object> dynamicFieldDiff = diff(mapDynamicFieldToPropValues(oldSchema.getDynamicFields()), mapDynamicFieldToPropValues(newSchema.getDynamicFields()));
+    Map<String, Object> copyFieldDiff = diff(getCopyFieldList(oldSchema), getCopyFieldList(newSchema));
+
+    if (!fieldsDiff.isEmpty()) {
+      diff.put(FIELDS_KEY_STRING, fieldsDiff);
+    }
+    if (!fieldTypesDiff.isEmpty()) {
+      diff.put(FIELD_TYPES_KEY_STRING, fieldTypesDiff);
+    }
+    if (!dynamicFieldDiff.isEmpty()) {
+      diff.put(DYNAMIC_FIELDS_KEY_STRING, dynamicFieldDiff);
+    }
+    if (!copyFieldDiff.isEmpty()) {
+      diff.put(COPY_FIELDS_KEY_STRING, copyFieldDiff);
+    }
+
+    return diff;
+  }
+
+  /**
+   * Compute difference between two map objects with {@link SimpleOrderedMap} as values.
+   *
+   * <pre> Example of the output format when rendered to json
+   *   {@code
+   *    {
+   *      "updated": {
+   *        "stringField": [
+   *        {
+   *          "docValues": "false"
+   *        },
+   *        {
+   *          "docValues": "true"
+   *        }
+   *      },
+   *      "added": {
+   *        "newstringfield: {
+   *          "name": "newstringfield",
+   *          "type": "string",
+   *          .....
+   *        }
+   *      },
+   *      "removed": {
+   *        "oldstringfield": {
+   *          "name": "oldstringfield",
+   *          "type": "string",
+   *          .....
+   *        }
+   *      }
+   *    }
+   *   }
+   * </pre>
+   *
+   * @param map1 instance of Map with {@link SimpleOrderedMap} elements
+   * @param map2 instance of Map with {@link SimpleOrderedMap} elements
+   * @return the difference between two Map
+   */
+  protected static Map<String, Object> diff(
+      Map<String, SimpleOrderedMap<Object>> map1,
+      Map<String, SimpleOrderedMap<Object>> map2) {
+    Map<String, List<Map<String, Object>>> changedValues = new HashMap<>();
+    Map<String, SimpleOrderedMap<Object>> newValues = new HashMap<>();
+    Map<String, SimpleOrderedMap<Object>> removedValues = new HashMap<>();
+    for (String fieldName : map1.keySet()) {
+      if (map2.containsKey(fieldName)) {
+        SimpleOrderedMap<Object> oldPropValues = map1.get(fieldName);
+        SimpleOrderedMap<Object> newPropValues = map2.get(fieldName);
+        if (!oldPropValues.equals(newPropValues)) {
+          List<Map<String, Object>> mapDiff = getMapDifference(oldPropValues, newPropValues);
+          if (!mapDiff.isEmpty()) {
+            changedValues.put(fieldName, mapDiff);
+          }
+        }
+      } else {
+        removedValues.put(fieldName, map1.get(fieldName));
+      }
+    }
+
+    for (String fieldName : map2.keySet()) {
+      if (!map1.containsKey(fieldName)) {
+        newValues.put(fieldName, map2.get(fieldName));
+      }
+    }
+
+    Map<String, Object> mapDiff = new HashMap<>();
+    if (!changedValues.isEmpty()) {
+      mapDiff.put(UPDATED_KEY_STRING, changedValues);
+    }
+    if (!newValues.isEmpty()) {
+      mapDiff.put(ADDED_KEY_STRING, newValues);
+    }
+    if (!removedValues.isEmpty()) {
+      mapDiff.put(REMOVED_KEY_STRING, removedValues);
+    }
+
+    return mapDiff;
+  }
+
+  /**
+   * Compute difference between two {@link SimpleOrderedMap} instances
+   *
+   * <pre> Output format example when rendered to json
+   *   {@code
+   *    [
+   *         {
+   *           "stored": false,
+   *           "type": "string",
+   *           "multiValued": false
+   *         },
+   *         {
+   *           "stored": true,
+   *           "type": "strings",
+   *           "multiValued": true
+   *         }
+   *       ]
+   *   }
+   * </pre>
+   *
+   * @param simpleOrderedMap1 Map to treat as "left" map
+   * @param simpleOrderedMap2 Map to treat as "right" map
+   * @return List containing the left diff and right diff
+   */
+  @SuppressWarnings("unchecked")
+  private static List<Map<String, Object>> getMapDifference(
+      SimpleOrderedMap<Object> simpleOrderedMap1,
+      SimpleOrderedMap<Object> simpleOrderedMap2) {
+    Map<String, Object> map1 = simpleOrderedMap1.toMap(new HashMap<>());
+    Map<String, Object> map2 = simpleOrderedMap2.toMap(new HashMap<>());
+    Map<String, MapDifference.ValueDifference<Object>> mapDiff = Maps.difference(map1, map2).entriesDiffering();
+    if (mapDiff.isEmpty()) {
+      return Collections.emptyList();
+    }
+    Map<String, Object> leftMapDiff = leftOrRightMapDiff(mapDiff, true);
+    Map<String, Object> rightMapDiff = leftOrRightMapDiff(mapDiff, false);
+    return Arrays.asList(leftMapDiff, rightMapDiff);
+  }
+
+  private static Map<String, Object> leftOrRightMapDiff(Map<String, MapDifference.ValueDifference<Object>> mapDiff, boolean left) {
+    Map<String, Object> leftMap = new HashMap<>(mapDiff.size() * 2);
+    for (Map.Entry<String, MapDifference.ValueDifference<Object>> e : mapDiff.entrySet()) {
+      Object value = left ? e.getValue().leftValue() : e.getValue().rightValue();
+      leftMap.put(e.getKey(), value);
+    }
+    return leftMap;
+  }
+
+  protected static Map<String, Object> diff(List<SimpleOrderedMap<Object>> list1, List<SimpleOrderedMap<Object>> list2) {
+    List<SimpleOrderedMap<Object>> oldList = new ArrayList<>(); // ordered map changed in list1 compared to list2
+    List<SimpleOrderedMap<Object>> newList = new ArrayList<>(); // ordered map changed in list2 compared to list1
+
+    list1.forEach(som -> {
+      if (!list2.contains(som)) {
+        oldList.add(som);
+      }
+    });
+
+    list2.forEach(som -> {
+      if (!list1.contains(som)) {
+        newList.add(som);
+      }
+    });
+
+    Map<String, Object> mapDiff = new HashMap<>();
+    if (!oldList.isEmpty()) {
+      mapDiff.put("old", oldList);
+    }
+    if (!newList.isEmpty()) {
+      mapDiff.put("new", newList);
+    }
+    return mapDiff;
+  }
+
+  protected static Map<String, SimpleOrderedMap<Object>> mapFieldsToPropertyValues(Map<String, SchemaField> fields) {
+    Map<String, SimpleOrderedMap<Object>> propValueMap = new HashMap<>();
+    fields.forEach((k, v) -> propValueMap.put(k, v.getNamedPropertyValues(true)));
+    return propValueMap;
+  }
+
+  protected static Map<String, SimpleOrderedMap<Object>> mapFieldTypesToPropValues(Map<String, FieldType> fieldTypes) {
+    Map<String, SimpleOrderedMap<Object>> propValueMap = new HashMap<>();
+    fieldTypes.forEach((k, v) -> propValueMap.put(k, v.getNamedPropertyValues(true)));
+    return propValueMap;
+  }
+
+  protected static Map<String, SimpleOrderedMap<Object>> mapDynamicFieldToPropValues(IndexSchema.DynamicField[] dynamicFields) {
+    Map<String, SimpleOrderedMap<Object>> map = new HashMap<>(dynamicFields.length * 2);
+    for (IndexSchema.DynamicField df : dynamicFields) {
+      map.put(df.getPrototype().getName(), df.getPrototype().getNamedPropertyValues(true));
+    }
+    return map;
+  }
+
+  protected static List<SimpleOrderedMap<Object>> getCopyFieldList(ManagedIndexSchema indexSchema) {
+    return indexSchema.getCopyFieldProperties(false, null, null);
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/designer/SampleDocuments.java b/solr/core/src/java/org/apache/solr/handler/designer/SampleDocuments.java
new file mode 100644
index 0000000..ee3cec8
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/designer/SampleDocuments.java
@@ -0,0 +1,68 @@
+/*
+ * 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.handler.designer;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Objects;
+import java.util.Set;
+import java.util.stream.Collectors;
+
+import org.apache.solr.common.SolrInputDocument;
+
+import static org.apache.solr.common.params.CommonParams.JSON_MIME;
+
+public class SampleDocuments {
+  public static final SampleDocuments NONE = new SampleDocuments(null, null, null);
+  public final String contentType;
+  public final String fileSource;
+  public List<SolrInputDocument> parsed;
+
+  public SampleDocuments(List<SolrInputDocument> parsed, String contentType, String fileSource) {
+    this.parsed = parsed != null ? parsed : new LinkedList<>(); // needs to be mutable
+    this.contentType = contentType;
+    this.fileSource = fileSource;
+  }
+
+  public String getSource() {
+    return fileSource != null ? fileSource : "paste";
+  }
+
+  private boolean isTextContentType() {
+    if (contentType == null) {
+      return false;
+    }
+    return contentType.contains(JSON_MIME) || contentType.startsWith("text/") || contentType.contains("application/xml");
+  }
+
+  public List<SolrInputDocument> appendDocs(String idFieldName, List<SolrInputDocument> add, int maxDocsToLoad) {
+    if (add != null && !add.isEmpty()) {
+      final Set<Object> ids =
+          parsed.stream().map(doc -> doc.getFieldValue(idFieldName)).filter(Objects::nonNull).collect(Collectors.toSet());
+      final List<SolrInputDocument> toAdd = add.stream().filter(doc -> {
+        Object id = doc.getFieldValue(idFieldName);
+        return id != null && !ids.contains(id); // doc has ID and it's not already in the set
+      }).collect(Collectors.toList());
+      parsed.addAll(toAdd);
+      if (maxDocsToLoad > 0 && parsed.size() > maxDocsToLoad) {
+        parsed = parsed.subList(0, maxDocsToLoad);
+      }
+    }
+    return parsed;
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/designer/SampleDocumentsLoader.java b/solr/core/src/java/org/apache/solr/handler/designer/SampleDocumentsLoader.java
new file mode 100644
index 0000000..0e0ba1c
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/designer/SampleDocumentsLoader.java
@@ -0,0 +1,53 @@
+/*
+ * 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.handler.designer;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Locale;
+import java.util.UUID;
+
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.util.ContentStream;
+import org.apache.solr.handler.designer.SampleDocuments;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.schema.StrField;
+import org.apache.solr.util.plugin.NamedListInitializedPlugin;
+
+public interface SampleDocumentsLoader extends NamedListInitializedPlugin {
+  SampleDocuments parseDocsFromStream(SolrParams params, ContentStream stream, int maxDocsToLoad) throws IOException;
+
+  /**
+   * Ensure every sample document has a unique ID, but only applies if the unique key field is a string.
+   */
+  default boolean ensureUniqueKey(final SchemaField idField, List<SolrInputDocument> docs) {
+    boolean updatedDocs = false;
+    // if the unique key field is a string, we can supply a UUID if needed, otherwise must come from the user.
+    if (StrField.class.equals(idField.getType().getClass())) {
+      String idFieldName = idField.getName();
+      for (SolrInputDocument d : docs) {
+        if (d.getFieldValue(idFieldName) == null) {
+          d.setField(idFieldName, UUID.randomUUID().toString().toLowerCase(Locale.ROOT));
+          updatedDocs = true;
+        }
+      }
+    }
+    return updatedDocs;
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerAPI.java b/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerAPI.java
new file mode 100644
index 0000000..2dc9942
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerAPI.java
@@ -0,0 +1,1224 @@
+/*
+ * 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.handler.designer;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Reader;
+import java.lang.invoke.MethodHandles;
+import java.nio.charset.StandardCharsets;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.LinkedHashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.stream.Collectors;
+
+import org.apache.solr.api.EndPoint;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.cloud.ZkSolrResourceLoader;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.SolrInputField;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.ZkConfigManager;
+import org.apache.solr.common.cloud.ZkMaintenanceUtils;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.util.ContentStream;
+import org.apache.solr.common.util.ContentStreamBase;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrConfig;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.RawResponseWriter;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.ManagedIndexSchema;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.util.RTimer;
+import org.apache.zookeeper.KeeperException;
+import org.noggit.JSONParser;
+import org.noggit.ObjectBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.GET;
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.POST;
+import static org.apache.solr.client.solrj.SolrRequest.METHOD.PUT;
+import static org.apache.solr.common.StringUtils.isEmpty;
+import static org.apache.solr.common.params.CommonParams.JSON_MIME;
+import static org.apache.solr.common.util.Utils.makeMap;
+import static org.apache.solr.handler.admin.ConfigSetsHandler.DEFAULT_CONFIGSET_NAME;
+import static org.apache.solr.schema.ManagedIndexSchemaFactory.DEFAULT_MANAGED_SCHEMA_RESOURCE_NAME;
+import static org.apache.solr.security.PermissionNameProvider.Name.CONFIG_EDIT_PERM;
+import static org.apache.solr.security.PermissionNameProvider.Name.CONFIG_READ_PERM;
+
+/**
+ * All V2 APIs that have a prefix of /api/schema-designer/
+ */
+public class SchemaDesignerAPI implements SchemaDesignerConstants {
+
+  private static final Set<String> excludeConfigSetNames = new HashSet<>(Arrays.asList(DEFAULT_CONFIGSET_NAME, BLOB_STORE_ID));
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final CoreContainer coreContainer;
+  private final SchemaSuggester schemaSuggester;
+  private final SampleDocumentsLoader sampleDocLoader;
+  private final SchemaDesignerSettingsDAO settingsDAO;
+  private final SchemaDesignerConfigSetHelper configSetHelper;
+  private final Map<String, Integer> indexedVersion = new ConcurrentHashMap<>();
+
+  public SchemaDesignerAPI(CoreContainer coreContainer) {
+    this(coreContainer, SchemaDesignerAPI.newSchemaSuggester(coreContainer), SchemaDesignerAPI.newSampleDocumentsLoader(coreContainer));
+  }
+
+  SchemaDesignerAPI(CoreContainer coreContainer, SchemaSuggester schemaSuggester, SampleDocumentsLoader sampleDocLoader) {
+    this.coreContainer = coreContainer;
+    this.schemaSuggester = schemaSuggester;
+    this.sampleDocLoader = sampleDocLoader;
+    this.settingsDAO = new SchemaDesignerSettingsDAO(coreContainer);
+    this.configSetHelper = new SchemaDesignerConfigSetHelper(this.coreContainer, this.schemaSuggester);
+  }
+
+  public static SchemaSuggester newSchemaSuggester(CoreContainer coreContainer) {
+    DefaultSchemaSuggester suggester = new DefaultSchemaSuggester();
+    suggester.init(new NamedList<>());
+    return suggester;
+  }
+
+  public static SampleDocumentsLoader newSampleDocumentsLoader(CoreContainer coreContainer) {
+    SampleDocumentsLoader loader = new DefaultSampleDocumentsLoader();
+    loader.init(new NamedList<>());
+    return loader;
+  }
+
+  static String getConfigSetZkPath(final String configSet) {
+    return getConfigSetZkPath(configSet, null);
+  }
+
+  static String getConfigSetZkPath(final String configSet, final String childNode) {
+    String path = ZkConfigManager.CONFIGS_ZKNODE + "/" + configSet;
+    if (childNode != null) {
+      path += "/" + childNode;
+    }
+    return path;
+  }
+
+  static String getMutableId(final String configSet) {
+    return DESIGNER_PREFIX + configSet;
+  }
+
+  @EndPoint(method = GET, path = "/schema-designer/info", permission = CONFIG_READ_PERM)
+  @SuppressWarnings("unchecked")
+  public void getInfo(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException {
+    final String configSet = getRequiredParam(CONFIG_SET_PARAM, req);
+
+    Map<String, Object> responseMap = new HashMap<>();
+    responseMap.put(CONFIG_SET_PARAM, configSet);
+    boolean exists = configExists(configSet);
+    responseMap.put("published", exists);
+
+    // mutable config may not exist yet as this is just an info check to gather some basic info the UI needs
+    String mutableId = getMutableId(configSet);
+
+    // tricky: if there's a mutable config, prefer the settings from that first but fallback to the original if not found
+    SolrConfig srcConfig = exists ? configSetHelper.loadSolrConfig(configSet) : null;
+    SolrConfig solrConfig = configExists(mutableId) ? configSetHelper.loadSolrConfig(mutableId) : srcConfig;
+    addSettingsToResponse(settingsDAO.getSettings(solrConfig), responseMap);
+
+    responseMap.put(SCHEMA_VERSION_PARAM, configSetHelper.getCurrentSchemaVersion(mutableId));
+    responseMap.put("collections", exists ? configSetHelper.listCollectionsForConfig(configSet) : Collections.emptyList());
+
+    // don't fail if loading sample docs fails
+    try {
+      responseMap.put("numDocs", configSetHelper.getStoredSampleDocs(configSet).size());
+    } catch (Exception exc) {
+      log.warn("Failed to load sample docs from blob store for {}", configSet, exc);
+    }
+
+    rsp.getValues().addAll(responseMap);
+  }
+
+  @EndPoint(method = POST, path = "/schema-designer/prep", permission = CONFIG_EDIT_PERM)
+  @SuppressWarnings("unchecked")
+  public void prepNewSchema(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException, SolrServerException {
+    final String configSet = getRequiredParam(CONFIG_SET_PARAM, req);
+    validateNewConfigSetName(configSet);
+
+    final String copyFrom = req.getParams().get(COPY_FROM_PARAM, DEFAULT_CONFIGSET_NAME);
+
+    SchemaDesignerSettings settings = getMutableSchemaForConfigSet(configSet, -1, copyFrom);
+    ManagedIndexSchema schema = settings.getSchema();
+    String mutableId = getMutableId(configSet);
+
+    // make sure the temp collection for this analysis exists
+    if (!zkStateReader().getClusterState().hasCollection(mutableId)) {
+      indexedVersion.remove(mutableId);
+      configSetHelper.createCollection(mutableId, mutableId);
+    }
+
+    settingsDAO.persistIfChanged(mutableId, settings);
+
+    rsp.getValues().addAll(buildResponse(configSet, schema, settings, null));
+  }
+
+  @EndPoint(method = PUT, path = "/schema-designer/cleanup", permission = CONFIG_EDIT_PERM)
+  public void cleanupTemp(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException, SolrServerException {
+    cleanupTemp(getRequiredParam(CONFIG_SET_PARAM, req));
+  }
+
+  @EndPoint(method = GET, path = "/schema-designer/file", permission = CONFIG_READ_PERM)
+  @SuppressWarnings("unchecked")
+  public void getFileContents(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException {
+    final String configSet = getRequiredParam(CONFIG_SET_PARAM, req);
+    final String file = getRequiredParam("file", req);
+    String filePath = getConfigSetZkPath(getMutableId(configSet), file);
+    byte[] data;
+    try {
+      data = zkStateReader().getZkClient().getData(filePath, null, null, true);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Error reading file: " + filePath, SolrZkClient.checkInterrupted(e));
+    }
+    String stringData = data != null && data.length > 0 ? new String(data, StandardCharsets.UTF_8) : "";
+    rsp.getValues().addAll(Collections.singletonMap(file, stringData));
+  }
+
+  @EndPoint(method = POST, path = "/schema-designer/file", permission = CONFIG_EDIT_PERM)
+  @SuppressWarnings("unchecked")
+  public void updateFileContents(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException, SolrServerException {
+    final String configSet = getRequiredParam(CONFIG_SET_PARAM, req);
+    final String file = getRequiredParam("file", req);
+
+    String mutableId = getMutableId(configSet);
+    String zkPath = getConfigSetZkPath(mutableId, file);
+    if (!pathExistsInZk(zkPath)) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "File '" + file + "' not found in configSet: " + configSet);
+    }
+
+    byte[] data = DefaultSampleDocumentsLoader.streamAsBytes(extractSingleContentStream(req, true).getStream());
+    Exception updateFileError = null;
+    if (SOLR_CONFIG_XML.equals(file)) {
+      // verify the updated solrconfig.xml is valid before saving to ZK (to avoid things blowing up later)
+      try {
+        InMemoryResourceLoader loader = new InMemoryResourceLoader(coreContainer, mutableId, SOLR_CONFIG_XML, data);
+        SolrConfig.readFromResourceLoader(loader, SOLR_CONFIG_XML, true, null);
+      } catch (Exception exc) {
+        updateFileError = exc;
+      }
+    }
+
+    if (updateFileError != null) {
+      // solrconfig.xml update failed, but haven't impacted the configSet yet, so just return the error directly
+      Throwable causedBy = SolrException.getRootCause(updateFileError);
+      Map<String, Object> response = new HashMap<>();
+      response.put("updateFileError", causedBy.getMessage());
+      response.put(file, new String(data, StandardCharsets.UTF_8));
+      rsp.getValues().addAll(response);
+      return;
+    }
+
+    // apply the update and reload the temp collection / re-index sample docs
+    SolrZkClient zkClient = zkStateReader().getZkClient();
+    try {
+      zkClient.setData(zkPath, data, true);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Failed to save data in ZK at path: " + zkPath, SolrZkClient.checkInterrupted(e));
+    }
+
+    configSetHelper.reloadTempCollection(mutableId, false);
+
+    ManagedIndexSchema schema = loadLatestSchema(mutableId);
+    Map<Object, Throwable> errorsDuringIndexing = null;
+    SolrException solrExc = null;
+    List<SolrInputDocument> docs = configSetHelper.getStoredSampleDocs(configSet);
+    String[] analysisErrorHolder = new String[1];
+    if (!docs.isEmpty()) {
+      String idField = schema.getUniqueKeyField().getName();
+      try {
+        errorsDuringIndexing =
+            indexSampleDocsWithRebuildOnAnalysisError(idField, docs, mutableId, true, analysisErrorHolder);
+      } catch (SolrException exc) {
+        solrExc = exc;
+      }
+    }
+
+    Map<String, Object> response = buildResponse(configSet, schema, null, docs);
+
+    if (analysisErrorHolder[0] != null) {
+      response.put(ANALYSIS_ERROR, analysisErrorHolder[0]);
+    }
+
+    addErrorToResponse(mutableId, solrExc, errorsDuringIndexing, response,
+        "Failed to re-index sample documents after update to the " + file + " file");
+
+    rsp.getValues().addAll(response);
+  }
+
+  @EndPoint(method = GET, path = "/schema-designer/sample", permission = CONFIG_READ_PERM)
+  @SuppressWarnings("unchecked")
+  public void getSampleValue(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException {
+    final String configSet = getRequiredParam(CONFIG_SET_PARAM, req);
+    final String fieldName = getRequiredParam(FIELD_PARAM, req);
+    final String idField = getRequiredParam(UNIQUE_KEY_FIELD_PARAM, req);
+    String docId = req.getParams().get(DOC_ID_PARAM);
+
+    final List<SolrInputDocument> docs = configSetHelper.getStoredSampleDocs(configSet);
+    String textValue = null;
+    if (isEmpty(docId)) {
+      // no doc ID from client ... find the first doc with a non-empty string value for fieldName
+      Optional<SolrInputDocument> doc = docs.stream()
+          .filter(d -> d.getField(fieldName) != null && d.getField(fieldName).getFirstValue() != null && !d.getField(fieldName).getFirstValue().toString().isEmpty())
+          .findFirst();
+      if (doc.isPresent()) {
+        docId = doc.get().getFieldValue(idField).toString();
+        textValue = doc.get().getField(fieldName).getFirstValue().toString();
+      }
+    } else {
+      final String idFilter = docId;
+      Optional<SolrInputDocument> doc = docs.stream().filter(d -> idFilter.equals(d.getFieldValue(idField))).findFirst();
+      if (doc.isPresent()) {
+        SolrInputField field = doc.get().getField(fieldName);
+        textValue = field != null && field.getFirstValue() != null ? field.getFirstValue().toString() : "";
+      }
+    }
+
+    if (textValue != null) {
+      Map<String, Object> analysis = configSetHelper.analyzeField(configSet, fieldName, textValue);
+      rsp.getValues().addAll(makeMap(idField, docId, fieldName, textValue, "analysis", analysis));
+    }
+  }
+
+  @EndPoint(method = GET, path = "/schema-designer/collectionsForConfig", permission = CONFIG_READ_PERM)
+  @SuppressWarnings("unchecked")
+  public void listCollectionsForConfig(SolrQueryRequest req, SolrQueryResponse rsp) {
+    final String configSet = getRequiredParam(CONFIG_SET_PARAM, req);
+    rsp.getValues().addAll(Collections.singletonMap("collections", configSetHelper.listCollectionsForConfig(configSet)));
+  }
+
+  // CONFIG_EDIT_PERM is required here since this endpoint is used by the UI to determine if the user has access to the Schema Designer UI
+  @EndPoint(method = GET, path = "/schema-designer/configs", permission = CONFIG_EDIT_PERM)
+  @SuppressWarnings("unchecked")
+  public void listConfigs(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException {
+    rsp.getValues().addAll(Collections.singletonMap("configSets", listEnabledConfigs()));
+  }
+
+  protected Map<String, Integer> listEnabledConfigs() throws IOException {
+    List<String> configsInZk = configSetHelper.listConfigsInZk();
+    final Map<String, Integer> configs = configsInZk.stream()
+        .filter(c -> !excludeConfigSetNames.contains(c) && !c.startsWith(DESIGNER_PREFIX))
+        .collect(Collectors.toMap(c -> c, c -> settingsDAO.isDesignerDisabled(c) ? 1 : 2));
+
+    // add the in-progress but drop the _designer prefix
+    configsInZk.stream().filter(c -> c.startsWith(DESIGNER_PREFIX))
+        .map(c -> c.substring(DESIGNER_PREFIX.length()))
+        .forEach(c -> configs.putIfAbsent(c, 0));
+
+    return configs;
+  }
+
+  @EndPoint(method = GET, path = "/schema-designer/download/*", permission = CONFIG_READ_PERM)
+  public void downloadConfig(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException {
+    final String configSet = getRequiredParam(CONFIG_SET_PARAM, req);
+    String mutableId = getMutableId(configSet);
+
+    // find the configSet to download
+    SolrZkClient zkClient = zkStateReader().getZkClient();
+    String configId = mutableId;
+    try {
+      if (!zkClient.exists(getConfigSetZkPath(mutableId, null), true)) {
+        if (zkClient.exists(getConfigSetZkPath(configSet, null), true)) {
+          configId = configSet;
+        } else {
+          throw new SolrException(SolrException.ErrorCode.NOT_FOUND, "ConfigSet " + configSet + " not found!");
+        }
+      }
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Error reading config from ZK", SolrZkClient.checkInterrupted(e));
+    }
+
+    ContentStreamBase content =
+        new ContentStreamBase.ByteArrayStream(
+            configSetHelper.downloadAndZipConfigSet(configId), configSet + ".zip", "application/zip");
+    rsp.add(RawResponseWriter.CONTENT, content);
+  }
+
+  @EndPoint(method = POST, path = "/schema-designer/add", permission = CONFIG_EDIT_PERM)
+  @SuppressWarnings("unchecked")
+  public void addSchemaObject(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException, SolrServerException {
+    final String configSet = getRequiredParam(CONFIG_SET_PARAM, req);
+    final String mutableId = checkMutable(configSet, req);
+
+    Map<String, Object> addJson = readJsonFromRequest(req);
+    log.info("Adding new schema object from JSON: {}", addJson);
+
+    String objectName = configSetHelper.addSchemaObject(configSet, addJson);
+    String action = addJson.keySet().iterator().next();
+
+    ManagedIndexSchema schema = loadLatestSchema(mutableId);
+    Map<String, Object> response = buildResponse(configSet, schema, null, configSetHelper.getStoredSampleDocs(configSet));
+    response.put(action, objectName);
+    rsp.getValues().addAll(response);
+  }
+
+  @EndPoint(method = PUT, path = "/schema-designer/update", permission = CONFIG_EDIT_PERM)
+  @SuppressWarnings("unchecked")
+  public void updateSchemaObject(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException, SolrServerException {
+    final String configSet = getRequiredParam(CONFIG_SET_PARAM, req);
+    final String mutableId = checkMutable(configSet, req);
+
+    // Updated field definition is in the request body as JSON
+    Map<String, Object> updateField = readJsonFromRequest(req);
+    String name = (String) updateField.get("name");
+    if (isEmpty(name)) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+          "Invalid update request! JSON payload is missing the required name property: " + updateField);
+    }
+    log.info("Updating schema object: configSet={}, mutableId={}, name={}, JSON={}", configSet, mutableId, name, updateField);
+
+    SchemaDesignerSettings settings = getMutableSchemaForConfigSet(configSet, -1, null);
+    ManagedIndexSchema schemaBeforeUpdate = settings.getSchema();
+
+    Map<String, Object> updateResult = configSetHelper.updateSchemaObject(configSet, updateField, schemaBeforeUpdate);
+    SolrException solrExc = (SolrException) updateResult.get("solrExc");
+    String updateError = (String) updateResult.get(UPDATE_ERROR);
+    String updateType = (String) updateResult.get("updateType");
+    boolean needsRebuild = (boolean) updateResult.get("rebuild");
+
+    // re-index the docs if no error to this point
+    final ManagedIndexSchema schema = loadLatestSchema(mutableId);
+    List<SolrInputDocument> docs = configSetHelper.getStoredSampleDocs(configSet);
+    Map<Object, Throwable> errorsDuringIndexing = null;
+    String[] analysisErrorHolder = new String[1];
+    if (solrExc == null && !docs.isEmpty()) {
+      try {
+        errorsDuringIndexing =
+            indexSampleDocsWithRebuildOnAnalysisError(schema.getUniqueKeyField().getName(), docs, mutableId, false, analysisErrorHolder);
+      } catch (SolrException exc) {
+        solrExc = exc;
+        updateError = "Failed to re-index sample documents after update to the " + name + " " + updateType + " due to: " + solrExc.getMessage();
+      }
+    }
+
+    Map<String, Object> response = buildResponse(configSet, schema, settings, docs);
+    response.put("updateType", updateType);
+    if (FIELD_PARAM.equals(updateType)) {
+      response.put(updateType, fieldToMap(schema.getField(name), schema));
+    } else if ("type".equals(updateType)) {
+      response.put(updateType, schema.getFieldTypeByName(name).getNamedPropertyValues(true));
+    }
+
+    if (analysisErrorHolder[0] != null) {
+      response.put(ANALYSIS_ERROR, analysisErrorHolder[0]);
+    }
+
+    addErrorToResponse(mutableId, solrExc, errorsDuringIndexing, response, updateError);
+
+    response.put("rebuild", needsRebuild);
+    rsp.getValues().addAll(response);
+  }
+
+  @EndPoint(method = PUT, path = "/schema-designer/publish", permission = CONFIG_EDIT_PERM)
+  @SuppressWarnings("unchecked")
+  public void publish(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException, SolrServerException {
+    final String configSet = getRequiredParam(CONFIG_SET_PARAM, req);
+    final String mutableId = checkMutable(configSet, req);
+
+    // verify the configSet we're going to apply changes to has not changed since being loaded for editing by the schema designer
+    SchemaDesignerSettings settings = settingsDAO.getSettings(mutableId);
+    final Optional<Integer> publishedVersion = settings.getPublishedVersion();
+    if (publishedVersion.isPresent()) {
+      int currentVersionOfSrc = configSetHelper.getCurrentSchemaVersion(configSet);
+      if (publishedVersion.get() != currentVersionOfSrc) {
+        throw new SolrException(SolrException.ErrorCode.CONFLICT,
+            "Version mismatch for " + configSet + "! Expected version " + publishedVersion.get() + " but current is " + currentVersionOfSrc +
+                "; another user may have changed the published schema while you were making edits. " +
+                "Publishing your changes would result in losing the edits from the other user.");
+      }
+    }
+
+    String newCollection = req.getParams().get(NEW_COLLECTION_PARAM);
+    if (!isEmpty(newCollection) && zkStateReader().getClusterState().hasCollection(newCollection)) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Collection '" + newCollection + "' already exists!");
+    }
+
+    if (configExists(configSet)) {
+      SolrZkClient zkClient = coreContainer.getZkController().getZkClient();
+      try {
+        zkClient.zkTransfer(getConfigSetZkPath(mutableId), true, getConfigSetZkPath(configSet), true, true);
+      } catch (KeeperException | InterruptedException e) {
+        throw new IOException("Failed to copy config set: " + mutableId, SolrZkClient.checkInterrupted(e));
+      }
+    } else {
+      copyConfig(mutableId, configSet);
+    }
+
+    boolean reloadCollections = req.getParams().getBool(RELOAD_COLLECTIONS_PARAM, false);
+    if (reloadCollections) {
+      log.debug("Reloading collections after update to configSet: {}", configSet);
+      List<String> collectionsForConfig = configSetHelper.listCollectionsForConfig(configSet);
+      CloudSolrClient csc = cloudClient();
+      for (String next : collectionsForConfig) {
+        CollectionAdminRequest.reloadCollection(next).processAsync(csc);
+      }
+    }
+
+    // create new collection
+    Map<Object, Throwable> errorsDuringIndexing = null;
+    if (!isEmpty(newCollection)) {
+      int numShards = req.getParams().getInt("numShards", 1);
+      int rf = req.getParams().getInt("replicationFactor", 1);
+      configSetHelper.createCollection(newCollection, configSet, numShards, rf);
+      if (req.getParams().getBool(INDEX_TO_COLLECTION_PARAM, false)) {
+        List<SolrInputDocument> docs = configSetHelper.getStoredSampleDocs(configSet);
+        if (!docs.isEmpty()) {
+          ManagedIndexSchema schema = loadLatestSchema(mutableId);
+          errorsDuringIndexing = indexSampleDocs(schema.getUniqueKeyField().getName(), docs, newCollection, true);
+        }
+      }
+    }
+
+    if (req.getParams().getBool(CLEANUP_TEMP_PARAM, true)) {
+      cleanupTemp(configSet);
+    }
+
+    settings.setDisabled(req.getParams().getBool(DISABLE_DESIGNER_PARAM, false));
+    settingsDAO.persistIfChanged(configSet, settings);
+
+    Map<String, Object> response = new HashMap<>();
+    response.put(CONFIG_SET_PARAM, configSet);
+    response.put(SCHEMA_VERSION_PARAM, configSetHelper.getCurrentSchemaVersion(configSet));
+    if (!isEmpty(newCollection)) {
+      response.put(NEW_COLLECTION_PARAM, newCollection);
+    }
+
+    addErrorToResponse(newCollection, null, errorsDuringIndexing, response, null);
+
+    rsp.getValues().addAll(response);
+  }
+
+  @EndPoint(method = POST, path = "/schema-designer/analyze", permission = CONFIG_EDIT_PERM)
+  @SuppressWarnings("unchecked")
+  public void analyze(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException, SolrServerException {
+    final int schemaVersion = req.getParams().getInt(SCHEMA_VERSION_PARAM, -1);
+    final String configSet = getRequiredParam(CONFIG_SET_PARAM, req);
+
+    // don't let the user edit the _default configSet with the designer (for now)
+    if (DEFAULT_CONFIGSET_NAME.equals(configSet)) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+          "'" + DEFAULT_CONFIGSET_NAME + "' is a reserved configSet name! Please choose a different name.");
+    }
+    validateNewConfigSetName(configSet);
+
+    // Get the sample documents to analyze, preferring those in the request but falling back to previously stored
+    SampleDocuments sampleDocuments = loadSampleDocuments(req, configSet);
+
+    // Get a mutable "temp" schema either from the specified copy source or configSet if it already exists.
+    String copyFrom = configExists(configSet) ? configSet
+        : req.getParams().get(COPY_FROM_PARAM, DEFAULT_CONFIGSET_NAME);
+
+    String mutableId = getMutableId(configSet);
+
+    // holds additional settings needed by the designer to maintain state
+    SchemaDesignerSettings settings = getMutableSchemaForConfigSet(configSet, schemaVersion, copyFrom);
+    ManagedIndexSchema schema = settings.getSchema();
+
+    String uniqueKeyFieldParam = req.getParams().get(UNIQUE_KEY_FIELD_PARAM);
+    if (!isEmpty(uniqueKeyFieldParam)) {
+      String uniqueKeyField = schema.getUniqueKeyField() != null ? schema.getUniqueKeyField().getName() : null;
+      if (!uniqueKeyFieldParam.equals(uniqueKeyField)) {
+        // The Schema API doesn't support changing the ID field so would have to use XML directly
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Changing the unique key field not supported yet!");
+      }
+    }
+
+    boolean langsUpdated = false;
+    String[] languages = req.getParams().getParams(LANGUAGES_PARAM);
+    List<String> langs;
+    if (languages != null) {
+      langs = languages.length == 0 || (languages.length == 1 && "*".equals(languages[0])) ? Collections.emptyList() : Arrays.asList(languages);
+      if (!langs.equals(settings.getLanguages())) {
+        settings.setLanguages(langs);
+        langsUpdated = true;
+      }
+    } else {
+      // nothing from client, go with what's stored in the settings ...
+      langs = settings.getLanguages();
+    }
+
+    boolean dynamicUpdated = false;
+    Boolean enableDynamicFields = req.getParams().getBool(ENABLE_DYNAMIC_FIELDS_PARAM);
+    if (enableDynamicFields != null && enableDynamicFields != settings.dynamicFieldsEnabled()) {
+      settings.setDynamicFieldsEnabled(enableDynamicFields);
+      dynamicUpdated = true;
+    }
+
+    if (langsUpdated) {
+      schema = configSetHelper.syncLanguageSpecificObjectsAndFiles(mutableId, schema, langs, settings.dynamicFieldsEnabled(), copyFrom);
+    }
+
+    if (dynamicUpdated) {
+      if (!enableDynamicFields) {
+        schema = configSetHelper.removeDynamicFields(schema);
+      } else {
+        schema = configSetHelper.restoreDynamicFields(schema, langs, copyFrom);
+      }
+    }
+
+    List<SolrInputDocument> docs = sampleDocuments.parsed;
+    if (!docs.isEmpty()) {
+      if (sampleDocLoader.ensureUniqueKey(schema.getUniqueKeyField(), docs)) {
+        configSetHelper.storeSampleDocs(configSet, docs);
+      }
+      schema = analyzeInputDocs(schemaSuggester.transposeDocs(docs), schema, langs);
+    }
+
+    // persist the updated schema
+    schema.persistManagedSchema(false);
+
+    Boolean enableFieldGuessing = req.getParams().getBool(ENABLE_FIELD_GUESSING_PARAM);
+    if (enableFieldGuessing != null && enableFieldGuessing != settings.fieldGuessingEnabled()) {
+      settings.setFieldGuessingEnabled(enableFieldGuessing);
+    }
+
+    // make sure the temp collection for this analysis exists
+    if (!zkStateReader().getClusterState().hasCollection(mutableId)) {
+      configSetHelper.createCollection(mutableId, mutableId);
+      indexedVersion.remove(mutableId);
+    } else {
+      // already created in the prep step ... reload it to pull in the updated schema
+      CollectionAdminRequest.reloadCollection(mutableId).process(cloudClient());
+    }
+
+    // nested docs
+    Boolean enableNestedDocs = req.getParams().getBool(ENABLE_NESTED_DOCS_PARAM);
+    if (enableNestedDocs != null && enableNestedDocs != settings.nestedDocsEnabled()) {
+      settings.setNestedDocsEnabled(enableNestedDocs);
+      configSetHelper.toggleNestedDocsFields(schema, enableNestedDocs);
+    }
+
+    // index the sample docs using the suggested schema
+    String[] analysisErrorHolder = new String[1];
+    Map<Object, Throwable> errorsDuringIndexing = null;
+    if (!docs.isEmpty()) {
+      String idField = schema.getUniqueKeyField().getName();
+      errorsDuringIndexing =
+          indexSampleDocsWithRebuildOnAnalysisError(idField, docs, mutableId, false, analysisErrorHolder);
+    }
+
+    if (settingsDAO.persistIfChanged(mutableId, settings)) {
+      CollectionAdminRequest.reloadCollection(mutableId).process(cloudClient());
+    }
+
+    Map<String, Object> response = buildResponse(configSet, loadLatestSchema(mutableId), settings, docs);
+    response.put("sampleSource", sampleDocuments.getSource());
+    if (analysisErrorHolder[0] != null) {
+      response.put(ANALYSIS_ERROR, analysisErrorHolder[0]);
+    }
+    addErrorToResponse(mutableId, null, errorsDuringIndexing, response, null);
+    rsp.getValues().addAll(response);
+  }
+
+  @EndPoint(method = GET, path = "/schema-designer/query", permission = CONFIG_READ_PERM)
+  @SuppressWarnings("unchecked")
+  public void query(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException, SolrServerException {
+    final String configSet = getRequiredParam(CONFIG_SET_PARAM, req);
+    String mutableId = getMutableId(configSet);
+    if (!configExists(mutableId)) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+          mutableId + " configSet not found! Are you sure " + configSet + " was being edited by the schema designer?");
+    }
+
+    if (!zkStateReader().getClusterState().hasCollection(mutableId)) {
+      indexedVersion.remove(mutableId);
+      configSetHelper.createCollection(mutableId, mutableId);
+    }
+
+    // only re-index if current state of test collection is not up-to-date
+    int currentVersion = configSetHelper.getCurrentSchemaVersion(mutableId);
+    Integer version = indexedVersion.get(mutableId);
+    Map<Object, Throwable> errorsDuringIndexing = null;
+    if (version == null || version != currentVersion) {
+      log.debug("Schema for collection {} is stale ({} != {}), need to re-index sample docs", mutableId, version, currentVersion);
+      List<SolrInputDocument> docs = configSetHelper.getStoredSampleDocs(configSet);
+      ManagedIndexSchema schema = loadLatestSchema(mutableId);
+      errorsDuringIndexing =
+          indexSampleDocsWithRebuildOnAnalysisError(schema.getUniqueKeyField().getName(), docs, mutableId, true, null);
+      // the version changes when you index (due to field guessing URP)
+      currentVersion = configSetHelper.getCurrentSchemaVersion(mutableId);
+      indexedVersion.put(mutableId, currentVersion);
+    }
+
+    if (errorsDuringIndexing != null) {
+      Map<String, Object> response = new HashMap<>();
+      rsp.setException(new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Failed to re-index sample documents after schema updated."));
+      response.put(ERROR_DETAILS, errorsDuringIndexing);
+      rsp.getValues().addAll(response);
+      return;
+    }
+
+    // execute the user's query against the temp collection
+    QueryResponse qr = cloudClient().query(mutableId, req.getParams());
+    rsp.getValues().addAll(qr.getResponse());
+  }
+
+  /**
+   * Return the diff of designer schema with the source schema (either previously published or the copyFrom).
+   */
+  @EndPoint(method = GET,
+      path = "/schema-designer/diff",
+      permission = CONFIG_READ_PERM)
+  @SuppressWarnings("unchecked")
+  public void getSchemaDiff(SolrQueryRequest req, SolrQueryResponse rsp) throws IOException {
+    final String configSet = getRequiredParam(CONFIG_SET_PARAM, req);
+
+    SchemaDesignerSettings settings = getMutableSchemaForConfigSet(configSet, -1, null);
+    // diff the published if found, else use the original source schema
+    String sourceSchema = configExists(configSet) ? configSet : settings.getCopyFrom();
+    Map<String, Object> response = new HashMap<>();
+    response.put("diff", ManagedSchemaDiff.diff(loadLatestSchema(sourceSchema), settings.getSchema()));
+    response.put("diff-source", sourceSchema);
+    addSettingsToResponse(settings, response);
+    rsp.getValues().addAll(response);
+  }
+
+  protected SampleDocuments loadSampleDocuments(SolrQueryRequest req, String configSet) throws IOException {
+    List<SolrInputDocument> docs = null;
+    ContentStream stream = extractSingleContentStream(req, false);
+    SampleDocuments sampleDocs = null;
+    if (stream != null && stream.getContentType() != null) {
+      sampleDocs = sampleDocLoader.parseDocsFromStream(req.getParams(), stream, MAX_SAMPLE_DOCS);
+      docs = sampleDocs.parsed;
+      if (!docs.isEmpty()) {
+        // user posted in some docs, if there are already docs stored in the blob store, then add these to the existing set
+        List<SolrInputDocument> stored = configSetHelper.getStoredSampleDocs(configSet);
+        if (!stored.isEmpty()) {
+          // keep the docs in the request as newest
+          ManagedIndexSchema latestSchema = loadLatestSchema(getMutableId(configSet));
+          docs = sampleDocs.appendDocs(latestSchema.getUniqueKeyField().getName(), stored, MAX_SAMPLE_DOCS);
+        }
+
+        // store in the blob store so we always have access to these docs
+        configSetHelper.storeSampleDocs(configSet, docs);
+      }
+    }
+
+    if (docs == null || docs.isEmpty()) {
+      // no sample docs in the request ... find in blob store (or fail if no docs previously stored)
+      docs = configSetHelper.getStoredSampleDocs(configSet);
+
+      // no docs? but if this schema has already been published, it's OK, we can skip the docs part
+      if (docs.isEmpty() && !configExists(configSet)) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            "No sample documents provided for analyzing schema! Only CSV/TSV, XML, JSON, and JSON lines supported.");
+      }
+
+      sampleDocs = new SampleDocuments(docs, "", "blob");
+    }
+
+    return sampleDocs;
+  }
+
+  protected ManagedIndexSchema analyzeInputDocs(final Map<String, List<Object>> docs, ManagedIndexSchema schema, List<String> langs) {
+    // collect the fields to add ... adding all fields at once is faster than one-at-a-time
+    List<SchemaField> fieldsToAdd = new LinkedList<>();
+    for (String field : docs.keySet()) {
+      List<Object> sampleValues = docs.getOrDefault(field, Collections.emptyList());
+
+      // Collapse all whitespace in fields to a single underscore
+      String normalizedField = field.trim().replaceAll("\\s+", "_");
+      if (schema.hasExplicitField(normalizedField)) {
+        SchemaField existing = schema.getField(normalizedField);
+        schema = schemaSuggester.adaptExistingFieldToData(existing, sampleValues, schema);
+        continue;
+      }
+
+      Optional<SchemaField> maybeSchemaField = schemaSuggester.suggestField(normalizedField, sampleValues, schema, langs);
+      maybeSchemaField.ifPresent(fieldsToAdd::add);
+    }
+
+    if (!fieldsToAdd.isEmpty()) {
+      schema = (ManagedIndexSchema) schema.addFields(fieldsToAdd);
+    }
+    
+    return schema;
+  }
+
+  protected String getManagedSchemaZkPath(final String configSet) {
+    return getConfigSetZkPath(configSet, DEFAULT_MANAGED_SCHEMA_RESOURCE_NAME);
+  }
+
+  protected SchemaDesignerSettings getMutableSchemaForConfigSet(final String configSet, final int schemaVersion, String copyFrom) throws IOException {
+    // The designer works with mutable config sets stored in a "temp" znode in ZK instead of the "live" configSet
+    final String mutableId = getMutableId(configSet);
+
+    ManagedIndexSchema schema;
+
+    // create new from the built-in "_default" configSet
+    int publishedVersion = -1;
+    boolean isNew = false;
+    if (!configExists(mutableId)) {
+
+      // are they opening a temp of an existing?
+      if (configExists(configSet)) {
+        if (settingsDAO.isDesignerDisabled(configSet)) {
+          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+              "Schema '" + configSet + "' is locked for edits by the schema designer!");
+        }
+        publishedVersion = configSetHelper.getCurrentSchemaVersion(configSet);
+        // ignore the copyFrom as we're making a mutable temp copy of an already published configSet
+        copyConfig(configSet, mutableId);
+        copyFrom = null;
+      } else {
+        copyConfig(copyFrom, mutableId);
+      }
+      isNew = true;
+    }
+
+    SolrConfig solrConfig = configSetHelper.loadSolrConfig(mutableId);
+    schema = configSetHelper.loadLatestSchema(solrConfig);
+    if (!isNew) {
+      // schema is not new, so the provided version must match, otherwise, we're trying to edit dirty data
+      configSetHelper.checkSchemaVersion(mutableId, schemaVersion, schema.getSchemaZkVersion());
+    }
+
+    SchemaDesignerSettings settings = settingsDAO.getSettings(solrConfig);
+    if (isNew) {
+      if (!configSet.equals(copyFrom)) {
+        settings.setDisabled(false);
+      }
+
+      // remember where this new one came from
+      if (copyFrom != null) {
+        settings.setCopyFrom(copyFrom);
+      }
+
+      if (publishedVersion != -1) {
+        // keep track of the version of the configSet the mutable is derived from in case another user
+        // changes the derived from schema before we publish the mutable on top of it
+        settings.setPublishedVersion(publishedVersion);
+      }
+
+      if (!settings.nestedDocsEnabled()) {
+        schema = configSetHelper.deleteNestedDocsFieldsIfNeeded(schema, false);
+      }
+
+      if (!settings.dynamicFieldsEnabled()) {
+        schema = configSetHelper.removeDynamicFields(schema);
+      }
+
+      schema.persistManagedSchema(false);
+    }
+
+    settings.setSchema(schema);
+
+    return settings;
+  }
+
+  ManagedIndexSchema loadLatestSchema(String configSet) {
+    return configSetHelper.loadLatestSchema(configSet);
+  }
+
+  protected ContentStream extractSingleContentStream(final SolrQueryRequest req, boolean required) {
+    Iterable<ContentStream> streams = req.getContentStreams();
+    Iterator<ContentStream> iter = streams != null ? streams.iterator() : null;
+    ContentStream stream = iter != null && iter.hasNext() ? iter.next() : null;
+    if (required && stream == null)
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "No JSON content found in the request body!");
+
+    return stream;
+  }
+
+  protected CloudSolrClient cloudClient() {
+    return coreContainer.getSolrClientCache().getCloudSolrClient(coreContainer.getZkController().getZkServerAddress());
+  }
+
+  protected ZkStateReader zkStateReader() {
+    return coreContainer.getZkController().getZkStateReader();
+  }
+
+  protected Map<Object, Throwable> indexSampleDocsWithRebuildOnAnalysisError(String idField,
+                                                                             List<SolrInputDocument> docs,
+                                                                             final String collectionName,
+                                                                             boolean asBatch,
+                                                                             String[] analysisErrorHolder) throws IOException, SolrServerException {
+    Map<Object, Throwable> results;
+    try {
+      results = indexSampleDocs(idField, docs, collectionName, asBatch);
+    } catch (IllegalArgumentException analysisExc) {
+      String errMsg = SolrException.getRootCause(analysisExc).getMessage();
+      if (analysisErrorHolder != null) {
+        analysisErrorHolder[0] = errMsg;
+      }
+      log.warn("Rebuilding temp collection {} after low-level Lucene indexing issue: {}", collectionName, errMsg);
+      configSetHelper.reloadTempCollection(collectionName, true);
+      results = indexSampleDocs(idField, docs, collectionName, asBatch);
+      log.info("Re-index sample docs into {} after rebuild due to {} succeeded; results: {}", collectionName, errMsg, results);
+    }
+    return results;
+  }
+
+  protected Map<Object, Throwable> indexSampleDocs(String idField,
+                                                   List<SolrInputDocument> docs,
+                                                   final String collectionName,
+                                                   boolean asBatch) throws IOException, SolrServerException {
+
+    Map<Object, Throwable> errorsDuringIndexing = new LinkedHashMap<>();
+
+    RTimer timer = new RTimer();
+
+    // load sample docs from blob store
+    CloudSolrClient cloudSolrClient = cloudClient();
+    cloudSolrClient.deleteByQuery(collectionName, "*:*", 1);
+    cloudSolrClient.optimize(collectionName, true, true, 1);
+
+    final int commitWithin = 100;
+    final int numDocs = docs.size();
+    int numAdded = 0;
+    if (asBatch) {
+      cloudSolrClient.add(collectionName, docs, commitWithin);
+      numAdded = docs.size();
+    } else {
+      int maxErrors = Math.min(100, Math.round(numDocs / 2f));
+      for (SolrInputDocument next : docs) {
+        try {
+          cloudSolrClient.add(collectionName, next, commitWithin);
+          ++numAdded;
+        } catch (Exception exc) {
+          Throwable rootCause = SolrException.getRootCause(exc);
+          if (String.valueOf(rootCause.getMessage()).contains("possible analysis error")) {
+            throw new IllegalArgumentException(rootCause);
+          } else {
+            Object docId = next.getFieldValue(idField);
+            if (docId == null) {
+              throw exc;
+            }
+            errorsDuringIndexing.put(docId, rootCause);
+
+            // break if there are a lot of errors in indexing as something is very wrong if so ...
+            if (errorsDuringIndexing.size() > 20 && errorsDuringIndexing.size() >= maxErrors) {
+              break;
+            }
+          }
+        }
+      }
+    }
+
+    cloudSolrClient.commit(collectionName, true, true, true);
+
+    if (!errorsDuringIndexing.isEmpty()) {
+      return errorsDuringIndexing;
+    }
+
+    long numFound = waitToSeeSampleDocs(collectionName, numAdded);
+    double tookMs = timer.getTime();
+    log.debug("Indexed {} docs into collection {}, took {} ms", numFound, collectionName, tookMs);
+
+    return !errorsDuringIndexing.isEmpty() ? errorsDuringIndexing : null;
+  }
+
+  protected long waitToSeeSampleDocs(String collectionName, long numAdded) throws IOException, SolrServerException {
+    CloudSolrClient cloudSolrClient = cloudClient();
+    SolrQuery query = new SolrQuery("*:*");
+    query.setRows(0);
+    QueryResponse queryResponse = cloudSolrClient.query(collectionName, query);
+    long numFound = queryResponse.getResults().getNumFound();
+    if (numFound < numAdded) {
+      // wait up to 5 seconds for this to occur
+      final long deadline = System.nanoTime() + TimeUnit.SECONDS.toNanos(5);
+      do {
+        cloudSolrClient.commit(collectionName, true, true, true);
+        queryResponse = cloudSolrClient.query(collectionName, query);
+        numFound = queryResponse.getResults().getNumFound();
+        if (numFound >= numAdded) {
+          break;
+        }
+        try {
+          Thread.sleep(200); // little pause to avoid flooding the server with requests in this loop
+        } catch (InterruptedException e) {
+          Thread.currentThread().interrupt();
+          break;
+        }
+      } while (System.nanoTime() < deadline);
+
+      if (numFound < numAdded) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+            "Failed to index " + numAdded + " sample docs into temp collection: " + collectionName);
+      }
+    }
+    return numFound;
+  }
+
+  protected Map<String, Object> buildResponse(String configSet,
+                                              final ManagedIndexSchema schema,
+                                              SchemaDesignerSettings settings,
+                                              List<SolrInputDocument> docs) throws IOException {
+    String mutableId = getMutableId(configSet);
+    int currentVersion = configSetHelper.getCurrentSchemaVersion(mutableId);
+    indexedVersion.put(mutableId, currentVersion);
+
+    // response is a map of data structures to support the schema designer
+    Map<String, Object> response = new HashMap<>();
+
+    DocCollection coll = zkStateReader().getCollection(mutableId);
+    if (coll.getActiveSlicesArr().length > 0) {
+      String coreName = coll.getActiveSlicesArr()[0].getLeader().getCoreName();
+      response.put("core", coreName);
+    }
+
+    response.put(UNIQUE_KEY_FIELD_PARAM, schema.getUniqueKeyField().getName());
+
+    response.put(CONFIG_SET_PARAM, configSet);
+    // important: pass the designer the current schema zk version for MVCC
+    response.put(SCHEMA_VERSION_PARAM, currentVersion);
+    response.put(TEMP_COLLECTION_PARAM, mutableId);
+    response.put("collectionsForConfig", configSetHelper.listCollectionsForConfig(configSet));
+    // Guess at a schema for each field found in the sample docs
+    // Collect all fields across all docs with mapping to values
+    response.put("fields", schema.getFields().values().stream()
+        .map(f -> fieldToMap(f, schema))
+        .sorted(Comparator.comparing(map -> ((String) map.get("name"))))
+        .collect(Collectors.toList()));
+
+    if (settings == null) {
+      settings = settingsDAO.getSettings(mutableId);
+    }
+    addSettingsToResponse(settings, response);
+
+    response.put("dynamicFields", Arrays.stream(schema.getDynamicFieldPrototypes())
+        .map(e -> e.getNamedPropertyValues(true))
+        .sorted(Comparator.comparing(map -> ((String) map.get("name"))))
+        .collect(Collectors.toList()));
+
+    response.put("fieldTypes", schema.getFieldTypes().values().stream()
+        .map(fieldType -> fieldType.getNamedPropertyValues(true))
+        .sorted(Comparator.comparing(map -> ((String) map.get("name"))))
+        .collect(Collectors.toList()));
+
+    // files
+    SolrZkClient zkClient = zkStateReader().getZkClient();
+    String configPathInZk = getConfigSetZkPath(mutableId);
+    final Set<String> files = new HashSet<>();
+    try {
+      ZkMaintenanceUtils.traverseZkTree(zkClient, configPathInZk, ZkMaintenanceUtils.VISIT_ORDER.VISIT_POST, files::add);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Failed to traverse files under: " + configPathInZk, SolrZkClient.checkInterrupted(e));
+    }
+    files.remove(configPathInZk);
+
+    final String prefix = configPathInZk + "/";
+    final int prefixLen = prefix.length();
+    Set<String> stripPrefix = files.stream().map(f -> f.startsWith(prefix) ? f.substring(prefixLen) : f).collect(Collectors.toSet());
+    stripPrefix.remove(DEFAULT_MANAGED_SCHEMA_RESOURCE_NAME);
+    stripPrefix.remove("lang");
+    stripPrefix.remove(CONFIGOVERLAY_JSON); // treat this file as private
+
+    List<String> sortedFiles = new ArrayList<>(stripPrefix);
+    Collections.sort(sortedFiles);
+    response.put("files", sortedFiles);
+
+    // info about the sample docs
+    if (docs != null) {
+      final String uniqueKeyField = schema.getUniqueKeyField().getName();
+      response.put("docIds", docs.stream()
+          .map(d -> (String) d.getFieldValue(uniqueKeyField))
+          .filter(Objects::nonNull)
+          .limit(100)
+          .collect(Collectors.toList()));
+    }
+
+    response.put("numDocs", docs != null ? docs.size() : -1);
+
+    return response;
+  }
+
+  protected void addErrorToResponse(String collection,
+                                    SolrException solrExc,
+                                    Map<Object, Throwable> errorsDuringIndexing,
+                                    Map<String, Object> response,
+                                    String updateError) {
+
+    if (solrExc == null && (errorsDuringIndexing == null || errorsDuringIndexing.isEmpty())) {
+      return; // no errors
+    }
+
+    if (updateError != null) {
+      response.put(UPDATE_ERROR, updateError);
+    }
+
+    if (solrExc != null) {
+      response.put("updateErrorCode", solrExc.code());
+      response.putIfAbsent(UPDATE_ERROR, solrExc.getMessage());
+    }
+
+    response.putIfAbsent(UPDATE_ERROR, "Index sample documents into " + collection + " failed!");
+    response.putIfAbsent("updateErrorCode", 400);
+    if (errorsDuringIndexing != null) {
+      response.put(ERROR_DETAILS, errorsDuringIndexing);
+    }
+  }
+
+  protected SimpleOrderedMap<Object> fieldToMap(SchemaField f, ManagedIndexSchema schema) {
+    SimpleOrderedMap<Object> map = f.getNamedPropertyValues(true);
+
+    // add the copy field destination field names
+    List<String> copyFieldNames =
+        schema.getCopyFieldsList((String) map.get("name")).stream().map(c -> c.getDestination().getName()).collect(Collectors.toList());
+    map.add("copyDest", String.join(",", copyFieldNames));
+
+    return map;
+  }
+
+  @SuppressWarnings("unchecked")
+  protected Map<String, Object> readJsonFromRequest(SolrQueryRequest req) throws IOException {
+    ContentStream stream = extractSingleContentStream(req, true);
+    String contentType = stream.getContentType();
+    if (isEmpty(contentType) || !contentType.toLowerCase(Locale.ROOT).contains(JSON_MIME)) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Expected JSON in request!");
+    }
+    final Object json;
+    try (Reader reader = stream.getReader()) {
+      json = ObjectBuilder.getVal(new JSONParser(reader));
+    }
+    return (Map<String, Object>) json;
+  }
+
+  protected void addSettingsToResponse(SchemaDesignerSettings settings, final Map<String, Object> response) {
+    response.put(LANGUAGES_PARAM, settings.getLanguages());
+    response.put(ENABLE_FIELD_GUESSING_PARAM, settings.fieldGuessingEnabled());
+    response.put(ENABLE_DYNAMIC_FIELDS_PARAM, settings.dynamicFieldsEnabled());
+    response.put(ENABLE_NESTED_DOCS_PARAM, settings.nestedDocsEnabled());
+    response.put(DISABLED, settings.isDisabled());
+    Optional<Integer> publishedVersion = settings.getPublishedVersion();
+    publishedVersion.ifPresent(version -> response.put(PUBLISHED_VERSION, version));
+    String copyFrom = settings.getCopyFrom();
+    if (copyFrom != null) {
+      response.put(COPY_FROM_PARAM, copyFrom);
+    }
+  }
+
+  protected String checkMutable(String configSet, SolrQueryRequest req) throws IOException {
+    // an apply just copies over the temp config to the "live" location
+    String mutableId = getMutableId(configSet);
+    if (!configExists(mutableId)) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+          mutableId + " configSet not found! Are you sure " + configSet + " was being edited by the schema designer?");
+    }
+
+    // check the versions agree
+    configSetHelper.checkSchemaVersion(mutableId, requireSchemaVersionFromClient(req), -1);
+
+    return mutableId;
+  }
+
+  protected int requireSchemaVersionFromClient(SolrQueryRequest req) {
+    final int schemaVersion = req.getParams().getInt(SCHEMA_VERSION_PARAM, -1);
+    if (schemaVersion == -1) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+          SCHEMA_VERSION_PARAM + " is a required parameter for the " + req.getPath() + " endpoint");
+    }
+    return schemaVersion;
+  }
+
+  protected String getRequiredParam(final String param, final SolrQueryRequest req) {
+    final String paramValue = req.getParams().get(param);
+    if (isEmpty(paramValue)) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+          param + " is a required parameter for the " + req.getPath() + " endpoint!");
+    }
+    return paramValue;
+  }
+
+  protected void cleanupTemp(String configSet) throws IOException, SolrServerException {
+    String mutableId = getMutableId(configSet);
+    indexedVersion.remove(mutableId);
+    CollectionAdminRequest.deleteCollection(mutableId).process(cloudClient());
+    configSetHelper.deleteStoredSampleDocs(configSet);
+    deleteConfig(mutableId);
+  }
+
+  private boolean configExists(String configSet) throws IOException {
+    return configSetHelper.checkConfigExists(configSet);
+  }
+
+  private void deleteConfig(String configSet) throws IOException {
+    configSetHelper.deleteConfig(configSet);
+  }
+
+  private void copyConfig(String from, String to) throws IOException {
+    configSetHelper.copyConfig(from, to);
+  }
+
+  private void validateNewConfigSetName(String configSet) {
+    if (configSet.length() > 50) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Schema name should be 50 characters or less");
+    }
+    if (configSet.contains(" ") || configSet.contains("/")) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Schema name should not contain spaces or forward slash");
+    }
+  }
+
+  private boolean pathExistsInZk(final String zkPath) throws IOException {
+    SolrZkClient zkClient = zkStateReader().getZkClient();
+    try {
+      return zkClient.exists(zkPath, true);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Failed to check if path exists: " + zkPath, SolrZkClient.checkInterrupted(e));
+    }
+  }
+
+  private static class InMemoryResourceLoader extends SolrResourceLoader {
+    String resource;
+    byte[] data;
+    ZkSolrResourceLoader delegate;
+
+    public InMemoryResourceLoader(CoreContainer cc, String configSet, String resource, byte[] data) {
+      super(cc.getResourceLoader().getInstancePath(), cc.getResourceLoader().getClassLoader());
+
+      final SolrResourceLoader resourceLoader = cc.getResourceLoader();
+      this.delegate = new ZkSolrResourceLoader(resourceLoader.getInstancePath(), configSet, resourceLoader.getClassLoader(), new Properties(), cc.getZkController());
+      this.resource = resource;
+      this.data = data;
+    }
+
+    @Override
+    public InputStream openResource(String res) throws IOException {
+      return (this.resource.equals(res)) ? new ByteArrayInputStream(data) : delegate.openResource(res);
+    }
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerConfigSetHelper.java b/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerConfigSetHelper.java
new file mode 100644
index 0000000..db0021f
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerConfigSetHelper.java
@@ -0,0 +1,1078 @@
+/*
+ * 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.handler.designer;
+
+import java.io.ByteArrayOutputStream;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+import java.util.Properties;
+import java.util.Set;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+import java.util.stream.Collectors;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipOutputStream;
+
+import com.google.common.collect.Sets;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.FilenameUtils;
+import org.apache.http.HttpResponse;
+import org.apache.http.HttpStatus;
+import org.apache.http.client.methods.HttpGet;
+import org.apache.http.client.methods.HttpPost;
+import org.apache.http.client.utils.URIBuilder;
+import org.apache.http.client.utils.URLEncodedUtils;
+import org.apache.http.entity.ByteArrayEntity;
+import org.apache.http.util.EntityUtils;
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.client.solrj.impl.CloudSolrClient;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.request.schema.FieldTypeDefinition;
+import org.apache.solr.client.solrj.request.schema.SchemaRequest;
+import org.apache.solr.client.solrj.response.schema.SchemaResponse;
+import org.apache.solr.cloud.ZkSolrResourceLoader;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.cloud.ClusterState;
+import org.apache.solr.common.cloud.DocCollection;
+import org.apache.solr.common.cloud.Replica;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.cloud.UrlScheme;
+import org.apache.solr.common.cloud.ZkConfigManager;
+import org.apache.solr.common.cloud.ZkMaintenanceUtils;
+import org.apache.solr.common.cloud.ZkStateReader;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.common.util.Utils;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrConfig;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.handler.admin.CollectionsHandler;
+import org.apache.solr.schema.CopyField;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.ManagedIndexSchema;
+import org.apache.solr.schema.ManagedIndexSchemaFactory;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.schema.TextField;
+import org.apache.solr.util.RTimer;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.common.params.CommonParams.VERSION_FIELD;
+import static org.apache.solr.common.util.Utils.fromJSONString;
+import static org.apache.solr.common.util.Utils.makeMap;
+import static org.apache.solr.common.util.Utils.toJavabin;
+import static org.apache.solr.handler.admin.ConfigSetsHandler.DEFAULT_CONFIGSET_NAME;
+import static org.apache.solr.handler.designer.SchemaDesignerAPI.getConfigSetZkPath;
+import static org.apache.solr.handler.designer.SchemaDesignerAPI.getMutableId;
+import static org.apache.solr.schema.IndexSchema.NEST_PATH_FIELD_NAME;
+import static org.apache.solr.schema.IndexSchema.ROOT_FIELD_NAME;
+import static org.apache.solr.schema.ManagedIndexSchemaFactory.DEFAULT_MANAGED_SCHEMA_RESOURCE_NAME;
+
+class SchemaDesignerConfigSetHelper implements SchemaDesignerConstants {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private static final Set<String> removeFieldProps = new HashSet<>(Arrays.asList("href", "id", "copyDest"));
+  private static final List<String> includeLangIds = Arrays.asList("ws", "general", "rev", "sort");
+  private static final String ZNODE_PATH_DELIM = "/";
+  private static final String MULTIVALUED = "multiValued";
+  private static final int TEXT_PREFIX_LEN = "text_".length();
+
+
+  private final CoreContainer cc;
+  private final SchemaSuggester schemaSuggester;
+  private final ZkConfigManager configManager;
+
+  SchemaDesignerConfigSetHelper(CoreContainer cc, SchemaSuggester schemaSuggester) {
+    this.cc = cc;
+    this.schemaSuggester = schemaSuggester;
+    this.configManager = new ZkConfigManager(cc.getZkController().getZkClient());
+  }
+
+  @SuppressWarnings("unchecked")
+  Map<String, Object> analyzeField(String configSet, String fieldName, String fieldText) throws IOException {
+    final String mutableId = getMutableId(configSet);
+    final URI uri;
+    try {
+      uri = collectionApiEndpoint(mutableId, "analysis", "field")
+          .setParameter(CommonParams.WT, CommonParams.JSON)
+          .setParameter("analysis.showmatch", "true")
+          .setParameter("analysis.fieldname", fieldName)
+          .setParameter("analysis.fieldvalue", "POST")
+          .build();
+    } catch (URISyntaxException e) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
+    }
+
+    Map<String, Object> analysis = Collections.emptyMap();
+    HttpPost httpPost = new HttpPost(uri);
+    httpPost.setHeader("Content-Type", "text/plain");
+    httpPost.setEntity(new ByteArrayEntity(fieldText.getBytes(StandardCharsets.UTF_8)));
+    try {
+      HttpResponse resp = cloudClient().getHttpClient().execute(httpPost);
+      int statusCode = resp.getStatusLine().getStatusCode();
+      if (statusCode != HttpStatus.SC_OK) {
+        throw new SolrException(SolrException.ErrorCode.getErrorCode(statusCode),
+            EntityUtils.toString(resp.getEntity(), StandardCharsets.UTF_8));
+      }
+
+      Map<String, Object> response = (Map<String, Object>) fromJSONString(EntityUtils.toString(resp.getEntity(), StandardCharsets.UTF_8));
+      if (response != null) {
+        analysis = (Map<String, Object>) response.get("analysis");
+      }
+    } finally {
+      httpPost.releaseConnection();
+    }
+
+    return analysis;
+  }
+
+  List<String> listCollectionsForConfig(String configSet) {
+    final List<String> collections = new LinkedList<>();
+    Map<String, ClusterState.CollectionRef> states = zkStateReader().getClusterState().getCollectionStates();
+    for (Map.Entry<String, ClusterState.CollectionRef> e : states.entrySet()) {
+      final String coll = e.getKey();
+      if (coll.startsWith(DESIGNER_PREFIX)) {
+        continue; // ignore temp
+      }
+
+      try {
+        if (configSet.equals(zkStateReader().readConfigName(coll)) && e.getValue().get() != null) {
+          collections.add(coll);
+        }
+      } catch (Exception exc) {
+        log.warn("Failed to get config name for {}", coll, exc);
+      }
+    }
+    return collections;
+  }
+
+  @SuppressWarnings("unchecked")
+  public String addSchemaObject(String configSet, Map<String, Object> addJson) throws IOException, SolrServerException {
+    String mutableId = getMutableId(configSet);
+    SchemaRequest.Update addAction;
+    String action;
+    String objectName = null;
+    if (addJson.containsKey("add-field")) {
+      action = "add-field";
+      Map<String, Object> fieldAttrs = (Map<String, Object>) addJson.get(action);
+      objectName = (String) fieldAttrs.get("name");
+      addAction = new SchemaRequest.AddField(fieldAttrs);
+    } else if (addJson.containsKey("add-dynamic-field")) {
+      action = "add-dynamic-field";
+      Map<String, Object> fieldAttrs = (Map<String, Object>) addJson.get(action);
+      objectName = (String) fieldAttrs.get("name");
+      addAction = new SchemaRequest.AddDynamicField(fieldAttrs);
+    } else if (addJson.containsKey("add-copy-field")) {
+      action = "add-copy-field";
+      Map<String, Object> map = (Map<String, Object>) addJson.get(action);
+      Object dest = map.get("dest");
+      List<String> destFields = null;
+      if (dest instanceof String) {
+        destFields = Collections.singletonList((String) dest);
+      } else if (dest instanceof List) {
+        destFields = (List<String>) dest;
+      } else if (dest instanceof Collection) {
+        Collection<String> destColl = (Collection<String>) dest;
+        destFields = new ArrayList<>(destColl);
+      }
+      addAction = new SchemaRequest.AddCopyField((String) map.get("source"), destFields);
+    } else if (addJson.containsKey("add-field-type")) {
+      action = "add-field-type";
+      Map<String, Object> fieldAttrs = (Map<String, Object>) addJson.get(action);
+      objectName = (String) fieldAttrs.get("name");
+      FieldTypeDefinition ftDef = new FieldTypeDefinition();
+      ftDef.setAttributes(fieldAttrs);
+      addAction = new SchemaRequest.AddFieldType(ftDef);
+    } else {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, "Unsupported action in request body! " + addJson);
+    }
+
+    // Using the SchemaAPI vs. working on the schema directly because SchemaField.create methods are package protected
+    log.info("Sending {} request for configSet {}: {}", action, mutableId, addJson);
+    SchemaResponse.UpdateResponse schemaResponse = addAction.process(cloudClient(), mutableId);
+    Exception exc = schemaResponse.getException();
+    if (exc instanceof SolrException) {
+      throw (SolrException) exc;
+    } else if (schemaResponse.getStatus() != 0) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, exc);
+    }
+
+    return objectName;
+  }
+
+  void reloadTempCollection(String mutableId, boolean delete) throws IOException, SolrServerException {
+    if (delete) {
+      log.debug("Deleting and re-creating existing collection {} after schema update", mutableId);
+      CollectionAdminRequest.deleteCollection(mutableId).process(cloudClient());
+      try {
+        zkStateReader().waitForState(mutableId, 30, TimeUnit.SECONDS, Objects::isNull);
+      } catch (InterruptedException | TimeoutException e) {
+        throw new IOException("Failed to see deleted collection " + mutableId + " reflected in cluster state", SolrZkClient.checkInterrupted(e));
+      }
+      createCollection(mutableId, mutableId);
+      log.debug("Deleted and re-created existing collection: {}", mutableId);
+    } else {
+      CollectionAdminRequest.reloadCollection(mutableId).process(cloudClient());
+      log.debug("Reloaded existing collection: {}", mutableId);
+    }
+  }
+
+  Map<String, Object> updateSchemaObject(String configSet, Map<String, Object> updateJson, ManagedIndexSchema schemaBeforeUpdate) throws IOException, SolrServerException {
+    String name = (String) updateJson.get("name");
+    String mutableId = getMutableId(configSet);
+
+    boolean needsRebuild = false;
+
+    SolrException solrExc = null;
+    String updateType;
+    String updateError = null;
+
+    if (updateJson.get("type") != null) {
+      updateType = schemaBeforeUpdate.isDynamicField(name) ? "dynamicField" : "field";
+      try {
+        needsRebuild = updateField(configSet, updateJson, schemaBeforeUpdate);
+      } catch (SolrException exc) {
+        if (exc.code() != 400) {
+          throw exc;
+        }
+        solrExc = exc;
+        updateError = solrExc.getMessage() + " Previous settings will be restored.";
+      }
+    } else {
+      updateType = "type";
+      needsRebuild = updateFieldType(configSet, name, updateJson, schemaBeforeUpdate);
+    }
+
+    // the update may have required a full rebuild of the index, otherwise, it's just a reload / re-index sample
+    reloadTempCollection(mutableId, needsRebuild);
+
+    Map<String, Object> results = new HashMap<>();
+    results.put("rebuild", needsRebuild);
+    results.put("updateType", updateType);
+    if (updateError != null) {
+      results.put("updateError", updateError);
+    }
+    if (solrExc != null) {
+      results.put("solrExc", solrExc);
+    }
+    return results;
+  }
+
+  protected boolean updateFieldType(String configSet, String typeName, Map<String, Object> updateJson, ManagedIndexSchema schemaBeforeUpdate) {
+    boolean needsRebuild = false;
+
+    Map<String, Object> typeAttrs = updateJson.entrySet().stream()
+        .filter(e -> !removeFieldProps.contains(e.getKey()))
+        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+    FieldType fieldType = schemaBeforeUpdate.getFieldTypeByName(typeName);
+
+    // this is a field type
+    Object multiValued = typeAttrs.get(MULTIVALUED);
+    if (typeHasMultiValuedChange(multiValued, fieldType)) {
+      needsRebuild = true;
+      log.warn("Re-building the temp collection for {} after type {} updated to multi-valued {}", configSet, typeName, multiValued);
+    }
+
+    // nice, the json for this field looks like
+    // "synonymQueryStyle": "org.apache.solr.parser.SolrQueryParserBase$SynonymQueryStyle:AS_SAME_TERM"
+    if (typeAttrs.get("synonymQueryStyle") instanceof String) {
+      String synonymQueryStyle = (String) typeAttrs.get("synonymQueryStyle");
+      if (synonymQueryStyle.lastIndexOf(':') != -1) {
+        typeAttrs.put("synonymQueryStyle", synonymQueryStyle.substring(synonymQueryStyle.lastIndexOf(':') + 1));
+      }
+    }
+
+    ManagedIndexSchema updatedSchema =
+        schemaBeforeUpdate.replaceFieldType(fieldType.getTypeName(), (String) typeAttrs.get("class"), typeAttrs);
+    updatedSchema.persistManagedSchema(false);
+
+    return needsRebuild;
+  }
+
+  boolean updateField(String configSet, Map<String, Object> updateField, ManagedIndexSchema schemaBeforeUpdate) throws IOException, SolrServerException {
+    String mutableId = getMutableId(configSet);
+
+    String name = (String) updateField.get("name");
+    String type = (String) updateField.get("type");
+    String copyDest = (String) updateField.get("copyDest");
+    Map<String, Object> fieldAttributes = updateField.entrySet().stream()
+        .filter(e -> !removeFieldProps.contains(e.getKey()))
+        .collect(Collectors.toMap(Map.Entry::getKey, Map.Entry::getValue));
+
+    boolean needsRebuild = false;
+
+    SchemaField schemaField = schemaBeforeUpdate.getField(name);
+    boolean isDynamic = schemaBeforeUpdate.isDynamicField(name);
+    String currentType = schemaField.getType().getTypeName();
+
+    SimpleOrderedMap<Object> fromTypeProps;
+    if (type.equals(currentType)) {
+      // no type change, so just pull the current type's props (with defaults) as we'll use these
+      // to determine which props get explicitly overridden on the field
+      fromTypeProps = schemaBeforeUpdate.getFieldTypeByName(currentType).getNamedPropertyValues(true);
+    } else {
+      // validate type change
+      FieldType newType = schemaBeforeUpdate.getFieldTypeByName(type);
+      if (newType == null) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            "Invalid update request for field " + name + "! Field type " + type + " doesn't exist!");
+      }
+      validateTypeChange(configSet, schemaField, newType);
+
+      // type change looks valid
+      fromTypeProps = newType.getNamedPropertyValues(true);
+    }
+
+    // the diff holds all the explicit properties not inherited from the type
+    Map<String, Object> diff = new HashMap<>();
+    for (Map.Entry<String, Object> e : fieldAttributes.entrySet()) {
+      String attr = e.getKey();
+      Object attrValue = e.getValue();
+      if ("name".equals(attr) || "type".equals(attr)) {
+        continue; // we don't want these in the diff map
+      }
+
+      if ("required".equals(attr)) {
+        diff.put(attr, attrValue != null ? attrValue : false);
+      } else {
+        Object fromType = fromTypeProps.get(attr);
+        if (fromType == null || !fromType.equals(attrValue)) {
+          diff.put(attr, attrValue);
+        }
+      }
+    }
+
+    // detect if they're trying to copy multi-valued fields into a single-valued field
+    Object multiValued = diff.get(MULTIVALUED);
+    if (multiValued == null) {
+      // mv not overridden explicitly, but we need the actual value, which will come from the new type (if that changed) or the current field
+      multiValued = type.equals(currentType) ? schemaField.multiValued() : schemaBeforeUpdate.getFieldTypeByName(type).isMultiValued();
+    }
+
+    if (!isDynamic && Boolean.FALSE.equals(multiValued)) {
+      // make sure there are no mv source fields if this is a copy dest
+      for (String src : schemaBeforeUpdate.getCopySources(name)) {
+        SchemaField srcField = schemaBeforeUpdate.getField(src);
+        if (srcField.multiValued()) {
+          log.warn("Cannot change multi-valued field {} to single-valued because it is a copy field destination for multi-valued field {}", name, src);
+          multiValued = Boolean.TRUE;
+          diff.put(MULTIVALUED, multiValued);
+          break;
+        }
+      }
+    }
+
+    if (Boolean.FALSE.equals(multiValued) && schemaField.multiValued()) {
+      // changing from multi- to single value ... verify the data agrees ...
+      validateMultiValuedChange(configSet, schemaField, Boolean.FALSE);
+    }
+
+    // switch from single-valued to multi-valued requires a full rebuild
+    // See SOLR-12185 ... if we're switching from single to multi-valued, then it's a big operation
+    if (fieldHasMultiValuedChange(multiValued, schemaField)) {
+      needsRebuild = true;
+      log.warn("Need to rebuild the temp collection for {} after field {} updated to multi-valued {}", configSet, name, multiValued);
+    }
+
+    if (!needsRebuild) {
+      // check term vectors too
+      Boolean storeTermVector = (Boolean) fieldAttributes.getOrDefault("termVectors", Boolean.FALSE);
+      if (schemaField.storeTermVector() != storeTermVector) {
+        // cannot change termVectors w/o a full-rebuild
+        needsRebuild = true;
+      }
+    }
+
+    log.info("For {}, replacing field {} with attributes: {}", configSet, name, diff);
+    final FieldType fieldType = schemaBeforeUpdate.getFieldTypeByName(type);
+    ManagedIndexSchema updatedSchema = isDynamic ? schemaBeforeUpdate.replaceDynamicField(name, fieldType, diff)
+        : schemaBeforeUpdate.replaceField(name, fieldType, diff);
+
+    // persist the change before applying the copy-field updates
+    updatedSchema.persistManagedSchema(false);
+
+    if (!isDynamic) {
+      applyCopyFieldUpdates(mutableId, copyDest, name, updatedSchema);
+    }
+
+    return needsRebuild;
+  }
+
+  protected void validateMultiValuedChange(String configSet, SchemaField field, Boolean multiValued) throws IOException {
+    List<SolrInputDocument> docs = getStoredSampleDocs(configSet);
+    if (!docs.isEmpty()) {
+      boolean isMV = schemaSuggester.isMultiValued(field.getName(), docs);
+      if (isMV && !multiValued) {
+        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+            "Cannot change field " + field.getName() + " to single-valued as some sample docs have multiple values!");
+      }
+    }
+  }
+
+  protected void validateTypeChange(String configSet, SchemaField field, FieldType toType) throws IOException {
+    if ("_version_".equals(field.getName())) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,
+          "Cannot change type of the _version_ field; it must be a plong.");
+    }
+    List<SolrInputDocument> docs = getStoredSampleDocs(configSet);
+    if (!docs.isEmpty()) {
+      schemaSuggester.validateTypeChange(field, toType, docs);
+    }
+  }
+
+  void deleteStoredSampleDocs(String configSet) {
+    try {
+      cloudClient().deleteByQuery(BLOB_STORE_ID, "id:" + configSet + "_sample/*", 10);
+    } catch (IOException | SolrServerException exc) {
+      log.warn("Failed to delete sample docs from blob store for {}", configSet, exc);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  List<SolrInputDocument> getStoredSampleDocs(final String configSet) throws IOException {
+    List<SolrInputDocument> docs = null;
+
+    final URI uri;
+    try {
+      uri = collectionApiEndpoint(BLOB_STORE_ID, "blob", configSet + "_sample")
+          .setParameter(CommonParams.WT, "filestream")
+          .build();
+    } catch (URISyntaxException e) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
+    }
+
+    HttpGet httpGet = new HttpGet(uri);
+    try {
+      HttpResponse entity = cloudClient().getHttpClient().execute(httpGet);
+      int statusCode = entity.getStatusLine().getStatusCode();
+      if (statusCode == HttpStatus.SC_OK) {
+        byte[] bytes = DefaultSampleDocumentsLoader.streamAsBytes(entity.getEntity().getContent());
+        if (bytes.length > 0) {
+          docs = (List<SolrInputDocument>) Utils.fromJavabin(bytes);
+        }
+      } else if (statusCode != HttpStatus.SC_NOT_FOUND) {
+        byte[] bytes = DefaultSampleDocumentsLoader.streamAsBytes(entity.getEntity().getContent());
+        throw new IOException("Failed to lookup stored docs for " + configSet + " due to: " + new String(bytes, StandardCharsets.UTF_8));
+      } // else not found is ok
+    } finally {
+      httpGet.releaseConnection();
+    }
+    return docs != null ? docs : Collections.emptyList();
+  }
+
+  void storeSampleDocs(final String configSet, List<SolrInputDocument> docs) throws IOException {
+    docs.forEach(d -> d.removeField(VERSION_FIELD)); // remove _version_ field before storing ...
+    postDataToBlobStore(cloudClient(), configSet + "_sample",
+        DefaultSampleDocumentsLoader.streamAsBytes(toJavabin(docs)));
+  }
+
+  protected void postDataToBlobStore(CloudSolrClient cloudClient, String blobName, byte[] bytes) throws IOException {
+    final URI uri;
+    try {
+      uri = collectionApiEndpoint(BLOB_STORE_ID, "blob", blobName).build();
+    } catch (URISyntaxException e) {
+      throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, e);
+    }
+
+    HttpPost httpPost = new HttpPost(uri);
+    try {
+      httpPost.setHeader("Content-Type", "application/octet-stream");
+      httpPost.setEntity(new ByteArrayEntity(bytes));
+      HttpResponse resp = cloudClient.getHttpClient().execute(httpPost);
+      int statusCode = resp.getStatusLine().getStatusCode();
+      if (statusCode != HttpStatus.SC_OK) {
+        throw new SolrException(SolrException.ErrorCode.getErrorCode(statusCode),
+            EntityUtils.toString(resp.getEntity(), StandardCharsets.UTF_8));
+      }
+    } finally {
+      httpPost.releaseConnection();
+    }
+  }
+
+  private String getBaseUrl(final String collection) {
+    String baseUrl = null;
+    try {
+      Set<String> liveNodes = zkStateReader().getClusterState().getLiveNodes();
+      DocCollection docColl = zkStateReader().getCollection(collection);
+      if (docColl != null && !liveNodes.isEmpty()) {
+        Optional<Replica> maybeActive = docColl.getReplicas().stream().filter(r -> r.isActive(liveNodes)).findAny();
+        if (maybeActive.isPresent()) {
+          baseUrl = maybeActive.get().getBaseUrl();
+        }
+      }
+    } catch (Exception exc) {
+      log.warn("Failed to lookup base URL for collection {}", collection, exc);
+    }
+
+    if (baseUrl == null) {
+      baseUrl = UrlScheme.INSTANCE.getBaseUrlForNodeName(cc.getZkController().getNodeName());
+    }
+
+    return baseUrl;
+  }
+
+  private URIBuilder collectionApiEndpoint(final String collection, final String... morePathSegments) throws URISyntaxException {
+    URI baseUrl = new URI(getBaseUrl(collection));
+    // build up a list of path segments including any path in the base URL, collection, and additional segments provided by caller
+    List<String> path = new ArrayList<>(URLEncodedUtils.parsePathSegments(baseUrl.getPath()));
+    path.add(collection);
+    if (morePathSegments != null && morePathSegments.length > 0) {
+      path.addAll(Arrays.asList(morePathSegments));
+    }
+    return new URIBuilder(baseUrl).setPathSegments(path);
+  }
+
+  protected String getManagedSchemaZkPath(final String configSet) {
+    return getConfigSetZkPath(configSet, DEFAULT_MANAGED_SCHEMA_RESOURCE_NAME);
+  }
+
+  ManagedIndexSchema toggleNestedDocsFields(ManagedIndexSchema schema, boolean enabled) {
+    return enabled ? enableNestedDocsFields(schema, true) : deleteNestedDocsFieldsIfNeeded(schema, true);
+  }
+
+  ManagedIndexSchema enableNestedDocsFields(ManagedIndexSchema schema, boolean persist) {
+    boolean madeChanges = false;
+
+    if (!schema.hasExplicitField(ROOT_FIELD_NAME)) {
+      Map<String, Object> fieldAttrs = makeMap("docValues", false, "indexed", true, "stored", false);
+      schema = (ManagedIndexSchema) schema.addField(schema.newField(ROOT_FIELD_NAME, "string", fieldAttrs), false);
+      madeChanges = true;
+    }
+
+    if (!schema.hasExplicitField(NEST_PATH_FIELD_NAME)) {
+      schema = (ManagedIndexSchema) schema.addField(schema.newField(NEST_PATH_FIELD_NAME, NEST_PATH_FIELD_NAME, Collections.emptyMap()), false);
+      madeChanges = true;
+    }
+
+    if (madeChanges && persist) {
+      schema.persistManagedSchema(false);
+    }
+
+    return schema;
+  }
+
+  ManagedIndexSchema deleteNestedDocsFieldsIfNeeded(ManagedIndexSchema schema, boolean persist) {
+    List<String> toDelete = new LinkedList<>();
+    if (schema.hasExplicitField(ROOT_FIELD_NAME)) {
+      toDelete.add(ROOT_FIELD_NAME);
+    }
+    if (schema.hasExplicitField(NEST_PATH_FIELD_NAME)) {
+      toDelete.add(NEST_PATH_FIELD_NAME);
+    }
+    if (!toDelete.isEmpty()) {
+      schema = schema.deleteFields(toDelete);
+      if (persist) {
+        schema.persistManagedSchema(false);
+      }
+    }
+    return schema;
+  }
+
+  SolrConfig loadSolrConfig(String configSet) {
+    SolrResourceLoader resourceLoader = cc.getResourceLoader();
+    ZkSolrResourceLoader zkLoader =
+        new ZkSolrResourceLoader(resourceLoader.getInstancePath(), configSet, resourceLoader.getClassLoader(), new Properties(), cc.getZkController());
+    return SolrConfig.readFromResourceLoader(zkLoader, SOLR_CONFIG_XML, true, null);
+  }
+
+  ManagedIndexSchema loadLatestSchema(String configSet) {
+    return loadLatestSchema(loadSolrConfig(configSet));
+  }
+
+  ManagedIndexSchema loadLatestSchema(SolrConfig solrConfig) {
+    ManagedIndexSchemaFactory factory = new ManagedIndexSchemaFactory();
+    factory.init(new NamedList<>());
+    return factory.create(DEFAULT_MANAGED_SCHEMA_RESOURCE_NAME, solrConfig, null);
+  }
+
+  int getCurrentSchemaVersion(final String configSet) throws IOException {
+    int currentVersion = -1;
+    final String path = getManagedSchemaZkPath(configSet);
+    try {
+      Stat stat = cc.getZkController().getZkClient().exists(path, null, true);
+      if (stat != null) {
+        currentVersion = stat.getVersion();
+      }
+    } catch (KeeperException.NoNodeException notExists) {
+      // safe to ignore
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Error getting version for schema: " + configSet, SolrZkClient.checkInterrupted(e));
+    }
+    return currentVersion;
+  }
+
+  void createCollection(final String collection, final String configSet) throws IOException, SolrServerException {
+    createCollection(collection, configSet, 1, 1);
+  }
+
+  void createCollection(final String collection, final String configSet, int numShards, int numReplicas) throws IOException, SolrServerException {
+    RTimer timer = new RTimer();
+    CollectionAdminRequest.Create create = CollectionAdminRequest.createCollection(collection, configSet, numShards, numReplicas);
+    create.setMaxShardsPerNode(-1);
+    SolrResponse rsp = create.process(cloudClient());
+    try {
+      CollectionsHandler.waitForActiveCollection(collection, cc, rsp);
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Failed waiting for new collection " + collection + " to reach the active state",
+          SolrZkClient.checkInterrupted(e));
+    }
+    double tookMs = timer.getTime();
+    log.debug("Took {} ms to create new collection {} with configSet {}", tookMs, collection, configSet);
+  }
+
+  protected CloudSolrClient cloudClient() {
+    return cc.getSolrClientCache().getCloudSolrClient(cc.getZkController().getZkServerAddress());
+  }
+
+  protected ZkStateReader zkStateReader() {
+    return cc.getZkController().getZkStateReader();
+  }
+
+  boolean applyCopyFieldUpdates(String mutableId, String copyDest, String fieldName, ManagedIndexSchema schema) throws IOException, SolrServerException {
+    boolean updated = false;
+
+    if (copyDest == null || copyDest.trim().isEmpty()) {
+      // delete all the copy field directives for this field
+      List<CopyField> copyFieldsList = schema.getCopyFieldsList(fieldName);
+      if (!copyFieldsList.isEmpty()) {
+        List<String> dests = copyFieldsList.stream().map(cf -> cf.getDestination().getName()).collect(Collectors.toList());
+        SchemaRequest.DeleteCopyField delAction = new SchemaRequest.DeleteCopyField(fieldName, dests);
+        SchemaResponse.UpdateResponse schemaResponse = delAction.process(cloudClient(), mutableId);
+        if (schemaResponse.getStatus() != 0) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, schemaResponse.getException());
+        }
+        updated = true;
+      }
+    } else {
+      SchemaField field = schema.getField(fieldName);
+      Set<String> desired = new HashSet<>();
+      for (String dest : copyDest.trim().split(",")) {
+        String toAdd = dest.trim();
+        if (toAdd.equals(fieldName)) {
+          continue; // cannot copy to self
+        }
+
+        // make sure the field exists and is multi-valued if this field is
+        SchemaField toAddField = schema.getFieldOrNull(toAdd);
+        if (toAddField != null) {
+          if (!field.multiValued() || toAddField.multiValued()) {
+            desired.add(toAdd);
+          } else {
+            log.warn("Skipping copy-field dest {} for {} because it is not multi-valued!", toAdd, fieldName);
+          }
+        } else {
+          log.warn("Skipping copy-field dest {} for {} because it doesn't exist!", toAdd, fieldName);
+        }
+      }
+      Set<String> existing = schema.getCopyFieldsList(fieldName).stream().map(cf -> cf.getDestination().getName()).collect(Collectors.toSet());
+      Set<String> add = Sets.difference(desired, existing);
+      if (!add.isEmpty()) {
+        SchemaRequest.AddCopyField addAction = new SchemaRequest.AddCopyField(fieldName, new ArrayList<>(add));
+        SchemaResponse.UpdateResponse schemaResponse = addAction.process(cloudClient(), mutableId);
+        if (schemaResponse.getStatus() != 0) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, schemaResponse.getException());
+        }
+        updated = true;
+      } // no additions ...
+
+      Set<String> del = Sets.difference(existing, desired);
+      if (!del.isEmpty()) {
+        SchemaRequest.DeleteCopyField delAction = new SchemaRequest.DeleteCopyField(fieldName, new ArrayList<>(del));
+        SchemaResponse.UpdateResponse schemaResponse = delAction.process(cloudClient(), mutableId);
+        if (schemaResponse.getStatus() != 0) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, schemaResponse.getException());
+        }
+        updated = true;
+      } // no deletions ...
+    }
+
+    return updated;
+  }
+
+  protected boolean fieldHasMultiValuedChange(Object multiValued, SchemaField schemaField) {
+    return (multiValued == null ||
+        (Boolean.TRUE.equals(multiValued) && !schemaField.multiValued()) ||
+        (Boolean.FALSE.equals(multiValued) && schemaField.multiValued()));
+  }
+
+  protected boolean typeHasMultiValuedChange(Object multiValued, FieldType fieldType) {
+    return (multiValued == null ||
+        (Boolean.TRUE.equals(multiValued) && !fieldType.isMultiValued()) ||
+        (Boolean.FALSE.equals(multiValued) && fieldType.isMultiValued()));
+  }
+
+  ManagedIndexSchema syncLanguageSpecificObjectsAndFiles(String configSet, ManagedIndexSchema schema, List<String> langs, boolean dynamicEnabled, String copyFrom) throws IOException {
+    if (!langs.isEmpty()) {
+      // there's a subset of languages applied, so remove all the other langs
+      schema = removeLanguageSpecificObjectsAndFiles(configSet, schema, langs);
+    }
+
+    // now restore any missing types / files for the languages we need, optionally adding back dynamic fields too
+    schema = restoreLanguageSpecificObjectsAndFiles(configSet, schema, langs, dynamicEnabled, copyFrom);
+
+    schema.persistManagedSchema(false);
+    return schema;
+  }
+
+  protected ManagedIndexSchema removeLanguageSpecificObjectsAndFiles(String configSet, ManagedIndexSchema schema, List<String> langs) throws IOException {
+    final Set<String> languages = new HashSet<>(includeLangIds);
+    languages.addAll(langs);
+
+    final Set<String> usedTypes =
+        schema.getFields().values().stream().map(f -> f.getType().getTypeName()).collect(Collectors.toSet());
+
+    final Set<String> usedLangs =
+        schema.getFields().values().stream().filter(f -> isTextType(f.getType()))
+            .map(f -> f.getType().getTypeName().substring(TEXT_PREFIX_LEN)).collect(Collectors.toSet());
+
+    // don't remove types / files for langs that are explicitly being used by a field
+    languages.addAll(usedLangs);
+
+    Map<String, FieldType> types = schema.getFieldTypes();
+    final Set<String> toRemove = types.values().stream()
+        .filter(this::isTextType)
+        .filter(t -> !languages.contains(t.getTypeName().substring(TEXT_PREFIX_LEN)))
+        .map(FieldType::getTypeName)
+        .filter(t -> !usedTypes.contains(t)) // not explicitly used by a field
+        .collect(Collectors.toSet());
+
+    // find dynamic fields that refer to the types we're removing ...
+    List<String> toRemoveDF = Arrays.stream(schema.getDynamicFields())
+        .filter(df -> toRemove.contains(df.getPrototype().getType().getTypeName()))
+        .map(df -> df.getPrototype().getName())
+        .collect(Collectors.toList());
+
+    schema = schema.deleteDynamicFields(toRemoveDF);
+    schema = schema.deleteFieldTypes(toRemove);
+
+    SolrZkClient zkClient = cc.getZkController().getZkClient();
+    final String configPathInZk = ZkConfigManager.CONFIGS_ZKNODE + ZNODE_PATH_DELIM + configSet;
+    final Set<String> toRemoveFiles = new HashSet<>();
+    final Set<String> langExt = languages.stream().map(l -> "_" + l).collect(Collectors.toSet());
+    try {
+      ZkMaintenanceUtils.traverseZkTree(zkClient, configPathInZk, ZkMaintenanceUtils.VISIT_ORDER.VISIT_POST, path -> {
+        if (!isMatchingLangOrNonLangFile(path, langExt)) toRemoveFiles.add(path);
+      });
+    } catch (KeeperException.NoNodeException nne) {
+      // no-op
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Failed to traverse znode path: " + configPathInZk, SolrZkClient.checkInterrupted(e));
+    }
+
+    for (String path : toRemoveFiles) {
+      try {
+        zkClient.delete(path, -1, false);
+      } catch (KeeperException.NoNodeException nne) {
+        // no-op
+      } catch (KeeperException | InterruptedException e) {
+        throw new IOException("Failed to delete znode: " + path, SolrZkClient.checkInterrupted(e));
+      }
+    }
+
+    return schema;
+  }
+
+  protected ManagedIndexSchema restoreLanguageSpecificObjectsAndFiles(String configSet,
+                                                                      ManagedIndexSchema schema,
+                                                                      List<String> langs,
+                                                                      boolean dynamicEnabled,
+                                                                      String copyFrom) throws IOException {
+    // pull the dynamic fields from the copyFrom schema
+    ManagedIndexSchema copyFromSchema = loadLatestSchema(copyFrom);
+
+    final Set<String> langSet = new HashSet<>(includeLangIds);
+    langSet.addAll(langs);
+
+    boolean restoreAllLangs = langs.isEmpty();
+
+    final Set<String> langFilesToRestore = new HashSet<>();
+
+    // Restore missing files
+    SolrZkClient zkClient = zkStateReader().getZkClient();
+    String configPathInZk = ZkConfigManager.CONFIGS_ZKNODE + ZNODE_PATH_DELIM + copyFrom;
+    final Set<String> langExt = langSet.stream().map(l -> "_" + l).collect(Collectors.toSet());
+    try {
+      ZkMaintenanceUtils.traverseZkTree(zkClient, configPathInZk, ZkMaintenanceUtils.VISIT_ORDER.VISIT_POST, path -> {
+        if (path.endsWith(".txt")) {
+          if (restoreAllLangs) {
+            langFilesToRestore.add(path);
+            return;
+          }
+
+          final String pathWoExt = path.substring(0, path.length() - 4);
+          for (String lang : langExt) {
+            if (pathWoExt.endsWith(lang)) {
+              langFilesToRestore.add(path);
+              break;
+            }
+          }
+        }
+      });
+    } catch (KeeperException.NoNodeException nne) {
+      // no-op
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Failed to traverse znode path: " + configPathInZk, SolrZkClient.checkInterrupted(e));
+    }
+
+    if (!langFilesToRestore.isEmpty()) {
+      final String replacePathDir = "/" + configSet;
+      final String origPathDir = "/" + copyFrom;
+      for (String path : langFilesToRestore) {
+        String copyToPath = path.replace(origPathDir, replacePathDir);
+        try {
+          if (!zkClient.exists(copyToPath, true)) {
+            zkClient.makePath(copyToPath, false, true);
+            zkClient.setData(copyToPath, zkClient.getData(path, null, null, true), true);
+          }
+        } catch (KeeperException | InterruptedException e) {
+          throw new IOException("Failed to restore file at znode path: " + copyToPath, SolrZkClient.checkInterrupted(e));
+        }
+      }
+    }
+
+    // Restore field types
+    final Map<String, FieldType> existingTypes = schema.getFieldTypes();
+    List<FieldType> addTypes = copyFromSchema.getFieldTypes().values().stream()
+        .filter(t -> isLangTextType(t, restoreAllLangs ? null : langSet) && !existingTypes.containsKey(t.getTypeName()))
+        .collect(Collectors.toList());
+    if (!addTypes.isEmpty()) {
+      schema = schema.addFieldTypes(addTypes, false);
+    }
+
+    if (dynamicEnabled) {
+      // restore language specific dynamic fields
+      final Set<String> existingDynFields = Arrays.stream(schema.getDynamicFieldPrototypes())
+          .map(SchemaField::getName)
+          .collect(Collectors.toSet());
+
+      final Set<String> langFieldTypeNames = schema.getFieldTypes().values().stream()
+          .filter(t -> isLangTextType(t, restoreAllLangs ? null : langSet))
+          .map(FieldType::getTypeName)
+          .collect(Collectors.toSet());
+
+      List<SchemaField> addDynFields = Arrays.stream(copyFromSchema.getDynamicFields())
+          .filter(df -> langFieldTypeNames.contains(df.getPrototype().getType().getTypeName()))
+          .filter(df -> !existingDynFields.contains(df.getPrototype().getName()))
+          .map(IndexSchema.DynamicField::getPrototype)
+          .collect(Collectors.toList());
+      if (!addDynFields.isEmpty()) {
+        schema = schema.addDynamicFields(addDynFields, null, false);
+      }
+    } else {
+      schema = removeDynamicFields(schema);
+    }
+
+    return schema;
+  }
+
+  private boolean isMatchingLangOrNonLangFile(final String path, final Set<String> langs) {
+    if (!path.endsWith(".txt"))
+      return true; // not a .txt file, always include
+
+    int slashAt = path.lastIndexOf('/');
+    String fileName = slashAt != -1 ? path.substring(slashAt + 1) : "";
+    if (!fileName.contains("_"))
+      return true; // looking for file names like stopwords_en.txt, not a match, so skip it
+
+    // remove the .txt extension
+    final String pathWoExt = fileName.substring(0, fileName.length() - 4);
+    for (String lang : langs) {
+      if (pathWoExt.endsWith(lang)) {
+        return true;
+      }
+    }
+
+    // if we fall thru to here, then the file should be excluded
+    return false;
+  }
+
+  private boolean isTextType(final FieldType t) {
+    return t.getTypeName().startsWith("text_") && TextField.class.equals(t.getClass());
+  }
+
+  private boolean isLangTextType(final FieldType t, final Set<String> langSet) {
+    return isTextType(t) && (langSet == null || langSet.contains(t.getTypeName().substring("text_".length())));
+  }
+
+  protected ManagedIndexSchema removeDynamicFields(ManagedIndexSchema schema) {
+    List<String> dynamicFieldNames =
+        Arrays.stream(schema.getDynamicFields()).map(f -> f.getPrototype().getName()).collect(Collectors.toList());
+    if (!dynamicFieldNames.isEmpty()) {
+      schema = schema.deleteDynamicFields(dynamicFieldNames);
+    }
+    return schema;
+  }
+
+  protected ManagedIndexSchema restoreDynamicFields(ManagedIndexSchema schema, List<String> langs, String copyFrom) {
+    // pull the dynamic fields from the copyFrom schema
+    ManagedIndexSchema copyFromSchema = loadLatestSchema(copyFrom);
+    IndexSchema.DynamicField[] dynamicFields = copyFromSchema.getDynamicFields();
+    if (dynamicFields.length == 0 && !DEFAULT_CONFIGSET_NAME.equals(copyFrom)) {
+      copyFromSchema = loadLatestSchema(DEFAULT_CONFIGSET_NAME);
+      dynamicFields = copyFromSchema.getDynamicFields();
+    }
+
+    if (dynamicFields.length == 0) {
+      return schema;
+    }
+
+    final Set<String> existingDFNames =
+        Arrays.stream(schema.getDynamicFields()).map(df -> df.getPrototype().getName()).collect(Collectors.toSet());
+    List<SchemaField> toAdd = Arrays.stream(dynamicFields)
+        .filter(df -> !existingDFNames.contains(df.getPrototype().getName()))
+        .map(IndexSchema.DynamicField::getPrototype)
+        .collect(Collectors.toList());
+
+    // only restore language specific dynamic fields that match our langSet
+    if (!langs.isEmpty()) {
+      final Set<String> langSet = new HashSet<>(includeLangIds);
+      langSet.addAll(langs);
+      toAdd = toAdd.stream()
+          .filter(df -> !df.getName().startsWith("*_txt_") || langSet.contains(df.getName().substring("*_txt_".length())))
+          .collect(Collectors.toList());
+    }
+
+    if (!toAdd.isEmpty()) {
+      // grab any field types that need to be re-added
+      final Map<String, FieldType> fieldTypes = schema.getFieldTypes();
+      List<FieldType> addTypes = toAdd.stream()
+          .map(SchemaField::getType)
+          .filter(t -> !fieldTypes.containsKey(t.getTypeName()))
+          .collect(Collectors.toList());
+      if (!addTypes.isEmpty()) {
+        schema = schema.addFieldTypes(addTypes, false);
+      }
+
+      schema = schema.addDynamicFields(toAdd, null, true);
+    }
+
+    return schema;
+  }
+
+  void checkSchemaVersion(String configSet, final int versionInRequest, int currentVersion) throws IOException {
+    if (versionInRequest < 0) {
+      return; // don't enforce the version check
+    }
+
+    if (currentVersion == -1) {
+      currentVersion = getCurrentSchemaVersion(configSet);
+    }
+
+    if (currentVersion != versionInRequest) {
+      if (configSet.startsWith(DESIGNER_PREFIX)) {
+        configSet = configSet.substring(DESIGNER_PREFIX.length());
+      }
+      throw new SolrException(SolrException.ErrorCode.CONFLICT,
+          "Your schema version " + versionInRequest + " for " + configSet + " is out-of-date; current version is: " + currentVersion +
+              ". Perhaps another user also updated the schema while you were editing it? You'll need to retry your update after the schema is refreshed.");
+    }
+  }
+
+  List<String> listConfigsInZk() throws IOException {
+    return configManager.listConfigs();
+  }
+
+  byte[] downloadAndZipConfigSet(String configId) throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream();
+    Path tmpDirectory = Files.createTempDirectory("schema-designer-" + FilenameUtils.getName(configId));
+    File tmpDir = tmpDirectory.toFile();
+    try {
+      configManager.downloadConfigDir(configId, tmpDirectory);
+      try (ZipOutputStream zipOut = new ZipOutputStream(baos)) {
+        zipIt(tmpDir, "", zipOut);
+      }
+    } finally {
+      FileUtils.deleteDirectory(tmpDir);
+    }
+    return baos.toByteArray();
+  }
+
+  protected void zipIt(File f, String fileName, ZipOutputStream zipOut) throws IOException {
+    if (f.isHidden()) {
+      return;
+    }
+
+    if (f.isDirectory()) {
+      String dirPrefix = "";
+      if (fileName.endsWith("/")) {
+        zipOut.putNextEntry(new ZipEntry(fileName));
+        zipOut.closeEntry();
+        dirPrefix = fileName;
+      } else if (!fileName.isEmpty()) {
+        dirPrefix = fileName + "/";
+        zipOut.putNextEntry(new ZipEntry(dirPrefix));
+        zipOut.closeEntry();
+      }
+      File[] files = f.listFiles();
+      if (files != null) {
+        for (File child : files) {
+          zipIt(child, dirPrefix + child.getName(), zipOut);
+        }
+      }
+      return;
+    }
+
+    byte[] bytes = new byte[1024];
+    int r;
+    try (FileInputStream fis = new FileInputStream(f)) {
+      ZipEntry zipEntry = new ZipEntry(fileName);
+      zipOut.putNextEntry(zipEntry);
+      while ((r = fis.read(bytes)) >= 0) {
+        zipOut.write(bytes, 0, r);
+      }
+    }
+  }
+
+  boolean checkConfigExists(String configSet) throws IOException {
+    return configManager.configExists(configSet);
+  }
+
+  void deleteConfig(String configSet) throws IOException {
+    configManager.deleteConfigDir(configSet);
+  }
+
+  void copyConfig(String from, String to) throws IOException {
+    configManager.copyConfigDir(from, to);
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerConstants.java b/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerConstants.java
new file mode 100644
index 0000000..b6afdc6
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerConstants.java
@@ -0,0 +1,50 @@
+/*
+ * 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.handler.designer;
+
+public interface SchemaDesignerConstants {
+  String CONFIG_SET_PARAM = "configSet";
+  String COPY_FROM_PARAM = "copyFrom";
+  String SCHEMA_VERSION_PARAM = "schemaVersion";
+  String RELOAD_COLLECTIONS_PARAM = "reloadCollections";
+  String INDEX_TO_COLLECTION_PARAM = "indexToCollection";
+  String NEW_COLLECTION_PARAM = "newCollection";
+  String CLEANUP_TEMP_PARAM = "cleanupTemp";
+  String ENABLE_DYNAMIC_FIELDS_PARAM = "enableDynamicFields";
+  String ENABLE_FIELD_GUESSING_PARAM = "enableFieldGuessing";
+  String ENABLE_NESTED_DOCS_PARAM = "enableNestedDocs";
+  String TEMP_COLLECTION_PARAM = "tempCollection";
+  String PUBLISHED_VERSION = "publishedVersion";
+  String DISABLE_DESIGNER_PARAM = "disableDesigner";
+  String DISABLED = "disabled";
+  String DOC_ID_PARAM = "docId";
+  String FIELD_PARAM = "field";
+  String UNIQUE_KEY_FIELD_PARAM = "uniqueKeyField";
+  String AUTO_CREATE_FIELDS = "update.autoCreateFields";
+  String SOLR_CONFIG_XML = "solrconfig.xml";
+  String DESIGNER_KEY = "_designer.";
+  String LANGUAGES_PARAM = "languages";
+  String CONFIGOVERLAY_JSON = "configoverlay.json";
+  String BLOB_STORE_ID = ".system";
+  String UPDATE_ERROR = "updateError";
+  String ANALYSIS_ERROR = "analysisError";
+  String ERROR_DETAILS = "errorDetails";
+
+  String DESIGNER_PREFIX = "._designer_";
+  int MAX_SAMPLE_DOCS = 1000;
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerSettings.java b/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerSettings.java
new file mode 100644
index 0000000..756a26d
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerSettings.java
@@ -0,0 +1,149 @@
+/*
+ * 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.handler.designer;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.Optional;
+
+import org.apache.solr.schema.ManagedIndexSchema;
+
+class SchemaDesignerSettings implements SchemaDesignerConstants {
+
+  private String copyFrom;
+  private boolean isDisabled;
+  private List<String> languages;
+  private boolean dynamicFieldsEnabled;
+  private boolean nestedDocsEnabled;
+  private boolean fieldGuessingEnabled;
+  private Integer publishedVersion;
+  private ManagedIndexSchema schema;
+
+  @SuppressWarnings("unchecked")
+  SchemaDesignerSettings(Map<String, Object> stored) {
+    this.isDisabled = (Boolean) stored.getOrDefault(DESIGNER_KEY + DISABLED, false);
+    this.publishedVersion = null;
+    this.copyFrom = (String) stored.get(DESIGNER_KEY + COPY_FROM_PARAM);
+    this.languages = (List<String>) stored.getOrDefault(DESIGNER_KEY + LANGUAGES_PARAM, Collections.emptyList());
+    this.dynamicFieldsEnabled = (Boolean) stored.getOrDefault(DESIGNER_KEY + ENABLE_DYNAMIC_FIELDS_PARAM, true);
+    this.nestedDocsEnabled = (Boolean) stored.getOrDefault(DESIGNER_KEY + ENABLE_NESTED_DOCS_PARAM, false);
+    this.fieldGuessingEnabled = (Boolean) stored.getOrDefault(AUTO_CREATE_FIELDS, true);
+  }
+
+  Map<String, Object> toMap() {
+    Map<String, Object> map = new HashMap<>();
+    map.put(DESIGNER_KEY + DISABLED, isDisabled);
+    map.put(DESIGNER_KEY + LANGUAGES_PARAM, languages);
+    map.put(DESIGNER_KEY + ENABLE_DYNAMIC_FIELDS_PARAM, dynamicFieldsEnabled);
+    map.put(DESIGNER_KEY + ENABLE_NESTED_DOCS_PARAM, nestedDocsEnabled);
+    map.put(AUTO_CREATE_FIELDS, fieldGuessingEnabled);
+    if (copyFrom != null) {
+      map.put(DESIGNER_KEY + COPY_FROM_PARAM, copyFrom);
+    }
+    if (publishedVersion != null) {
+      map.put(DESIGNER_KEY + PUBLISHED_VERSION, publishedVersion);
+    }
+    return map;
+  }
+
+  public ManagedIndexSchema getSchema() {
+    return schema;
+  }
+
+  public void setSchema(ManagedIndexSchema schema) {
+    this.schema = schema;
+  }
+
+  public Optional<Integer> getPublishedVersion() {
+    return Optional.ofNullable(publishedVersion);
+  }
+
+  public void setPublishedVersion(int publishedVersion) {
+    this.publishedVersion = publishedVersion;
+  }
+
+  public String getCopyFrom() {
+    return copyFrom;
+  }
+
+  public void setCopyFrom(String copyFrom) {
+    this.copyFrom = copyFrom;
+  }
+
+  public boolean isDisabled() {
+    return isDisabled;
+  }
+
+  public void setDisabled(boolean isDisabled) {
+    this.isDisabled = isDisabled;
+  }
+
+  public List<String> getLanguages() {
+    return languages;
+  }
+
+  public void setLanguages(List<String> langs) {
+    this.languages = langs != null ? langs : Collections.emptyList();
+  }
+
+  public boolean dynamicFieldsEnabled() {
+    return dynamicFieldsEnabled;
+  }
+
+  public void setDynamicFieldsEnabled(boolean enabled) {
+    this.dynamicFieldsEnabled = enabled;
+  }
+
+  public boolean nestedDocsEnabled() {
+    return nestedDocsEnabled;
+  }
+
+  public void setNestedDocsEnabled(boolean enabled) {
+    this.nestedDocsEnabled = enabled;
+  }
+
+  public boolean fieldGuessingEnabled() {
+    return fieldGuessingEnabled;
+  }
+
+  public void setFieldGuessingEnabled(boolean enabled) {
+    this.fieldGuessingEnabled = enabled;
+  }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) return true;
+    if (o == null || getClass() != o.getClass()) return false;
+    SchemaDesignerSettings that = (SchemaDesignerSettings) o;
+    return isDisabled == that.isDisabled &&
+        dynamicFieldsEnabled == that.dynamicFieldsEnabled &&
+        nestedDocsEnabled == that.nestedDocsEnabled &&
+        fieldGuessingEnabled == that.fieldGuessingEnabled &&
+        Objects.equals(copyFrom, that.copyFrom) &&
+        languages.equals(that.languages) &&
+        Objects.equals(publishedVersion, that.publishedVersion);
+  }
+
+  @Override
+  public int hashCode() {
+    return Objects.hash(copyFrom, isDisabled, languages, dynamicFieldsEnabled, nestedDocsEnabled, fieldGuessingEnabled, publishedVersion);
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerSettingsDAO.java b/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerSettingsDAO.java
new file mode 100644
index 0000000..38efdf4
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/designer/SchemaDesignerSettingsDAO.java
@@ -0,0 +1,175 @@
+/*
+ * 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.handler.designer;
+
+import java.io.IOException;
+import java.lang.invoke.MethodHandles;
+import java.nio.charset.StandardCharsets;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.solr.cloud.ZkController;
+import org.apache.solr.cloud.ZkSolrResourceLoader;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.core.ConfigOverlay;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.PluginInfo;
+import org.apache.solr.core.SolrConfig;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.update.processor.UpdateRequestProcessorChain;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.noggit.JSONParser;
+import org.noggit.ObjectBuilder;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.solr.handler.designer.SchemaDesignerAPI.getConfigSetZkPath;
+
+class SchemaDesignerSettingsDAO implements SchemaDesignerConstants {
+
+  private static final Logger log = LoggerFactory.getLogger(MethodHandles.lookup().lookupClass());
+
+  private final CoreContainer cc;
+
+  SchemaDesignerSettingsDAO(CoreContainer cc) {
+    this.cc = cc;
+  }
+
+  SchemaDesignerSettings getSettings(String configSet) {
+    SolrConfig solrConfig =
+        SolrConfig.readFromResourceLoader(zkLoaderForConfigSet(configSet), SOLR_CONFIG_XML, true, null);
+    return getSettings(solrConfig);
+  }
+
+  SchemaDesignerSettings getSettings(SolrConfig solrConfig) {
+    Map<String, Object> map = new HashMap<>();
+
+    boolean isFieldGuessingEnabled = true;
+    if (solrConfig != null) {
+      ConfigOverlay overlay = solrConfig.getOverlay();
+      Map<String, Object> userProps = overlay != null ? overlay.getUserProps() : null;
+      if (userProps != null) {
+        map.putAll(userProps);
+      }
+      isFieldGuessingEnabled = isFieldGuessingEnabled(solrConfig);
+    }
+
+    map.putIfAbsent(AUTO_CREATE_FIELDS, isFieldGuessingEnabled);
+    map.putIfAbsent(DESIGNER_KEY + LANGUAGES_PARAM, Collections.emptyList());
+    map.putIfAbsent(DESIGNER_KEY + ENABLE_DYNAMIC_FIELDS_PARAM, true);
+    map.putIfAbsent(DESIGNER_KEY + ENABLE_NESTED_DOCS_PARAM, false);
+
+    return new SchemaDesignerSettings(map);
+  }
+
+  boolean persistIfChanged(String configSet, SchemaDesignerSettings settings) throws IOException {
+    boolean changed = false;
+
+    ConfigOverlay overlay = getConfigOverlay(configSet);
+    Map<String, Object> storedUserProps = overlay != null ? overlay.getUserProps() : Collections.emptyMap();
+    for (Map.Entry<String, Object> e : settings.toMap().entrySet()) {
+      String key = e.getKey();
+      Object propValue = e.getValue();
+      if (propValue != null && !propValue.equals(storedUserProps.get(key))) {
+        if (overlay == null) overlay = new ConfigOverlay(null, -1);
+        overlay = overlay.setUserProperty(key, propValue);
+        changed = true;
+      }
+    }
+
+    if (changed) {
+      ZkController.persistConfigResourceToZooKeeper(zkLoaderForConfigSet(configSet), overlay.getZnodeVersion(),
+          ConfigOverlay.RESOURCE_NAME, overlay.toByteArray(), true);
+    }
+
+    return changed;
+  }
+
+  boolean isDesignerDisabled(String configSet) {
+    // filter out any configs that don't want to be edited by the Schema Designer
+    // this allows users to lock down specific configs from being edited by the designer
+    ConfigOverlay overlay = null;
+    try {
+      overlay = getConfigOverlay(configSet);
+    } catch (IOException exc) {
+      log.error("Failed to get config overlay for {}", configSet, exc);
+    }
+    Map<String, Object> userProps = overlay != null ? overlay.getUserProps() : Collections.emptyMap();
+    return (boolean) userProps.getOrDefault(DESIGNER_KEY + DISABLED, false);
+  }
+
+  @SuppressWarnings("unchecked")
+  private ConfigOverlay getConfigOverlay(String config) throws IOException {
+    ConfigOverlay overlay = null;
+    String path = getConfigSetZkPath(config, CONFIGOVERLAY_JSON);
+    byte[] data = null;
+    Stat stat = new Stat();
+    try {
+      data = cc.getZkController().getZkStateReader().getZkClient().getData(path, null, stat, true);
+    } catch (KeeperException.NoNodeException nne) {
+      // ignore
+    } catch (KeeperException | InterruptedException e) {
+      throw new IOException("Error reading path: " + path, SolrZkClient.checkInterrupted(e));
+    }
+    if (data != null && data.length > 0) {
+      Map<String, Object> json =
+          (Map<String, Object>) ObjectBuilder.getVal(new JSONParser(new String(data, StandardCharsets.UTF_8)));
+      overlay = new ConfigOverlay(json, stat.getVersion());
+    }
+    return overlay;
+  }
+
+  private boolean isFieldGuessingEnabled(final SolrConfig solrConfig) {
+    if (!hasFieldGuessingURPChain(solrConfig)) {
+      return false; // no URP chain, so can't be enabled
+    }
+
+    boolean isEnabled = true;
+    ConfigOverlay overlay = solrConfig.getOverlay();
+    if (overlay != null) {
+      Map<String, Object> userProps = overlay.getUserProps();
+      if (userProps != null) {
+        isEnabled = (Boolean) userProps.getOrDefault(AUTO_CREATE_FIELDS, true);
+      }
+    }
+    return isEnabled;
+  }
+
+  private boolean hasFieldGuessingURPChain(final SolrConfig solrConfig) {
+    boolean hasPlugin = false;
+    List<PluginInfo> plugins = solrConfig.getPluginInfos(UpdateRequestProcessorChain.class.getName());
+    if (plugins != null) {
+      for (PluginInfo next : plugins) {
+        if ("add-unknown-fields-to-the-schema".equals(next.name)) {
+          hasPlugin = true;
+          break;
+        }
+      }
+    }
+    return hasPlugin;
+  }
+
+  private ZkSolrResourceLoader zkLoaderForConfigSet(final String configSet) {
+    SolrResourceLoader loader = cc.getResourceLoader();
+    return new ZkSolrResourceLoader(loader.getInstancePath(), configSet, loader.getClassLoader(), new Properties(), cc.getZkController());
+  }
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/designer/SchemaSuggester.java b/solr/core/src/java/org/apache/solr/handler/designer/SchemaSuggester.java
new file mode 100644
index 0000000..ca70486
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/designer/SchemaSuggester.java
@@ -0,0 +1,42 @@
+/*
+ * 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.handler.designer;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.ManagedIndexSchema;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.util.plugin.NamedListInitializedPlugin;
+
+public interface SchemaSuggester extends NamedListInitializedPlugin {
+  Optional<SchemaField> suggestField(String fieldName, List<Object> sampleValues, IndexSchema schema, List<String> langs);
+
+  ManagedIndexSchema adaptExistingFieldToData(SchemaField schemaField, List<Object> sampleValues, ManagedIndexSchema schema);
+
+  Map<String, List<Object>> transposeDocs(List<SolrInputDocument> docs);
+
+  void validateTypeChange(SchemaField field, FieldType toType, List<SolrInputDocument> docs) throws IOException;
+
+  boolean isMultiValued(String name, List<SolrInputDocument> docs);
+}
diff --git a/solr/core/src/java/org/apache/solr/handler/designer/package-info.java b/solr/core/src/java/org/apache/solr/handler/designer/package-info.java
new file mode 100644
index 0000000..30be4e2
--- /dev/null
+++ b/solr/core/src/java/org/apache/solr/handler/designer/package-info.java
@@ -0,0 +1,26 @@
+/*
+ * This software was produced for the U. S. Government
+ * under Contract No. W15P7T-11-C-F600, and is
+ * subject to the Rights in Noncommercial Computer Software
+ * and Noncommercial Computer Software Documentation
+ * Clause 252.227-7014 (JUN 1995)
+ *
+ * Copyright 2013 The MITRE Corporation. All Rights Reserved.
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+/**
+ * The {@link org.apache.solr.handler.designer.SchemaDesignerAPI} and supporting classes.
+ */
+package org.apache.solr.handler.designer;
diff --git a/solr/core/src/java/org/apache/solr/handler/loader/CSVLoader.java b/solr/core/src/java/org/apache/solr/handler/loader/CSVLoader.java
index 11866a4..6e2fb8b 100644
--- a/solr/core/src/java/org/apache/solr/handler/loader/CSVLoader.java
+++ b/solr/core/src/java/org/apache/solr/handler/loader/CSVLoader.java
@@ -37,7 +37,7 @@ class SingleThreadedCSVLoader extends CSVLoaderBase {
   }
 
   @Override
-  void addDoc(int line, String[] vals) throws IOException {
+  public void addDoc(int line, String[] vals) throws IOException {
     templateAdd.clear();
     SolrInputDocument doc = new SolrInputDocument();
     doAdd(line, vals, doc, templateAdd);
diff --git a/solr/core/src/java/org/apache/solr/handler/loader/CSVLoaderBase.java b/solr/core/src/java/org/apache/solr/handler/loader/CSVLoaderBase.java
index f5bc2c8..2cc7662 100644
--- a/solr/core/src/java/org/apache/solr/handler/loader/CSVLoaderBase.java
+++ b/solr/core/src/java/org/apache/solr/handler/loader/CSVLoaderBase.java
@@ -37,7 +37,7 @@ import java.util.HashMap;
 import java.util.Iterator;
 import java.io.*;
 
-abstract class CSVLoaderBase extends ContentStreamLoader {
+public abstract class CSVLoaderBase extends ContentStreamLoader {
   public static final String SEPARATOR="separator";
   public static final String FIELDNAMES="fieldnames";
   public static final String HEADER="header";
@@ -59,7 +59,7 @@ abstract class CSVLoaderBase extends ContentStreamLoader {
   
   final SolrParams params;
   final CSVStrategy strategy;
-  final UpdateRequestProcessor processor;
+  protected final UpdateRequestProcessor processor;
   // hashmap to save any literal fields and their values
   HashMap <String, String> literals;
 
@@ -71,7 +71,7 @@ abstract class CSVLoaderBase extends ContentStreamLoader {
 
   int skipLines;    // number of lines to skip at start of file
 
-  final AddUpdateCommand templateAdd;
+  protected final AddUpdateCommand templateAdd;
 
 
 
@@ -157,7 +157,7 @@ abstract class CSVLoaderBase extends ContentStreamLoader {
 
   String errHeader="CSVLoader:";
 
-  CSVLoaderBase(SolrQueryRequest req, UpdateRequestProcessor processor) {
+  protected CSVLoaderBase(SolrQueryRequest req, UpdateRequestProcessor processor) {
     this.processor = processor;
     this.params = req.getParams();
     this.literals = new HashMap<>();
@@ -368,10 +368,10 @@ abstract class CSVLoaderBase extends ContentStreamLoader {
   }
 
   /** called for each line of values (document) */
-  abstract void addDoc(int line, String[] vals) throws IOException;
+  public abstract void addDoc(int line, String[] vals) throws IOException;
 
   /** this must be MT safe... may be called concurrently from multiple threads. */
-  void doAdd(int line, String[] vals, SolrInputDocument doc, AddUpdateCommand template) throws IOException {
+  protected void doAdd(int line, String[] vals, SolrInputDocument doc, AddUpdateCommand template) throws IOException {
     // the line number is passed for error reporting in MT mode as well as for optional rowId.
     // first, create the lucene document
     for (int i=0; i<vals.length; i++) {
diff --git a/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java b/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java
index 2964834..6791f0a 100644
--- a/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java
+++ b/solr/core/src/java/org/apache/solr/handler/loader/JsonLoader.java
@@ -84,6 +84,41 @@ public class JsonLoader extends ContentStreamLoader {
     new SingleThreadedJsonLoader(req, rsp, processor).load(req, rsp, stream, processor);
   }
 
+  @SuppressWarnings({"unchecked", "rawtypes"})
+  public static SolrInputDocument buildDoc(Map<String, Object> m) {
+    SolrInputDocument result = new SolrInputDocument();
+    for (Map.Entry<String, Object> e : m.entrySet()) {
+      if (mapEntryIsChildDoc(e.getValue())) { // parse child documents
+        if (e.getValue() instanceof List) {
+          List value = (List) e.getValue();
+          for (Object o : value) {
+            if (o instanceof Map) {
+              // retain the value as a list, even if the list contains a single value.
+              if(!result.containsKey(e.getKey())) {
+                result.setField(e.getKey(), new ArrayList<>(1));
+              }
+              result.addField(e.getKey(), buildDoc((Map) o));
+            }
+          }
+        } else if (e.getValue() instanceof Map) {
+          result.addField(e.getKey(), buildDoc((Map) e.getValue()));
+        }
+      } else {
+        result.setField(e.getKey(), e.getValue());
+      }
+    }
+    return result;
+  }
+
+  private static boolean mapEntryIsChildDoc(Object val) {
+    if(val instanceof List) {
+      @SuppressWarnings({"rawtypes"})
+      List listVal = (List) val;
+      if (listVal.size() == 0) return false;
+      return  listVal.get(0) instanceof Map;
+    }
+    return val instanceof Map;
+  }
 
   static class SingleThreadedJsonLoader extends ContentStreamLoader {
 
@@ -248,32 +283,6 @@ public class JsonLoader extends ContentStreamLoader {
     }
 
     @SuppressWarnings({"unchecked", "rawtypes"})
-    private SolrInputDocument buildDoc(Map<String, Object> m) {
-      SolrInputDocument result = new SolrInputDocument();
-      for (Map.Entry<String, Object> e : m.entrySet()) {
-        if (mapEntryIsChildDoc(e.getValue())) { // parse child documents
-          if (e.getValue() instanceof List) {
-            List value = (List) e.getValue();
-            for (Object o : value) {
-              if (o instanceof Map) {
-                // retain the value as a list, even if the list contains a single value.
-                if(!result.containsKey(e.getKey())) {
-                  result.setField(e.getKey(), new ArrayList<>(1));
-                }
-                result.addField(e.getKey(), buildDoc((Map) o));
-              }
-            }
-          } else if (e.getValue() instanceof Map) {
-            result.addField(e.getKey(), buildDoc((Map) e.getValue()));
-          }
-        } else {
-          result.setField(e.getKey(), e.getValue());
-        }
-      }
-      return result;
-    }
-
-    @SuppressWarnings({"unchecked", "rawtypes"})
     private Map<String, Object> getDocMap(Map<String, Object> record, JSONParser parser, String srcField, boolean mapUniqueKeyOnly) {
       Map result = record;
       if (srcField != null && parser instanceof RecordingJSONParser) {
@@ -625,15 +634,6 @@ public class JsonLoader extends ContentStreamLoader {
       return extendedFieldValue.containsKey(req.getSchema().getUniqueKeyField().getName());
     }
 
-    private boolean mapEntryIsChildDoc(Object val) {
-      if(val instanceof List) {
-        @SuppressWarnings({"rawtypes"})
-        List listVal = (List) val;
-        if (listVal.size() == 0) return false;
-        return  listVal.get(0) instanceof Map;
-      }
-      return val instanceof Map;
-    }
   }
 
   @SuppressWarnings({"unchecked", "rawtypes"})
diff --git a/solr/core/src/java/org/apache/solr/update/processor/ParseBooleanFieldUpdateProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/ParseBooleanFieldUpdateProcessorFactory.java
index ea1863f..cc25612 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/ParseBooleanFieldUpdateProcessorFactory.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/ParseBooleanFieldUpdateProcessorFactory.java
@@ -73,8 +73,8 @@ public class ParseBooleanFieldUpdateProcessorFactory extends FieldMutatingUpdate
   private static final String FALSE_VALUES_PARAM = "falseValue";
   private static final String CASE_SENSITIVE_PARAM = "caseSensitive";
   
-  private Set<String> trueValues = new HashSet<>(Arrays.asList(new String[] { "true" }));
-  private Set<String> falseValues = new HashSet<>(Arrays.asList(new String[] { "false" }));
+  private final Set<String> trueValues = new HashSet<>(Arrays.asList("true"));
+  private final Set<String> falseValues = new HashSet<>(Arrays.asList("false"));
   private boolean caseSensitive = false;
 
   @Override
@@ -84,20 +84,8 @@ public class ParseBooleanFieldUpdateProcessorFactory extends FieldMutatingUpdate
     return new AllValuesOrNoneFieldMutatingUpdateProcessor(getSelector(), next) {
       @Override
       protected Object mutateValue(Object srcVal) {
-        if (srcVal instanceof CharSequence) {
-          String stringVal = caseSensitive ? srcVal.toString() : srcVal.toString().toLowerCase(Locale.ROOT);
-          if (trueValues.contains(stringVal)) {
-            return true;
-          } else if (falseValues.contains(stringVal)) {
-            return false;
-          } else {
-            return SKIP_FIELD_VALUE_LIST_SINGLETON;
-          }
-        }
-        if (srcVal instanceof Boolean) {
-          return srcVal;
-        }
-        return SKIP_FIELD_VALUE_LIST_SINGLETON;
+        Object parsed = parsePossibleBoolean(srcVal, caseSensitive, trueValues, falseValues);
+        return parsed != null ? parsed : SKIP_FIELD_VALUE_LIST_SINGLETON;
       }
     };
   }
@@ -109,7 +97,7 @@ public class ParseBooleanFieldUpdateProcessorFactory extends FieldMutatingUpdate
       if (caseSensitiveParam instanceof Boolean) {
         caseSensitive = (Boolean)caseSensitiveParam;
       } else {
-        caseSensitive = Boolean.valueOf(caseSensitiveParam.toString());
+        caseSensitive = Boolean.parseBoolean(caseSensitiveParam.toString());
       }
     }
 
@@ -152,4 +140,21 @@ public class ParseBooleanFieldUpdateProcessorFactory extends FieldMutatingUpdate
       return (null == type) || (type instanceof BoolField);
     };
   }
+
+  public static Object parsePossibleBoolean(Object srcVal, boolean caseSensitive, Set<String> trueValues, Set<String> falseValues) {
+    if (srcVal instanceof CharSequence) {
+      String stringVal = caseSensitive ? srcVal.toString() : srcVal.toString().toLowerCase(Locale.ROOT);
+      if (trueValues.contains(stringVal)) {
+        return true;
+      } else if (falseValues.contains(stringVal)) {
+        return false;
+      } else {
+        return null;
+      }
+    }
+    if (srcVal instanceof Boolean) {
+      return srcVal;
+    }
+    return null;
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/update/processor/ParseDateFieldUpdateProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/ParseDateFieldUpdateProcessorFactory.java
index bb5a88c..b31d429 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/ParseDateFieldUpdateProcessorFactory.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/ParseDateFieldUpdateProcessorFactory.java
@@ -131,36 +131,8 @@ public class ParseDateFieldUpdateProcessorFactory extends FieldMutatingUpdatePro
 
       @Override
       protected Object mutateValue(Object srcVal) {
-        if (srcVal instanceof CharSequence) {
-          String srcStringVal = srcVal.toString();
-          // trim single quotes around date if present
-          // see issue #5279  (Apache HttpClient)
-          int stringValLen = srcStringVal.length();
-          if (stringValLen > 1
-              && srcStringVal.startsWith("'")
-              && srcStringVal.endsWith("'")
-          ) {
-            srcStringVal = srcStringVal.substring(1, stringValLen - 1);
-          }
-
-          for (Map.Entry<String,DateTimeFormatter> format : formats.entrySet()) {
-            DateTimeFormatter parser = format.getValue();
-            try {
-              return Date.from(parseInstant(parser, srcStringVal, parsePosition));
-            } catch (DateTimeParseException e) {
-              if (log.isDebugEnabled()) {
-                log.debug("value '{}' is not parseable with format '{}'",
-                    new Object[]{srcStringVal, format.getKey()});
-              }
-            }
-          }
-          log.debug("value '{}' was not parsed by any configured format, thus was not mutated", srcStringVal);
-          return SKIP_FIELD_VALUE_LIST_SINGLETON;
-        }
-        if (srcVal instanceof Date) {
-          return srcVal;
-        }
-        return SKIP_FIELD_VALUE_LIST_SINGLETON;
+        Object parsed = parsePossibleDate(srcVal, formats.values(), parsePosition);
+        return parsed != null ? parsed : SKIP_FIELD_VALUE_LIST_SINGLETON;
       }
     };
   }
@@ -211,6 +183,37 @@ public class ParseDateFieldUpdateProcessorFactory extends FieldMutatingUpdatePro
     };
   }
 
+  public static Object parsePossibleDate(Object srcVal, Collection<DateTimeFormatter> parsers, ParsePosition parsePosition) {
+    if (srcVal instanceof CharSequence) {
+      String srcStringVal = srcVal.toString();
+      // trim single quotes around date if present
+      // see issue #5279  (Apache HttpClient)
+      int stringValLen = srcStringVal.length();
+      if (stringValLen > 1
+          && srcStringVal.startsWith("'")
+          && srcStringVal.endsWith("'")
+      ) {
+        srcStringVal = srcStringVal.substring(1, stringValLen - 1);
+      }
+
+      for (DateTimeFormatter parser: parsers) {
+        try {
+          return Date.from(parseInstant(parser, srcStringVal, parsePosition));
+        } catch (DateTimeParseException e) {
+          if (log.isDebugEnabled()) {
+            log.debug("value '{}' is not parseable with format '{}'", srcStringVal, parser);
+          }
+        }
+      }
+      log.debug("value '{}' was not parsed by any configured format, thus was not mutated", srcStringVal);
+      return null;
+    }
+    if (srcVal instanceof Date) {
+      return srcVal;
+    }
+    return null;
+  }
+
   public static void validateFormatter(DateTimeFormatter formatter) {
     // check it's valid via round-trip
     try {
diff --git a/solr/core/src/java/org/apache/solr/update/processor/ParseDoubleFieldUpdateProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/ParseDoubleFieldUpdateProcessorFactory.java
index ad7eabb..af4b12f 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/ParseDoubleFieldUpdateProcessorFactory.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/ParseDoubleFieldUpdateProcessorFactory.java
@@ -97,23 +97,8 @@ public class ParseDoubleFieldUpdateProcessorFactory extends ParseNumericFieldUpd
 
     @Override
     protected Object mutateValue(Object srcVal) {
-      if (srcVal instanceof CharSequence) {
-        String stringVal = srcVal.toString(); 
-        ParsePosition pos = new ParsePosition(0);
-        Number number = numberFormat.get().parse(stringVal, pos);
-        if (pos.getIndex() != stringVal.length()) {
-          if (log.isDebugEnabled()) {
-            log.debug("value '{}' is not parseable, thus not mutated; unparsed chars: '{}'",
-                new Object[]{srcVal, stringVal.substring(pos.getIndex())});
-          }
-          return SKIP_FIELD_VALUE_LIST_SINGLETON;
-        }
-        return number.doubleValue();
-      }
-      if (srcVal instanceof Double) {
-        return srcVal;
-      }
-      return SKIP_FIELD_VALUE_LIST_SINGLETON;
+      Object parsed = parsePossibleDouble(srcVal, numberFormat.get());
+      return parsed != null ? parsed : SKIP_FIELD_VALUE_LIST_SINGLETON;
     }
   }
 
@@ -121,4 +106,27 @@ public class ParseDoubleFieldUpdateProcessorFactory extends ParseNumericFieldUpd
   protected boolean isSchemaFieldTypeCompatible(FieldType type) {
     return type instanceof DoubleValueFieldType;
   }
+
+  public static Object parsePossibleDouble(Object srcVal, NumberFormat numberFormat) {
+    if (srcVal instanceof CharSequence) {
+      String stringVal = srcVal.toString();
+      ParsePosition pos = new ParsePosition(0);
+      Number number = numberFormat.parse(stringVal, pos);
+      if (pos.getIndex() != stringVal.length()) {
+        if (log.isDebugEnabled()) {
+          log.debug("value '{}' is not parseable, thus not mutated; unparsed chars: '{}'",
+              srcVal, stringVal.substring(pos.getIndex()));
+        }
+        return null;
+      }
+      return number.doubleValue();
+    }
+    if (srcVal instanceof Double) {
+      return srcVal;
+    }
+    if (srcVal instanceof Float) {
+      return ((Float) srcVal).doubleValue();
+    }
+    return null;
+  }
 }
diff --git a/solr/core/src/java/org/apache/solr/update/processor/ParseLongFieldUpdateProcessorFactory.java b/solr/core/src/java/org/apache/solr/update/processor/ParseLongFieldUpdateProcessorFactory.java
index b93a28f..ee8ce0a 100644
--- a/solr/core/src/java/org/apache/solr/update/processor/ParseLongFieldUpdateProcessorFactory.java
+++ b/solr/core/src/java/org/apache/solr/update/processor/ParseLongFieldUpdateProcessorFactory.java
@@ -93,23 +93,8 @@ public class ParseLongFieldUpdateProcessorFactory extends ParseNumericFieldUpdat
     
     @Override
     protected Object mutateValue(Object srcVal) {
-      if (srcVal instanceof CharSequence) {
-        String stringVal = srcVal.toString();
-        ParsePosition pos = new ParsePosition(0);
-        Number number = numberFormat.get().parse(stringVal, pos);
-        if (pos.getIndex() != stringVal.length()) {
-          if (log.isDebugEnabled()) {
-            log.debug("value '{}' is not parseable, thus not mutated; unparsed chars: '{}'",
-                new Object[]{srcVal, stringVal.substring(pos.getIndex())});
-          }
-          return SKIP_FIELD_VALUE_LIST_SINGLETON;
-        }
-        return number.longValue();
-      }
-      if (srcVal instanceof Long) {
-        return srcVal;
-      }
-      return SKIP_FIELD_VALUE_LIST_SINGLETON;
+      Object parsed = parsePossibleLong(srcVal, numberFormat.get());
+      return parsed != null ? parsed : SKIP_FIELD_VALUE_LIST_SINGLETON;
     }
   }
 
@@ -117,4 +102,24 @@ public class ParseLongFieldUpdateProcessorFactory extends ParseNumericFieldUpdat
   protected boolean isSchemaFieldTypeCompatible(FieldType type) {
     return type instanceof LongValueFieldType;
   }
+
+  public static Object parsePossibleLong(Object srcVal, NumberFormat numberFormat) {
+    if (srcVal instanceof CharSequence) {
+      String stringVal = srcVal.toString();
+      ParsePosition pos = new ParsePosition(0);
+      Number number = numberFormat.parse(stringVal, pos);
+      if (pos.getIndex() != stringVal.length()) {
+        if (log.isDebugEnabled()) {
+          log.debug("value '{}' is not parseable, thus not mutated; unparsed chars: '{}'",
+              srcVal, stringVal.substring(pos.getIndex()));
+        }
+        return null;
+      }
+      return number.longValue();
+    }
+    if (srcVal instanceof Long) {
+      return srcVal;
+    }
+    return null;
+  }
 }
diff --git a/solr/core/src/test-files/schema-designer/add-new-field.json b/solr/core/src/test-files/schema-designer/add-new-field.json
new file mode 100644
index 0000000..c25575f
--- /dev/null
+++ b/solr/core/src/test-files/schema-designer/add-new-field.json
@@ -0,0 +1,9 @@
+{
+  "add-field": {
+    "name":"keywords",
+    "type":"string",
+    "stored":true,
+    "docValues": true,
+    "indexed": false
+  }
+}
\ No newline at end of file
diff --git a/solr/core/src/test-files/schema-designer/add-new-type.json b/solr/core/src/test-files/schema-designer/add-new-type.json
new file mode 100644
index 0000000..e5d395e
--- /dev/null
+++ b/solr/core/src/test-files/schema-designer/add-new-type.json
@@ -0,0 +1,41 @@
+{ "add-field-type": {
+  "name": "test_txt",
+  "class": "solr.TextField",
+  "positionIncrementGap": "100",
+  "autoGeneratePhraseQueries": false,
+  "enableGraphQueries": true,
+  "synonymQueryStyle": "AS_SAME_TERM",
+  "indexed": true,
+  "stored": true,
+  "docValues": false,
+  "termVectors": false,
+  "omitNorms": false,
+  "omitTermFreqAndPositions": false,
+  "omitPositions": false,
+  "storeOffsetsWithPositions": false,
+  "multiValued": false,
+  "large": false,
+  "uninvertible": true,
+  "tokenized": true,
+  "indexAnalyzer": {
+    "tokenizer": {
+      "class": "solr.WhitespaceTokenizerFactory"
+    },
+    "filters": [
+      {
+        "class": "solr.ASCIIFoldingFilterFactory"
+      }
+    ]
+  },
+  "queryAnalyzer": {
+    "tokenizer": {
+      "class": "solr.WhitespaceTokenizerFactory"
+    },
+    "filters": [
+      {
+        "class": "solr.ASCIIFoldingFilterFactory"
+      }
+    ]
+  }
+}
+}
diff --git a/solr/core/src/test-files/schema-designer/update-author-field.json b/solr/core/src/test-files/schema-designer/update-author-field.json
new file mode 100644
index 0000000..13d06cd
--- /dev/null
+++ b/solr/core/src/test-files/schema-designer/update-author-field.json
@@ -0,0 +1,11 @@
+{
+  "href": "field/author",
+  "name": "author",
+  "type": "string",
+  "indexed": true,
+  "stored": true,
+  "docValues": true,
+  "multiValued": true,
+  "id": "field/author_anchor",
+  "copyDest": "_text_"
+}
\ No newline at end of file
diff --git a/solr/core/src/test-files/schema-designer/update-type.json b/solr/core/src/test-files/schema-designer/update-type.json
new file mode 100644
index 0000000..b685bd9
--- /dev/null
+++ b/solr/core/src/test-files/schema-designer/update-type.json
@@ -0,0 +1,41 @@
+{
+  "name": "test_txt",
+  "class": "solr.TextField",
+  "positionIncrementGap": "100",
+  "enableGraphQueries": true,
+  "autoGeneratePhraseQueries": false,
+  "synonymQueryStyle": "org.apache.solr.parser.SolrQueryParserBase$SynonymQueryStyle:AS_SAME_TERM",
+  "indexed": true,
+  "stored": true,
+  "docValues": false,
+  "termVectors": false,
+  "omitNorms": false,
+  "omitTermFreqAndPositions": false,
+  "omitPositions": false,
+  "storeOffsetsWithPositions": false,
+  "multiValued": true,
+  "large": false,
+  "uninvertible": true,
+  "tokenized": true,
+  "indexAnalyzer": {
+    "tokenizer": {
+      "class": "solr.WhitespaceTokenizerFactory"
+    },
+    "filters": [
+      {
+        "class": "solr.ASCIIFoldingFilterFactory"
+      }
+    ]
+
+  },
+  "queryAnalyzer": {
+    "tokenizer": {
+      "class": "solr.WhitespaceTokenizerFactory"
+    },
+    "filters": [
+      {
+        "class": "solr.ASCIIFoldingFilterFactory"
+      }
+    ]
+  }
+}
\ No newline at end of file
diff --git a/solr/core/src/test/org/apache/solr/handler/TestSampleDocumentsLoader.java b/solr/core/src/test/org/apache/solr/handler/TestSampleDocumentsLoader.java
new file mode 100644
index 0000000..057b97f
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/TestSampleDocumentsLoader.java
@@ -0,0 +1,123 @@
+/*
+ * 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.handler;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.SolrInputField;
+import org.apache.solr.common.params.ModifiableSolrParams;
+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.NamedList;
+import org.apache.solr.handler.designer.DefaultSampleDocumentsLoader;
+import org.apache.solr.handler.designer.SampleDocuments;
+import org.apache.solr.handler.designer.SampleDocumentsLoader;
+import org.apache.solr.util.ExternalPaths;
+import org.junit.Before;
+import org.junit.Test;
+
+import static org.apache.solr.handler.designer.DefaultSampleDocumentsLoader.CSV_MULTI_VALUE_DELIM_PARAM;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestSampleDocumentsLoader {
+
+  SampleDocumentsLoader loader;
+  File exampleDir;
+
+  @Before
+  public void setup() throws IOException {
+    loader = new DefaultSampleDocumentsLoader();
+    loader.init(new NamedList<>());
+    exampleDir = new File(ExternalPaths.SOURCE_HOME, "example");
+    assertTrue("Required test data directory " + exampleDir.getCanonicalPath() + " not found!", exampleDir.isDirectory());
+  }
+
+  @Test
+  public void testJson() throws Exception {
+    loadTestDocs(null, new File(exampleDir, "films/films.json"), 500, 500);
+  }
+
+  @Test
+  public void testCsv() throws Exception {
+    ModifiableSolrParams params = new ModifiableSolrParams();
+    params.set(CSV_MULTI_VALUE_DELIM_PARAM, "\\|");
+    List<SolrInputDocument> docs = loadTestDocs(params, new File(exampleDir, "films/films.csv"), -1, 1100);
+    boolean foundIt = false;
+    for (SolrInputDocument next : docs) {
+      if (".45".equals(next.getFieldValue("name"))) {
+        SolrInputField genreField = next.getField("genre");
+        assertNotNull(genreField);
+        assertEquals(8, genreField.getValueCount());
+        foundIt = true;
+        break;
+      }
+    }
+    if (!foundIt) {
+      fail("Didn't find the expected film with name '.45' in the parsed docs!");
+    }
+  }
+
+  @Test
+  public void testSolrXml() throws Exception {
+    loadTestDocs(null, new File(exampleDir, "films/films.xml"), 1000, 1000);
+  }
+
+  protected List<SolrInputDocument> loadTestDocs(SolrParams params, File inputDocs, int maxDocsToLoad, int expectedDocs) throws IOException {
+    assertTrue(inputDocs.getCanonicalPath() + " not found", inputDocs.isFile());
+    ContentStream stream = getContentStream(inputDocs);
+    SampleDocuments sampleDocs = loader.parseDocsFromStream(params, stream, maxDocsToLoad);
+    assertNotNull(sampleDocs);
+    assertEquals(sampleDocs.parsed.size(), expectedDocs);
+    return sampleDocs.parsed;
+  }
+
+  public static String guessContentTypeFromFilename(String name) {
+    int dotAt = name.lastIndexOf('.');
+    if (dotAt != -1) {
+      final String ext = name.substring(dotAt + 1);
+      switch (ext) {
+        case "csv":
+          return "text/csv";
+        case "txt":
+          return "text/plain";
+        case "json":
+          return "application/json";
+        case "xml":
+          return "text/xml";
+      }
+    }
+    return "application/octet-stream";
+  }
+
+  protected ContentStream getContentStream(File file) {
+    return getContentStream(file, guessContentTypeFromFilename(file.getName()));
+  }
+
+  protected ContentStream getContentStream(File file, String contentType) {
+    ContentStreamBase.FileStream stream = new ContentStreamBase.FileStream(file);
+    stream.setContentType(contentType);
+    return stream;
+  }
+}
diff --git a/solr/core/src/test/org/apache/solr/handler/designer/ManagedSchemaDiffTest.java b/solr/core/src/test/org/apache/solr/handler/designer/ManagedSchemaDiffTest.java
new file mode 100644
index 0000000..8b8b271
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/designer/ManagedSchemaDiffTest.java
@@ -0,0 +1,117 @@
+/*
+ * 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.handler.designer;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.schema.BoolField;
+import org.apache.solr.schema.IntPointField;
+import org.apache.solr.schema.ManagedIndexSchema;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.util.ExternalPaths;
+import org.junit.Assert;
+import org.junit.BeforeClass;
+
+import static org.apache.solr.common.util.Utils.makeMap;
+import static org.apache.solr.handler.admin.ConfigSetsHandler.DEFAULT_CONFIGSET_NAME;
+import static org.apache.solr.handler.designer.ManagedSchemaDiff.mapFieldsToPropertyValues;
+
+public class ManagedSchemaDiffTest extends SolrCloudTestCase {
+
+  @BeforeClass
+  public static void createCluster() throws Exception {
+    System.setProperty("managed.schema.mutable", "true");
+    configureCluster(1).addConfig(DEFAULT_CONFIGSET_NAME, new File(ExternalPaths.DEFAULT_CONFIGSET).toPath()).configure();
+  }
+
+  public void testFieldDiff() {
+    SchemaDesignerConfigSetHelper helper =
+        new SchemaDesignerConfigSetHelper(cluster.getJettySolrRunner(0).getCoreContainer(), null);
+    ManagedIndexSchema schema = helper.loadLatestSchema(DEFAULT_CONFIGSET_NAME);
+
+    Map<String, SchemaField> schema1FieldMap = new HashMap<>();
+    schema1FieldMap.put("strfield", schema.newField("strfield", "string", Collections.emptyMap()));
+    schema1FieldMap.put("boolfield", new SchemaField("boolfield", new BoolField()));
+
+    Map<String, SchemaField> schema2FieldMap = new HashMap<>();
+    schema2FieldMap.put("strfield", schema.newField("strfield", "strings", Collections.emptyMap()));
+    schema2FieldMap.put("intfield", new SchemaField("intfield", new IntPointField()));
+
+    Map<String, Object> diff = ManagedSchemaDiff.diff(mapFieldsToPropertyValues(schema1FieldMap), mapFieldsToPropertyValues(schema2FieldMap));
+    Assert.assertTrue(diff.containsKey("updated"));
+    Assert.assertTrue(diff.containsKey("added"));
+    Assert.assertTrue(diff.containsKey("removed"));
+
+    Map<String, Object> changedFields = getInnerMap(diff, "updated");
+    Assert.assertEquals(1, changedFields.size());
+    Assert.assertTrue(changedFields.containsKey("strfield"));
+    Assert.assertEquals(
+        Arrays.asList(makeMap("type", "string", "multiValued", false),
+            makeMap("type", "strings", "multiValued", true)),
+        changedFields.get("strfield"));
+
+    Map<String, Object> addedFields = getInnerMap(diff, "added");
+    Assert.assertEquals(1, addedFields.size());
+    Assert.assertTrue(addedFields.containsKey("intfield"));
+    Assert.assertEquals(schema2FieldMap.get("intfield").getNamedPropertyValues(true), addedFields.get("intfield"));
+
+    Map<String, Object> removedFields = getInnerMap(diff, "removed");
+    Assert.assertEquals(1, removedFields.size());
+    Assert.assertTrue(removedFields.containsKey("boolfield"));
+    Assert.assertEquals(schema1FieldMap.get("boolfield").getNamedPropertyValues(true), removedFields.get("boolfield"));
+  }
+
+  public void testSimpleOrderedMapListDiff() {
+    SimpleOrderedMap<Object> obj1 = new SimpleOrderedMap<>();
+    obj1.add("name", "obj1");
+    obj1.add("type", "objtype1");
+
+    SimpleOrderedMap<Object> obj2 = new SimpleOrderedMap<>();
+    obj2.add("name", "obj2");
+    obj2.add("type", "objtype2");
+
+    SimpleOrderedMap<Object> obj3 = new SimpleOrderedMap<>();
+    obj3.add("name", "obj3");
+    obj3.add("type", "objtype3");
+
+    SimpleOrderedMap<Object> obj4 = new SimpleOrderedMap<>();
+    obj4.add("name", "obj4");
+    obj4.add("type", "objtype4");
+
+    List<SimpleOrderedMap<Object>> list1 = Arrays.asList(obj1, obj2);
+    List<SimpleOrderedMap<Object>> list2 = Arrays.asList(obj1, obj3, obj4);
+
+    Map<String, Object> diff = ManagedSchemaDiff.diff(list1, list2);
+    Assert.assertTrue(diff.containsKey("old"));
+    Assert.assertTrue(diff.containsKey("new"));
+    Assert.assertEquals(Collections.singletonList(obj2), diff.get("old"));
+    Assert.assertEquals(Arrays.asList(obj3, obj4), diff.get("new"));
+  }
+
+  @SuppressWarnings("unchecked")
+  private Map<String, Object> getInnerMap(Map<String, Object> map, String key) {
+    return (Map<String, Object>) map.get(key);
+  }
+}
diff --git a/solr/core/src/test/org/apache/solr/handler/designer/TestSchemaDesignerAPI.java b/solr/core/src/test/org/apache/solr/handler/designer/TestSchemaDesignerAPI.java
new file mode 100644
index 0000000..4bae61c
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/designer/TestSchemaDesignerAPI.java
@@ -0,0 +1,874 @@
+/*
+ * 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.handler.designer;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Optional;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.solr.client.solrj.SolrQuery;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.client.solrj.response.QueryResponse;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrDocumentList;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.cloud.SolrZkClient;
+import org.apache.solr.common.params.CommonParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+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.NamedList;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.TestSampleDocumentsLoader;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.schema.ManagedIndexSchema;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.util.ExternalPaths;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.noggit.JSONUtil;
+
+import static org.apache.solr.common.params.CommonParams.JSON_MIME;
+import static org.apache.solr.common.util.Utils.makeMap;
+import static org.apache.solr.handler.admin.ConfigSetsHandler.DEFAULT_CONFIGSET_NAME;
+import static org.apache.solr.handler.designer.SchemaDesignerAPI.getMutableId;
+import static org.apache.solr.response.RawResponseWriter.CONTENT;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+public class TestSchemaDesignerAPI extends SolrCloudTestCase implements SchemaDesignerConstants {
+
+  private CoreContainer cc;
+  private SchemaDesignerAPI schemaDesignerAPI;
+
+  @BeforeClass
+  public static void createCluster() throws Exception {
+    System.setProperty("managed.schema.mutable", "true");
+    configureCluster(1).addConfig(DEFAULT_CONFIGSET_NAME, new File(ExternalPaths.DEFAULT_CONFIGSET).toPath()).configure();
+    // SchemaDesignerAPI depends on the blob store
+    CollectionAdminRequest.createCollection(BLOB_STORE_ID, 1, 1).process(cluster.getSolrClient());
+    cluster.waitForActiveCollection(BLOB_STORE_ID, 1, 1);
+  }
+
+  @AfterClass
+  public static void tearDownCluster() throws Exception {
+    if (cluster != null && cluster.getSolrClient() != null) {
+      cluster.deleteAllCollections();
+      cluster.deleteAllConfigSets();
+    }
+  }
+
+  @Before
+  public void setupTest() {
+    assumeWorkingMockito();
+    assertNotNull(cluster);
+    cc = cluster.getJettySolrRunner(0).getCoreContainer();
+    assertNotNull(cc);
+    schemaDesignerAPI = new SchemaDesignerAPI(cc);
+  }
+
+  public void testTSV() throws Exception {
+    String configSet = "testTSV";
+
+    ModifiableSolrParams reqParams = new ModifiableSolrParams();
+
+    // GET /schema-designer/info
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    SolrQueryRequest req = mock(SolrQueryRequest.class);
+
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    reqParams.set(LANGUAGES_PARAM, "en");
+    reqParams.set(ENABLE_DYNAMIC_FIELDS_PARAM, false);
+    when(req.getParams()).thenReturn(reqParams);
+
+    String tsv = "id\tcol1\tcol2\n1\tfoo\tbar\n2\tbaz\tbah\n";
+
+    // POST some sample TSV docs
+    ContentStream stream = new ContentStreamBase.StringStream(tsv, "text/csv");
+    when(req.getContentStreams()).thenReturn(Collections.singletonList(stream));
+
+    // POST /schema-designer/analyze
+    schemaDesignerAPI.analyze(req, rsp);
+    assertNotNull(rsp.getValues().get(CONFIG_SET_PARAM));
+    assertNotNull(rsp.getValues().get(SCHEMA_VERSION_PARAM));
+    assertEquals(2, rsp.getValues().get("numDocs"));
+
+    reqParams.clear();
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    rsp = new SolrQueryResponse();
+    schemaDesignerAPI.cleanupTemp(req, rsp);
+
+    String mutableId = getMutableId(configSet);
+    assertFalse(cc.getZkController().getClusterState().hasCollection(mutableId));
+    SolrZkClient zkClient = cc.getZkController().getZkClient();
+    assertFalse(zkClient.exists("/configs/" + mutableId, true));
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testAddTechproductsProgressively() throws Exception {
+    File docsDir = new File(ExternalPaths.SOURCE_HOME, "example/exampledocs");
+    assertTrue(docsDir.getAbsolutePath() + " not found!", docsDir.isDirectory());
+    File[] toAdd = docsDir.listFiles((dir, name) -> name.endsWith(".xml") || name.endsWith(".json") || name.endsWith(".csv") || name.endsWith(".jsonl"));
+    assertNotNull("No test data files found in " + docsDir.getAbsolutePath(), toAdd);
+
+    String configSet = "techproducts";
+
+    ModifiableSolrParams reqParams = new ModifiableSolrParams();
+
+    // GET /schema-designer/info
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    SolrQueryRequest req = mock(SolrQueryRequest.class);
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    when(req.getParams()).thenReturn(reqParams);
+    schemaDesignerAPI.getInfo(req, rsp);
+    // response should just be the default values
+    Map<String, Object> expSettings = makeMap(
+        ENABLE_DYNAMIC_FIELDS_PARAM, true,
+        ENABLE_FIELD_GUESSING_PARAM, true,
+        ENABLE_NESTED_DOCS_PARAM, false,
+        LANGUAGES_PARAM, Collections.emptyList());
+    assertDesignerSettings(expSettings, rsp.getValues());
+    SolrParams rspData = rsp.getValues().toSolrParams();
+    int schemaVersion = rspData.getInt(SCHEMA_VERSION_PARAM);
+    assertEquals(schemaVersion, -1); // shouldn't exist yet
+
+    // Use the prep endpoint to prepare the new schema
+    reqParams.clear();
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    rsp = new SolrQueryResponse();
+    req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    schemaDesignerAPI.prepNewSchema(req, rsp);
+    assertNotNull(rsp.getValues().get(CONFIG_SET_PARAM));
+    assertNotNull(rsp.getValues().get(SCHEMA_VERSION_PARAM));
+    rspData = rsp.getValues().toSolrParams();
+    schemaVersion = rspData.getInt(SCHEMA_VERSION_PARAM);
+
+    for (File next : toAdd) {
+      // Analyze some sample documents to refine the schema
+      reqParams.clear();
+      reqParams.set(CONFIG_SET_PARAM, configSet);
+      reqParams.set(LANGUAGES_PARAM, "en");
+      reqParams.set(ENABLE_DYNAMIC_FIELDS_PARAM, false);
+      reqParams.set(SCHEMA_VERSION_PARAM, String.valueOf(schemaVersion));
+      req = mock(SolrQueryRequest.class);
+      when(req.getParams()).thenReturn(reqParams);
+
+      // POST some sample JSON docs
+      ContentStreamBase.FileStream stream = new ContentStreamBase.FileStream(next);
+      stream.setContentType(TestSampleDocumentsLoader.guessContentTypeFromFilename(next.getName()));
+      when(req.getContentStreams()).thenReturn(Collections.singletonList(stream));
+
+      rsp = new SolrQueryResponse();
+
+      // POST /schema-designer/analyze
+      schemaDesignerAPI.analyze(req, rsp);
+
+      assertNotNull(rsp.getValues().get(CONFIG_SET_PARAM));
+      assertNotNull(rsp.getValues().get(SCHEMA_VERSION_PARAM));
+      assertNotNull(rsp.getValues().get("fields"));
+      assertNotNull(rsp.getValues().get("fieldTypes"));
+      assertNotNull(rsp.getValues().get("docIds"));
+
+      // capture the schema version for MVCC
+      rspData = rsp.getValues().toSolrParams();
+      schemaVersion = rspData.getInt(SCHEMA_VERSION_PARAM);
+    }
+
+    // get info (from the temp)
+    reqParams.clear();
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    rsp = new SolrQueryResponse();
+
+    // GET /schema-designer/info
+    schemaDesignerAPI.getInfo(req, rsp);
+    expSettings = makeMap(
+        ENABLE_DYNAMIC_FIELDS_PARAM, false,
+        ENABLE_FIELD_GUESSING_PARAM, true,
+        ENABLE_NESTED_DOCS_PARAM, false,
+        LANGUAGES_PARAM, Collections.singletonList("en"),
+        COPY_FROM_PARAM, "_default");
+    assertDesignerSettings(expSettings, rsp.getValues());
+
+    // query to see how the schema decisions impact retrieval / ranking
+    reqParams.clear();
+    reqParams.set(SCHEMA_VERSION_PARAM, String.valueOf(schemaVersion));
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    reqParams.set(CommonParams.Q, "*:*");
+    req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    rsp = new SolrQueryResponse();
+
+    // GET /schema-designer/query
+    schemaDesignerAPI.query(req, rsp);
+    assertNotNull(rsp.getResponseHeader());
+    SolrDocumentList results = (SolrDocumentList) rsp.getResponse();
+    assertEquals(48, results.getNumFound());
+
+    // publish schema to a config set that can be used by real collections
+    reqParams.clear();
+    reqParams.set(SCHEMA_VERSION_PARAM, String.valueOf(schemaVersion));
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+
+    String collection = "techproducts";
+    reqParams.set(NEW_COLLECTION_PARAM, collection);
+    reqParams.set(INDEX_TO_COLLECTION_PARAM, true);
+    reqParams.set(RELOAD_COLLECTIONS_PARAM, true);
+    reqParams.set(CLEANUP_TEMP_PARAM, true);
+    reqParams.set(DISABLE_DESIGNER_PARAM, true);
+
+    rsp = new SolrQueryResponse();
+    schemaDesignerAPI.publish(req, rsp);
+    assertNotNull(cc.getZkController().zkStateReader.getCollection(collection));
+
+    // listCollectionsForConfig
+    reqParams.clear();
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    rsp = new SolrQueryResponse();
+    schemaDesignerAPI.listCollectionsForConfig(req, rsp);
+    List<String> collections = (List<String>) rsp.getValues().get("collections");
+    assertNotNull(collections);
+    assertTrue(collections.contains(collection));
+
+    // now try to create another temp, which should fail since designer is disabled for this configSet now
+    reqParams.clear();
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    rsp = new SolrQueryResponse();
+    req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    try {
+      schemaDesignerAPI.prepNewSchema(req, rsp);
+      fail("Prep should fail for locked schema " + configSet);
+    } catch (SolrException solrExc) {
+      assertEquals(SolrException.ErrorCode.BAD_REQUEST.code, solrExc.code());
+    }
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testSuggestFilmsXml() throws Exception {
+    String configSet = "films";
+
+    ModifiableSolrParams reqParams = new ModifiableSolrParams();
+
+    File filmsDir = new File(ExternalPaths.SOURCE_HOME, "example/films");
+    assertTrue(filmsDir.getAbsolutePath() + " not found!", filmsDir.isDirectory());
+    File filmsXml = new File(filmsDir, "films.xml");
+    assertTrue("example/films/films.xml not found", filmsXml.isFile());
+
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    reqParams.set(ENABLE_DYNAMIC_FIELDS_PARAM, "true");
+
+    SolrQueryRequest req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+
+    // POST some sample XML docs
+    ContentStreamBase.FileStream stream = new ContentStreamBase.FileStream(filmsXml);
+    stream.setContentType("application/xml");
+    when(req.getContentStreams()).thenReturn(Collections.singletonList(stream));
+
+    SolrQueryResponse rsp = new SolrQueryResponse();
+
+    // POST /schema-designer/analyze
+    schemaDesignerAPI.analyze(req, rsp);
+
+    assertNotNull(rsp.getValues().get(CONFIG_SET_PARAM));
+    assertNotNull(rsp.getValues().get(SCHEMA_VERSION_PARAM));
+    assertNotNull(rsp.getValues().get("fields"));
+    assertNotNull(rsp.getValues().get("fieldTypes"));
+    List<String> docIds = (List<String>) rsp.getValues().get("docIds");
+    assertNotNull(docIds);
+    assertEquals(100, docIds.size()); // designer limits doc ids to top 100
+
+    String idField = rsp.getValues()._getStr(UNIQUE_KEY_FIELD_PARAM, null);
+    assertNotNull(idField);
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testBasicUserWorkflow() throws Exception {
+    String configSet = "testJson";
+
+    ModifiableSolrParams reqParams = new ModifiableSolrParams();
+
+    // Use the prep endpoint to prepare the new schema
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    SolrQueryRequest req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    schemaDesignerAPI.prepNewSchema(req, rsp);
+    assertNotNull(rsp.getValues().get(CONFIG_SET_PARAM));
+    assertNotNull(rsp.getValues().get(SCHEMA_VERSION_PARAM));
+
+    Map<String, Object> expSettings = makeMap(
+        ENABLE_DYNAMIC_FIELDS_PARAM, true,
+        ENABLE_FIELD_GUESSING_PARAM, true,
+        ENABLE_NESTED_DOCS_PARAM, false,
+        LANGUAGES_PARAM, Collections.emptyList(),
+        COPY_FROM_PARAM, "_default");
+    assertDesignerSettings(expSettings, rsp.getValues());
+
+    // Analyze some sample documents to refine the schema
+    reqParams.clear();
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+
+    // POST some sample JSON docs
+    File booksJson = new File(ExternalPaths.SOURCE_HOME, "example/exampledocs/books.json");
+    ContentStreamBase.FileStream stream = new ContentStreamBase.FileStream(booksJson);
+    stream.setContentType(JSON_MIME);
+    when(req.getContentStreams()).thenReturn(Collections.singletonList(stream));
+
+    rsp = new SolrQueryResponse();
+
+    // POST /schema-designer/analyze
+    schemaDesignerAPI.analyze(req, rsp);
+
+    assertNotNull(rsp.getValues().get(CONFIG_SET_PARAM));
+    assertNotNull(rsp.getValues().get(SCHEMA_VERSION_PARAM));
+    assertNotNull(rsp.getValues().get("fields"));
+    assertNotNull(rsp.getValues().get("fieldTypes"));
+    assertNotNull(rsp.getValues().get("docIds"));
+    String idField = rsp.getValues()._getStr(UNIQUE_KEY_FIELD_PARAM, null);
+    assertNotNull(idField);
+    assertDesignerSettings(expSettings, rsp.getValues());
+
+    // capture the schema version for MVCC
+    SolrParams rspData = rsp.getValues().toSolrParams();
+    reqParams.clear();
+    int schemaVersion = rspData.getInt(SCHEMA_VERSION_PARAM);
+
+    // load the contents of a file
+    Collection<String> files = (Collection<String>) rsp.getValues().get("files");
+    assertTrue(files != null && !files.isEmpty());
+
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    String file = null;
+    for (String f : files) {
+      if ("solrconfig.xml".equals(f)) {
+        file = f;
+        break;
+      }
+    }
+    assertNotNull("solrconfig.xml not found in files!", file);
+    reqParams.set("file", file);
+    req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    rsp = new SolrQueryResponse();
+    schemaDesignerAPI.getFileContents(req, rsp);
+    String solrconfigXml = (String) rsp.getValues().get(file);
+    assertNotNull(solrconfigXml);
+    reqParams.clear();
+
+    // Update solrconfig.xml
+    rsp = new SolrQueryResponse();
+    reqParams.set(SCHEMA_VERSION_PARAM, String.valueOf(schemaVersion));
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    reqParams.set("file", file);
+
+    req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    when(req.getContentStreams()).thenReturn(
+        Collections.singletonList(new ContentStreamBase.StringStream(solrconfigXml, "application/xml")));
+
+    schemaDesignerAPI.updateFileContents(req, rsp);
+    rspData = rsp.getValues().toSolrParams();
+    reqParams.clear();
+    schemaVersion = rspData.getInt(SCHEMA_VERSION_PARAM);
+
+    // update solrconfig.xml with some invalid XML mess
+    rsp = new SolrQueryResponse();
+    reqParams.set(SCHEMA_VERSION_PARAM, String.valueOf(schemaVersion));
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    reqParams.set("file", file);
+
+    req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    when(req.getContentStreams()).thenReturn(
+        Collections.singletonList(new ContentStreamBase.StringStream("<config/>", "application/xml")));
+
+    // this should fail b/c the updated solrconfig.xml is invalid
+    schemaDesignerAPI.updateFileContents(req, rsp);
+    rspData = rsp.getValues().toSolrParams();
+    reqParams.clear();
+    assertNotNull(rspData.get("updateFileError"));
+
+    // remove dynamic fields and change the language to "en" only
+    rsp = new SolrQueryResponse();
+    // POST /schema-designer/analyze
+    reqParams.set(SCHEMA_VERSION_PARAM, String.valueOf(schemaVersion));
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    reqParams.set(LANGUAGES_PARAM, "en");
+    reqParams.set(ENABLE_DYNAMIC_FIELDS_PARAM, false);
+    reqParams.set(ENABLE_FIELD_GUESSING_PARAM, false);
+    req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    schemaDesignerAPI.analyze(req, rsp);
+
+    expSettings = makeMap(
+        ENABLE_DYNAMIC_FIELDS_PARAM, false,
+        ENABLE_FIELD_GUESSING_PARAM, false,
+        ENABLE_NESTED_DOCS_PARAM, false,
+        LANGUAGES_PARAM, Collections.singletonList("en"),
+        COPY_FROM_PARAM, "_default");
+    assertDesignerSettings(expSettings, rsp.getValues());
+
+    List<String> filesInResp = (List<String>) rsp.getValues().get("files");
+    assertEquals(5, filesInResp.size());
+    assertTrue(filesInResp.contains("lang/stopwords_en.txt"));
+
+    rspData = rsp.getValues().toSolrParams();
+    schemaVersion = rspData.getInt(SCHEMA_VERSION_PARAM);
+
+    reqParams.clear();
+
+    // add the dynamic fields back and change the languages too
+    rsp = new SolrQueryResponse();
+    reqParams.set(SCHEMA_VERSION_PARAM, String.valueOf(schemaVersion));
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    reqParams.add(LANGUAGES_PARAM, "en");
+    reqParams.add(LANGUAGES_PARAM, "fr");
+    reqParams.set(ENABLE_DYNAMIC_FIELDS_PARAM, true);
+    reqParams.set(ENABLE_FIELD_GUESSING_PARAM, false);
+    req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    schemaDesignerAPI.analyze(req, rsp);
+
+    expSettings = makeMap(
+        ENABLE_DYNAMIC_FIELDS_PARAM, true,
+        ENABLE_FIELD_GUESSING_PARAM, false,
+        ENABLE_NESTED_DOCS_PARAM, false,
+        LANGUAGES_PARAM, Arrays.asList("en", "fr"),
+        COPY_FROM_PARAM, "_default");
+    assertDesignerSettings(expSettings, rsp.getValues());
+
+    filesInResp = (List<String>) rsp.getValues().get("files");
+    assertEquals(7, filesInResp.size());
+    assertTrue(filesInResp.contains("lang/stopwords_fr.txt"));
+
+    rspData = rsp.getValues().toSolrParams();
+    reqParams.clear();
+    schemaVersion = rspData.getInt(SCHEMA_VERSION_PARAM);
+
+    // add back all the default languages
+    rsp = new SolrQueryResponse();
+    reqParams.set(SCHEMA_VERSION_PARAM, String.valueOf(schemaVersion));
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    reqParams.add(LANGUAGES_PARAM, "*");
+    reqParams.set(ENABLE_DYNAMIC_FIELDS_PARAM, false);
+    req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    schemaDesignerAPI.analyze(req, rsp);
+
+    expSettings = makeMap(
+        ENABLE_DYNAMIC_FIELDS_PARAM, false,
+        ENABLE_FIELD_GUESSING_PARAM, false,
+        ENABLE_NESTED_DOCS_PARAM, false,
+        LANGUAGES_PARAM, Collections.emptyList(),
+        COPY_FROM_PARAM, "_default");
+    assertDesignerSettings(expSettings, rsp.getValues());
+
+    filesInResp = (List<String>) rsp.getValues().get("files");
+    assertEquals(43, filesInResp.size());
+    assertTrue(filesInResp.contains("lang/stopwords_fr.txt"));
+    assertTrue(filesInResp.contains("lang/stopwords_en.txt"));
+    assertTrue(filesInResp.contains("lang/stopwords_it.txt"));
+
+    rspData = rsp.getValues().toSolrParams();
+    reqParams.clear();
+    schemaVersion = rspData.getInt(SCHEMA_VERSION_PARAM);
+
+    // Get the value of a sample document
+    String docId = "978-0641723445";
+    String fieldName = "series_t";
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    reqParams.set(DOC_ID_PARAM, docId);
+    reqParams.set(FIELD_PARAM, fieldName);
+    reqParams.set(UNIQUE_KEY_FIELD_PARAM, idField);
+    req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    rsp = new SolrQueryResponse();
+
+    // GET /schema-designer/sample
+    schemaDesignerAPI.getSampleValue(req, rsp);
+    rspData = rsp.getValues().toSolrParams();
+    assertNotNull(rspData.get(idField));
+    assertNotNull(rspData.get(fieldName));
+    assertNotNull(rspData.get("analysis"));
+
+    reqParams.clear();
+
+    // at this point the user would refine the schema by
+    // editing suggestions for fields and adding/removing fields / field types as needed
+
+    // add a new field
+    reqParams.set(SCHEMA_VERSION_PARAM, String.valueOf(schemaVersion));
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+
+    req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    stream = new ContentStreamBase.FileStream(getFile("schema-designer/add-new-field.json"));
+    stream.setContentType(JSON_MIME);
+    when(req.getContentStreams()).thenReturn(Collections.singletonList(stream));
+    rsp = new SolrQueryResponse();
+
+    // POST /schema-designer/add
+    schemaDesignerAPI.addSchemaObject(req, rsp);
+    assertNotNull(rsp.getValues().get("add-field"));
+    rspData = rsp.getValues().toSolrParams();
+    schemaVersion = rspData.getInt(SCHEMA_VERSION_PARAM);
+    assertNotNull(rsp.getValues().get("fields"));
+
+    // update an existing field
+    reqParams.clear();
+    reqParams.set(SCHEMA_VERSION_PARAM, String.valueOf(schemaVersion));
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+
+    req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    // switch a single-valued field to a multi-valued field, which triggers a full rebuild of the "temp" collection
+    stream = new ContentStreamBase.FileStream(getFile("schema-designer/update-author-field.json"));
+    stream.setContentType(JSON_MIME);
+    when(req.getContentStreams()).thenReturn(Collections.singletonList(stream));
+
+    rsp = new SolrQueryResponse();
+
+    // PUT /schema-designer/update
+    schemaDesignerAPI.updateSchemaObject(req, rsp);
+    assertNotNull(rsp.getValues().get("field"));
+    rspData = rsp.getValues().toSolrParams();
+    schemaVersion = rspData.getInt(SCHEMA_VERSION_PARAM);
+
+    // add a new type
+    reqParams.set(SCHEMA_VERSION_PARAM, String.valueOf(schemaVersion));
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+
+    req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    stream = new ContentStreamBase.FileStream(getFile("schema-designer/add-new-type.json"));
+    stream.setContentType(JSON_MIME);
+    when(req.getContentStreams()).thenReturn(Collections.singletonList(stream));
+    rsp = new SolrQueryResponse();
+
+    // POST /schema-designer/add
+    schemaDesignerAPI.addSchemaObject(req, rsp);
+    final String expectedTypeName = "test_txt";
+    assertEquals(expectedTypeName, rsp.getValues().get("add-field-type"));
+    rspData = rsp.getValues().toSolrParams();
+    schemaVersion = rspData.getInt(SCHEMA_VERSION_PARAM);
+    assertNotNull(rsp.getValues().get("fieldTypes"));
+    List<SimpleOrderedMap<Object>> fieldTypes = (List<SimpleOrderedMap<Object>>) rsp.getValues().get("fieldTypes");
+    Optional<SimpleOrderedMap<Object>> expected = fieldTypes.stream().filter(m -> expectedTypeName.equals(m.get("name"))).findFirst();
+    assertTrue("New field type '" + expectedTypeName + "' not found in add type response!", expected.isPresent());
+
+    reqParams.clear();
+    reqParams.set(SCHEMA_VERSION_PARAM, String.valueOf(schemaVersion));
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+
+    req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    stream = new ContentStreamBase.FileStream(getFile("schema-designer/update-type.json"));
+    stream.setContentType(JSON_MIME);
+    when(req.getContentStreams()).thenReturn(Collections.singletonList(stream));
+    rsp = new SolrQueryResponse();
+
+    // POST /schema-designer/update
+    schemaDesignerAPI.updateSchemaObject(req, rsp);
+    rspData = rsp.getValues().toSolrParams();
+    schemaVersion = rspData.getInt(SCHEMA_VERSION_PARAM);
+
+    // query to see how the schema decisions impact retrieval / ranking
+    reqParams.clear();
+    reqParams.set(SCHEMA_VERSION_PARAM, String.valueOf(schemaVersion));
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    reqParams.set(CommonParams.Q, "*:*");
+    req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    rsp = new SolrQueryResponse();
+
+    // GET /schema-designer/query
+    schemaDesignerAPI.query(req, rsp);
+    assertNotNull(rsp.getResponseHeader());
+    SolrDocumentList results = (SolrDocumentList) rsp.getResponse();
+    assertEquals(4, results.size());
+
+    // Download ZIP
+    reqParams.clear();
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    rsp = new SolrQueryResponse();
+    schemaDesignerAPI.downloadConfig(req, rsp);
+    assertNotNull(rsp.getValues().get(CONTENT));
+
+    // publish schema to a config set that can be used by real collections
+    reqParams.clear();
+    reqParams.set(SCHEMA_VERSION_PARAM, String.valueOf(schemaVersion));
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+
+    String collection = "test123";
+    reqParams.set(NEW_COLLECTION_PARAM, collection);
+    reqParams.set(INDEX_TO_COLLECTION_PARAM, true);
+    reqParams.set(RELOAD_COLLECTIONS_PARAM, true);
+    reqParams.set(CLEANUP_TEMP_PARAM, true);
+
+    rsp = new SolrQueryResponse();
+    schemaDesignerAPI.publish(req, rsp);
+
+    assertNotNull(cc.getZkController().zkStateReader.getCollection(collection));
+
+    // listCollectionsForConfig
+    reqParams.clear();
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    rsp = new SolrQueryResponse();
+    schemaDesignerAPI.listCollectionsForConfig(req, rsp);
+    List<String> collections = (List<String>) rsp.getValues().get("collections");
+    assertNotNull(collections);
+    assertTrue(collections.contains(collection));
+
+    // verify temp designer objects were cleaned up during the publish operation ...
+    String mutableId = getMutableId(configSet);
+    assertFalse(cc.getZkController().getClusterState().hasCollection(mutableId));
+    SolrZkClient zkClient = cc.getZkController().getZkClient();
+    assertFalse(zkClient.exists("/configs/" + mutableId, true));
+
+    SolrQuery query = new SolrQuery("*:*");
+    query.setRows(0);
+    QueryResponse qr = cluster.getSolrClient().query(collection, query);
+    // this proves the docs were stored in the blob store too
+    assertEquals(4, qr.getResults().getNumFound());
+  }
+
+  @SuppressWarnings("unchecked")
+  public void testFieldUpdates() throws Exception {
+    String configSet = "fieldUpdates";
+
+    ModifiableSolrParams reqParams = new ModifiableSolrParams();
+
+    // Use the prep endpoint to prepare the new schema
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    SolrQueryRequest req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    schemaDesignerAPI.prepNewSchema(req, rsp);
+    assertNotNull(rsp.getValues().get(CONFIG_SET_PARAM));
+    assertNotNull(rsp.getValues().get(SCHEMA_VERSION_PARAM));
+    SolrParams rspData = rsp.getValues().toSolrParams();
+    int schemaVersion = rspData.getInt(SCHEMA_VERSION_PARAM);
+
+    // add our test field that we'll test various updates to
+    reqParams.clear();
+    reqParams.set(SCHEMA_VERSION_PARAM, String.valueOf(schemaVersion));
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    ContentStreamBase.FileStream stream = new ContentStreamBase.FileStream(getFile("schema-designer/add-new-field.json"));
+    stream.setContentType(JSON_MIME);
+    when(req.getContentStreams()).thenReturn(Collections.singletonList(stream));
+    rsp = new SolrQueryResponse();
+
+    // POST /schema-designer/add
+    schemaDesignerAPI.addSchemaObject(req, rsp);
+    assertNotNull(rsp.getValues().get("add-field"));
+
+    final String fieldName = "keywords";
+
+    Optional<SimpleOrderedMap<Object>> maybeField =
+        ((List<SimpleOrderedMap<Object>>) rsp.getValues().get("fields")).stream().filter(m -> fieldName.equals(m.get("name"))).findFirst();
+    assertTrue(maybeField.isPresent());
+    SimpleOrderedMap<Object> field = maybeField.get();
+    assertEquals(Boolean.FALSE, field.get("indexed"));
+    assertEquals(Boolean.FALSE, field.get("required"));
+    assertEquals(Boolean.TRUE, field.get("stored"));
+    assertEquals(Boolean.TRUE, field.get("docValues"));
+    assertEquals(Boolean.TRUE, field.get("useDocValuesAsStored"));
+    assertEquals(Boolean.FALSE, field.get("multiValued"));
+    assertEquals("string", field.get("type"));
+
+    String mutableId = getMutableId(configSet);
+    SchemaDesignerConfigSetHelper configSetHelper =
+        new SchemaDesignerConfigSetHelper(cc, SchemaDesignerAPI.newSchemaSuggester(cc));
+    ManagedIndexSchema schema = schemaDesignerAPI.loadLatestSchema(mutableId);
+
+    // make it required
+    Map<String, Object> updateField = makeMap("name", fieldName, "type", field.get("type"), "required", true);
+    configSetHelper.updateField(configSet, updateField, schema);
+
+    schema = schemaDesignerAPI.loadLatestSchema(mutableId);
+    SchemaField schemaField = schema.getField(fieldName);
+    assertTrue(schemaField.isRequired());
+
+    updateField = makeMap("name", fieldName, "type", field.get("type"), "required", false, "stored", false);
+    configSetHelper.updateField(configSet, updateField, schema);
+    schema = schemaDesignerAPI.loadLatestSchema(mutableId);
+    schemaField = schema.getField(fieldName);
+    assertFalse(schemaField.isRequired());
+    assertFalse(schemaField.stored());
+
+    updateField = makeMap("name", fieldName, "type", field.get("type"), "required", false, "stored", false, "multiValued", true);
+    configSetHelper.updateField(configSet, updateField, schema);
+    schema = schemaDesignerAPI.loadLatestSchema(mutableId);
+    schemaField = schema.getField(fieldName);
+    assertFalse(schemaField.isRequired());
+    assertFalse(schemaField.stored());
+    assertTrue(schemaField.multiValued());
+
+    updateField = makeMap("name", fieldName, "type", "strings", "copyDest", "_text_");
+    configSetHelper.updateField(configSet, updateField, schema);
+    schema = schemaDesignerAPI.loadLatestSchema(mutableId);
+    schemaField = schema.getField(fieldName);
+    assertTrue(schemaField.multiValued());
+    assertEquals("strings", schemaField.getType().getTypeName());
+    assertFalse(schemaField.isRequired());
+    assertTrue(schemaField.stored());
+    List<String> srcFields = schema.getCopySources("_text_");
+    assertEquals(Collections.singletonList(fieldName), srcFields);
+  }
+
+  @SuppressWarnings({"unchecked"})
+  public void testSchemaDiffEndpoint() throws Exception {
+    String configSet = "testDiff";
+
+    ModifiableSolrParams reqParams = new ModifiableSolrParams();
+
+    // Use the prep endpoint to prepare the new schema
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    SolrQueryRequest req = mock(SolrQueryRequest.class);
+    when(req.getParams()).thenReturn(reqParams);
+    schemaDesignerAPI.prepNewSchema(req, rsp);
+    assertNotNull(rsp.getValues().get(CONFIG_SET_PARAM));
+    assertNotNull(rsp.getValues().get(SCHEMA_VERSION_PARAM));
+
+    // publish schema to a config set that can be used by real collections
+    reqParams.clear();
+    reqParams.set(SCHEMA_VERSION_PARAM, String.valueOf(rsp.getValues().get(SCHEMA_VERSION_PARAM)));
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+
+    String collection = "diff456";
+    reqParams.set(NEW_COLLECTION_PARAM, collection);
+    reqParams.set(INDEX_TO_COLLECTION_PARAM, true);
+    reqParams.set(RELOAD_COLLECTIONS_PARAM, true);
+    reqParams.set(CLEANUP_TEMP_PARAM, true);
+
+    rsp = new SolrQueryResponse();
+    schemaDesignerAPI.publish(req, rsp);
+
+    assertNotNull(cc.getZkController().zkStateReader.getCollection(collection));
+
+    // Load the schema designer for the existing config set and make some changes to it
+    reqParams.clear();
+    reqParams.set(CONFIG_SET_PARAM, configSet);
+    reqParams.set(ENABLE_DYNAMIC_FIELDS_PARAM, "true");
+    reqParams.set(ENABLE_FIELD_GUESSING_PARAM, "false");
+    rsp = new SolrQueryResponse();
+    schemaDesignerAPI.analyze(req, rsp);
+
+    // Update id field to not use docValues
+    List<SimpleOrderedMap<Object>> fields = (List<SimpleOrderedMap<Object>>) rsp.getValues().get("fields");
+    SimpleOrderedMap<Object> idFieldMap = fields.stream().filter(field -> field.get("name").equals("id")).findFirst().get();
+    idFieldMap.remove("copyDest"); // Don't include copyDest as it is not a property of SchemaField
+    SimpleOrderedMap<Object> idFieldMapUpdated = idFieldMap.clone();
+    idFieldMapUpdated.setVal(idFieldMapUpdated.indexOf("docValues", 0), Boolean.FALSE);
+    idFieldMapUpdated.setVal(idFieldMapUpdated.indexOf("useDocValuesAsStored", 0), Boolean.FALSE);
+    idFieldMapUpdated.setVal(idFieldMapUpdated.indexOf("omitTermFreqAndPositions", 0), Boolean.FALSE);
+
+    SolrParams solrParams = idFieldMapUpdated.toSolrParams();
+    Map<String, Object> mapParams = solrParams.toMap(new HashMap<>());
+    mapParams.put("termVectors", Boolean.FALSE);
+    reqParams.set(SCHEMA_VERSION_PARAM, rsp.getValues().toSolrParams().getInt(SCHEMA_VERSION_PARAM));
+
+    ContentStreamBase.StringStream stringStream = new ContentStreamBase.StringStream(JSONUtil.toJSON(mapParams), JSON_MIME);
+    when(req.getContentStreams()).thenReturn(Collections.singletonList(stringStream));
+
+    rsp = new SolrQueryResponse();
+    schemaDesignerAPI.updateSchemaObject(req, rsp);
+
+    // Add a new field
+    Integer schemaVersion = rsp.getValues().toSolrParams().getInt(SCHEMA_VERSION_PARAM);
+    reqParams.set(SCHEMA_VERSION_PARAM, schemaVersion);
+    ContentStreamBase.FileStream fileStream = new ContentStreamBase.FileStream(getFile("schema-designer/add-new-field.json"));
+    fileStream.setContentType(JSON_MIME);
+    when(req.getContentStreams()).thenReturn(Collections.singletonList(fileStream));
+    rsp = new SolrQueryResponse();
+    // POST /schema-designer/add
+    schemaDesignerAPI.addSchemaObject(req, rsp);
+    assertNotNull(rsp.getValues().get("add-field"));
+
+    // Add a new field type
+    schemaVersion = rsp.getValues().toSolrParams().getInt(SCHEMA_VERSION_PARAM);
+    reqParams.set(SCHEMA_VERSION_PARAM, schemaVersion);
+    fileStream = new ContentStreamBase.FileStream(getFile("schema-designer/add-new-type.json"));
+    fileStream.setContentType(JSON_MIME);
+    when(req.getContentStreams()).thenReturn(Collections.singletonList(fileStream));
+    rsp = new SolrQueryResponse();
+    // POST /schema-designer/add
+    schemaDesignerAPI.addSchemaObject(req, rsp);
+    assertNotNull(rsp.getValues().get("add-field-type"));
+
+    // Let's do a diff now
+    rsp = new SolrQueryResponse();
+    schemaDesignerAPI.getSchemaDiff(req, rsp);
+
+    Map<String, Object> diff = (Map<String, Object>) rsp.getValues().get("diff");
+
+    // field asserts
+    assertNotNull(diff.get("fields"));
+    Map<String, Object> fieldsDiff = (Map<String, Object>) diff.get("fields");
+    assertNotNull(fieldsDiff.get("updated"));
+    Map<String, Object> mapDiff = (Map<String, Object>) fieldsDiff.get("updated");
+    assertEquals(
+        Arrays.asList(
+            ImmutableMap.of("omitTermFreqAndPositions", true, "useDocValuesAsStored", true, "docValues", true),
+            ImmutableMap.of("omitTermFreqAndPositions", false, "useDocValuesAsStored", false, "docValues", false)),
+        mapDiff.get("id"));
+    assertNotNull(fieldsDiff.get("added"));
+    Map<String, Object> fieldsAdded = (Map<String, Object>) fieldsDiff.get("added");
+    assertNotNull(fieldsAdded.get("keywords"));
+
+    // field type asserts
+    assertNotNull(diff.get("fieldTypes"));
+    Map<String, Object> fieldTypesDiff = (Map<String, Object>) diff.get("fieldTypes");
+    assertNotNull(fieldTypesDiff.get("added"));
+    Map<String, Object> fieldTypesAdded = (Map<String, Object>) fieldTypesDiff.get("added");
+    assertNotNull(fieldTypesAdded.get("test_txt"));
+  }
+
+  @SuppressWarnings("rawtypes")
+  protected void assertDesignerSettings(Map<String, Object> expected, NamedList actual) {
+    for (String expKey : expected.keySet()) {
+      Object expValue = expected.get(expKey);
+      assertEquals("Value for designer setting '" + expKey + "' not match expected!", expValue, actual.get(expKey));
+    }
+  }
+}
diff --git a/solr/core/src/test/org/apache/solr/handler/designer/TestSchemaDesignerConfigSetHelper.java b/solr/core/src/test/org/apache/solr/handler/designer/TestSchemaDesignerConfigSetHelper.java
new file mode 100644
index 0000000..ba1a081
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/designer/TestSchemaDesignerConfigSetHelper.java
@@ -0,0 +1,360 @@
+/*
+ * 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.handler.designer;
+
+import java.io.File;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.core.SolrConfig;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.schema.ManagedIndexSchema;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.util.ExternalPaths;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.makeMap;
+import static org.apache.solr.common.util.Utils.toJavabin;
+import static org.apache.solr.handler.admin.ConfigSetsHandler.DEFAULT_CONFIGSET_NAME;
+import static org.apache.solr.handler.designer.SchemaDesignerAPI.getMutableId;
+import static org.apache.solr.schema.IndexSchema.NEST_PATH_FIELD_NAME;
+import static org.apache.solr.schema.IndexSchema.ROOT_FIELD_NAME;
+
+public class TestSchemaDesignerConfigSetHelper extends SolrCloudTestCase implements SchemaDesignerConstants {
+
+  private CoreContainer cc;
+  private SchemaDesignerConfigSetHelper helper;
+
+  @BeforeClass
+  public static void createCluster() throws Exception {
+    System.setProperty("managed.schema.mutable", "true");
+    configureCluster(1).addConfig(DEFAULT_CONFIGSET_NAME, new File(ExternalPaths.DEFAULT_CONFIGSET).toPath()).configure();
+    // SchemaDesignerConfigSetHelper depends on the blob store
+    CollectionAdminRequest.createCollection(BLOB_STORE_ID, 1, 1).process(cluster.getSolrClient());
+    cluster.waitForActiveCollection(BLOB_STORE_ID, 1, 1);
+  }
+
+  @AfterClass
+  public static void tearDownCluster() throws Exception {
+    if (cluster != null && cluster.getSolrClient() != null) {
+      cluster.deleteAllCollections();
+      cluster.deleteAllConfigSets();
+    }
+  }
+
+  @Before
+  public void setupTest() {
+    assumeWorkingMockito();
+    assertNotNull(cluster);
+    cc = cluster.getJettySolrRunner(0).getCoreContainer();
+    assertNotNull(cc);
+    helper = new SchemaDesignerConfigSetHelper(cc, SchemaDesignerAPI.newSchemaSuggester(cc));
+  }
+
+  @Test
+  public void testSetupMutable() throws Exception {
+    String configSet = "testSetupMutable";
+    String mutableId = getMutableId(configSet);
+    // create our test config by copying from _default
+    helper.copyConfig(DEFAULT_CONFIGSET_NAME, mutableId);
+
+    SolrConfig solrConfig = helper.loadSolrConfig(mutableId);
+    assertNotNull(solrConfig);
+    ManagedIndexSchema schema = helper.loadLatestSchema(solrConfig);
+    assertNotNull(schema);
+    int currentVersion = helper.getCurrentSchemaVersion(mutableId);
+    assertEquals(schema.getSchemaZkVersion(), currentVersion);
+
+    assertTrue(schema.persistManagedSchema(false));
+
+    // create the temp collection
+    helper.createCollection(mutableId, mutableId);
+
+    List<String> collsForConfig = helper.listCollectionsForConfig(configSet);
+    assertTrue(collsForConfig.isEmpty());
+
+    helper.reloadTempCollection(mutableId, true);
+
+    // version is incremented b/c we called persist on the schema above
+    int version = helper.getCurrentSchemaVersion(mutableId);
+    assertEquals(1, version);
+
+    helper.checkSchemaVersion(mutableId, version, -1);
+
+    schema = helper.syncLanguageSpecificObjectsAndFiles(configSet, schema, Collections.emptyList(), true, DEFAULT_CONFIGSET_NAME);
+    assertEquals(2, schema.getSchemaZkVersion());
+
+    byte[] zipped = helper.downloadAndZipConfigSet(mutableId);
+    assertTrue(zipped != null && zipped.length > 0);
+  }
+
+  @Test
+  public void testEnableDisableOptions() throws Exception {
+    String configSet = "testEnableDisableOptions";
+    String mutableId = getMutableId(configSet);
+    helper.copyConfig(DEFAULT_CONFIGSET_NAME, mutableId);
+
+    ManagedIndexSchema schema = helper.loadLatestSchema(helper.loadSolrConfig(mutableId));
+    assertEquals(schema.getSchemaZkVersion(), helper.getCurrentSchemaVersion(mutableId));
+
+    schema = helper.syncLanguageSpecificObjectsAndFiles(configSet, schema, Collections.singletonList("en"), true, DEFAULT_CONFIGSET_NAME);
+    assertNotNull(schema.getFieldTypeByName("text_en"));
+    assertNotNull(schema.getFieldOrNull("*_txt_en"));
+    assertNull(schema.getFieldTypeByName("text_fr"));
+
+    schema = helper.syncLanguageSpecificObjectsAndFiles(configSet, schema, Collections.singletonList("en"), false, DEFAULT_CONFIGSET_NAME);
+    assertNotNull(schema.getFieldTypeByName("text_en"));
+    assertNull(schema.getFieldOrNull("*_txt_en"));
+    assertNull(schema.getFieldTypeByName("text_fr"));
+
+    schema = helper.syncLanguageSpecificObjectsAndFiles(configSet, schema, Arrays.asList("en", "fr"), false, DEFAULT_CONFIGSET_NAME);
+    assertNotNull(schema.getFieldTypeByName("text_en"));
+    assertNull(schema.getFieldOrNull("*_txt_en"));
+    assertNotNull(schema.getFieldTypeByName("text_fr"));
+
+    schema = helper.syncLanguageSpecificObjectsAndFiles(configSet, schema, Arrays.asList("en", "fr"), true, DEFAULT_CONFIGSET_NAME);
+    assertNotNull(schema.getFieldTypeByName("text_en"));
+    assertNotNull(schema.getFieldOrNull("*_txt_en"));
+    assertTrue(cluster.getZkClient().exists(SchemaDesignerAPI.getConfigSetZkPath(mutableId, "lang/stopwords_en.txt"), true));
+    assertNotNull(schema.getFieldTypeByName("text_fr"));
+    assertNotNull(schema.getFieldOrNull("*_txt_fr"));
+    assertNull(schema.getFieldOrNull("*_txt_ga"));
+
+    // add a field that uses text_en and then try removing "en" from the lang set
+    helper.createCollection(mutableId, mutableId); // need to create field
+    Map<String, Object> addField = makeMap("name", "title", "type", "text_en");
+    String addedFieldName = helper.addSchemaObject(configSet, Collections.singletonMap("add-field", addField));
+    assertEquals("title", addedFieldName);
+
+    schema = helper.loadLatestSchema(helper.loadSolrConfig(mutableId));
+    assertNotNull(schema.getField("title"));
+
+    schema = helper.syncLanguageSpecificObjectsAndFiles(configSet, schema, Collections.singletonList("fr"), true, DEFAULT_CONFIGSET_NAME);
+    assertNotNull(schema.getFieldTypeByName("text_en")); // being used, so not removed
+    assertNotNull(schema.getFieldOrNull("*_txt_en"));
+    assertTrue(cluster.getZkClient().exists(SchemaDesignerAPI.getConfigSetZkPath(mutableId, "lang/stopwords_en.txt"), true));
+    assertNotNull(schema.getFieldTypeByName("text_fr"));
+    assertNotNull(schema.getFieldOrNull("*_txt_fr"));
+    assertNull(schema.getFieldOrNull("*_txt_ga"));
+
+    schema = helper.syncLanguageSpecificObjectsAndFiles(configSet, schema, Collections.emptyList(), true, DEFAULT_CONFIGSET_NAME);
+    assertNotNull(schema.getFieldTypeByName("text_en"));
+    assertNotNull(schema.getFieldOrNull("*_txt_en"));
+    assertNotNull(schema.getFieldTypeByName("text_fr"));
+    assertNotNull(schema.getFieldOrNull("*_txt_fr"));
+    assertNotNull(schema.getFieldTypeByName("text_ga"));
+    assertNotNull(schema.getFieldOrNull("*_txt_ga"));
+
+    schema = helper.syncLanguageSpecificObjectsAndFiles(configSet, schema, Collections.emptyList(), false, DEFAULT_CONFIGSET_NAME);
+    assertNotNull(schema.getFieldTypeByName("text_en"));
+    assertNull(schema.getFieldOrNull("*_txt_en"));
+    assertNotNull(schema.getFieldTypeByName("text_fr"));
+    assertNull(schema.getFieldOrNull("*_txt_fr"));
+    assertNotNull(schema.getFieldTypeByName("text_ga"));
+    assertNull(schema.getFieldOrNull("*_txt_ga"));
+
+    schema = helper.toggleNestedDocsFields(schema, true);
+    assertTrue(schema.hasExplicitField(ROOT_FIELD_NAME));
+    assertTrue(schema.hasExplicitField(NEST_PATH_FIELD_NAME));
+    schema = helper.toggleNestedDocsFields(schema, false);
+    assertFalse(schema.hasExplicitField(ROOT_FIELD_NAME));
+    assertFalse(schema.hasExplicitField(NEST_PATH_FIELD_NAME));
+    schema = helper.toggleNestedDocsFields(schema, true);
+    assertTrue(schema.hasExplicitField(ROOT_FIELD_NAME));
+    assertTrue(schema.hasExplicitField(NEST_PATH_FIELD_NAME));
+  }
+
+  @Test
+  public void testPersistSampleDocs() throws Exception {
+    String configSet = "testPersistSampleDocs";
+
+    SolrInputDocument doc = new SolrInputDocument();
+    doc.setField("id", "1");
+    doc.setField("author", "Ken Follet");
+    doc.setField("title", "The Pillars of the Earth");
+    doc.setField("series", "Kingsbridge Series");
+    doc.setField("pages", 809);
+    doc.setField("published_year", 1989);
+
+    helper.postDataToBlobStore(cluster.getSolrClient(), configSet + "_sample",
+        DefaultSampleDocumentsLoader.streamAsBytes(toJavabin(Collections.singletonList(doc))));
+
+    List<SolrInputDocument> docs = helper.getStoredSampleDocs(configSet);
+    assertTrue(docs != null && docs.size() == 1);
+    assertEquals("1", docs.get(0).getFieldValue("id"));
+  }
+
+  @Test
+  @SuppressWarnings("unchecked")
+  public void testAnalyzeField() throws Exception {
+    String configSet = "testAnalyzeField";
+
+    String mutableId = getMutableId(configSet);
+    helper.copyConfig(DEFAULT_CONFIGSET_NAME, mutableId);
+
+    ManagedIndexSchema schema = helper.loadLatestSchema(helper.loadSolrConfig(mutableId));
+    assertEquals(schema.getSchemaZkVersion(), helper.getCurrentSchemaVersion(mutableId));
+    helper.createCollection(mutableId, mutableId);
+
+    Map<String, Object> addField = makeMap("name", "title", "type", "text_en");
+    String addedFieldName = helper.addSchemaObject(configSet, Collections.singletonMap("add-field", addField));
+    assertEquals("title", addedFieldName);
+
+    Map<String, Object> analysis = helper.analyzeField(configSet, "title", "The Pillars of the Earth");
+
+    Map<String, Object> title = (Map<String, Object>) ((Map<String, Object>) analysis.get("field_names")).get("title");
+    assertNotNull(title);
+    List<Object> index = (List<Object>) title.get("index");
+    assertNotNull(index);
+    assertFalse(index.isEmpty());
+  }
+
+  @Test
+  public void testCopyFieldUpdates() throws Exception {
+    String configSet = "testCopyFieldUpdates";
+    String mutableId = getMutableId(configSet);
+    helper.copyConfig(DEFAULT_CONFIGSET_NAME, mutableId);
+
+    ManagedIndexSchema schema = helper.loadLatestSchema(helper.loadSolrConfig(mutableId));
+    assertEquals(schema.getSchemaZkVersion(), helper.getCurrentSchemaVersion(mutableId));
+    helper.createCollection(mutableId, mutableId);
+
+    // add / update field
+    Map<String, Object> addField = makeMap("name", "author", "type", "string");
+    String addedFieldName = helper.addSchemaObject(configSet, Collections.singletonMap("add-field", addField));
+    assertEquals("author", addedFieldName);
+
+    helper.addSchemaObject(configSet,
+        Collections.singletonMap("add-field", makeMap("name", "_catch_all_", "type", "text_general")));
+
+    Map<String, Object> updateField = makeMap("name", "author", "type", "string", "copyDest", "_text_");
+    ManagedIndexSchema latest = helper.loadLatestSchema(helper.loadSolrConfig(mutableId));
+    latest.getField("_catch_all_");
+
+    Map<String, Object> resp = helper.updateSchemaObject(configSet, updateField, latest);
+    assertNotNull(resp);
+    assertEquals("field", resp.get("updateType"));
+    assertEquals(false, resp.get("rebuild"));
+
+    updateField = makeMap("name", "author", "type", "string", "copyDest", "_text_,_catch_all_");
+    latest = helper.loadLatestSchema(helper.loadSolrConfig(mutableId));
+    resp = helper.updateSchemaObject(configSet, updateField, latest);
+    assertNotNull(resp);
+    assertEquals("field", resp.get("updateType"));
+    assertEquals(false, resp.get("rebuild"));
+
+    latest = helper.loadLatestSchema(helper.loadSolrConfig(mutableId));
+    assertEquals(Collections.singletonList("author"), latest.getCopySources("_text_"));
+    assertEquals(Collections.singletonList("author"), latest.getCopySources("_catch_all_"));
+
+    updateField = makeMap("name", "author", "type", "string", "copyDest", "");
+    latest = helper.loadLatestSchema(helper.loadSolrConfig(mutableId));
+    resp = helper.updateSchemaObject(configSet, updateField, latest);
+    assertNotNull(resp);
+    assertEquals("field", resp.get("updateType"));
+    assertEquals(false, resp.get("rebuild"));
+    latest = helper.loadLatestSchema(helper.loadSolrConfig(mutableId));
+    assertTrue(latest.getCopySources("_text_").isEmpty());
+    assertTrue(latest.getCopySources("_catch_all_").isEmpty());
+  }
+
+  @Test
+  public void testAddUpdateObjects() throws Exception {
+    String configSet = "testAddUpdateObjects";
+    String mutableId = getMutableId(configSet);
+    helper.copyConfig(DEFAULT_CONFIGSET_NAME, mutableId);
+
+    ManagedIndexSchema schema = helper.loadLatestSchema(helper.loadSolrConfig(mutableId));
+    assertEquals(schema.getSchemaZkVersion(), helper.getCurrentSchemaVersion(mutableId));
+    helper.createCollection(mutableId, mutableId);
+
+    // add / update field
+    Map<String, Object> addField = makeMap("name", "author", "type", "string");
+    String addedFieldName = helper.addSchemaObject(configSet, Collections.singletonMap("add-field", addField));
+    assertEquals("author", addedFieldName);
+
+    Map<String, Object> updateField = makeMap("name", "author", "type", "string", "required", true);
+    ManagedIndexSchema latest = helper.loadLatestSchema(helper.loadSolrConfig(mutableId));
+    Map<String, Object> resp = helper.updateSchemaObject(configSet, updateField, latest);
+    assertNotNull(resp);
+    assertEquals("field", resp.get("updateType"));
+    assertEquals(false, resp.get("rebuild"));
+
+    SchemaField addedField = latest.getField("author");
+    assertFalse(addedField.multiValued());
+    assertTrue(addedField.hasDocValues());
+
+    // an update that requires a full-rebuild
+    updateField = makeMap("name", "author", "type", "string", "required", true, "docValues", true, "multiValued", true, "copyDest", "_text_");
+    resp = helper.updateSchemaObject(configSet, updateField, helper.loadLatestSchema(helper.loadSolrConfig(mutableId)));
+    assertNotNull(resp);
+    assertEquals("field", resp.get("updateType"));
+    assertEquals(true, resp.get("rebuild"));
+
+    // did the copy field update get applied?
+    latest = helper.loadLatestSchema(helper.loadSolrConfig(mutableId));
+    assertEquals(Collections.singletonList("author"), latest.getCopySources("_text_"));
+
+    // switch the author field type to strings
+    updateField = makeMap("name", "author", "type", "strings", "docValues", true, "copyDest", "_text_");
+    resp = helper.updateSchemaObject(configSet, updateField, helper.loadLatestSchema(helper.loadSolrConfig(mutableId)));
+    assertNotNull(resp);
+    assertEquals("field", resp.get("updateType"));
+    assertEquals(false, resp.get("rebuild")); // tricky, we didn't actually change the field to multiValue (it already was)
+
+    // add / update field type
+    Map<String, Object> addType = makeMap("name", "testType", "class", "solr.StrField", "docValues", true);
+    String addTypeName = helper.addSchemaObject(configSet, Collections.singletonMap("add-field-type", addType));
+    assertEquals("testType", addTypeName);
+
+    latest = helper.loadLatestSchema(helper.loadSolrConfig(mutableId));
+    FieldType addedType = latest.getFieldTypeByName(addTypeName);
+    assertNotNull(addedType);
+    SimpleOrderedMap<Object> props = addedType.getNamedPropertyValues(false);
+    assertTrue(props.getBooleanArg("docValues"));
+    assertFalse(addedType.isMultiValued());
+
+    Map<String, Object> updateType = makeMap("name", "testType", "class", "solr.StrField", "docValues", true, "multiValued", true);
+    resp = helper.updateSchemaObject(configSet, updateType, helper.loadLatestSchema(helper.loadSolrConfig(mutableId)));
+    assertNotNull(resp);
+    assertEquals("type", resp.get("updateType"));
+    assertEquals(true, resp.get("rebuild"));
+
+    // add / update dynamic field
+    Map<String, Object> addDynField = makeMap("name", "*_test", "type", "string");
+    String addedDynFieldName = helper.addSchemaObject(configSet, Collections.singletonMap("add-dynamic-field", addDynField));
+    assertEquals("*_test", addedDynFieldName);
+
+    // update the dynamic field
+    Map<String, Object> updateDynField = makeMap("name", "*_test", "type", "string", "docValues", false);
+    resp = helper.updateSchemaObject(configSet, updateDynField, helper.loadLatestSchema(helper.loadSolrConfig(mutableId)));
+    assertEquals("*_test", addedDynFieldName);
+    assertNotNull(resp);
+    assertEquals("dynamicField", resp.get("updateType"));
+  }
+}
diff --git a/solr/core/src/test/org/apache/solr/handler/designer/TestSchemaDesignerSettingsDAO.java b/solr/core/src/test/org/apache/solr/handler/designer/TestSchemaDesignerSettingsDAO.java
new file mode 100644
index 0000000..2f3a037
--- /dev/null
+++ b/solr/core/src/test/org/apache/solr/handler/designer/TestSchemaDesignerSettingsDAO.java
@@ -0,0 +1,126 @@
+/*
+ * 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.handler.designer;
+
+import java.io.File;
+import java.util.Collections;
+import java.util.Map;
+
+import org.apache.solr.client.solrj.SolrResponse;
+import org.apache.solr.client.solrj.request.CollectionAdminRequest;
+import org.apache.solr.cloud.SolrCloudTestCase;
+import org.apache.solr.core.CoreContainer;
+import org.apache.solr.handler.admin.CollectionsHandler;
+import org.apache.solr.util.ExternalPaths;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+import static org.apache.solr.common.util.Utils.makeMap;
+import static org.apache.solr.handler.admin.ConfigSetsHandler.DEFAULT_CONFIGSET_NAME;
+
+public class TestSchemaDesignerSettingsDAO extends SolrCloudTestCase implements SchemaDesignerConstants {
+
+  private CoreContainer cc;
+
+  @BeforeClass
+  public static void createCluster() throws Exception {
+    System.setProperty("managed.schema.mutable", "true");
+    configureCluster(1).addConfig(DEFAULT_CONFIGSET_NAME, new File(ExternalPaths.DEFAULT_CONFIGSET).toPath()).configure();
+  }
+
+  @AfterClass
+  public static void tearDownCluster() throws Exception {
+    if (cluster != null && cluster.getSolrClient() != null) {
+      cluster.deleteAllCollections();
+      cluster.deleteAllConfigSets();
+    }
+  }
+
+  @Before
+  public void setupTest() {
+    assertNotNull(cluster);
+    cc = cluster.getJettySolrRunner(0).getCoreContainer();
+    assertNotNull(cc);
+  }
+
+  @Test
+  public void testDAO() throws Exception {
+    String collection = "testDAO";
+    String configSet = DEFAULT_CONFIGSET_NAME;
+
+    SolrResponse rsp =
+        CollectionAdminRequest.createCollection(collection, configSet, 1, 1).process(cluster.getSolrClient());
+    CollectionsHandler.waitForActiveCollection(collection, cc, rsp);
+
+    SchemaDesignerSettingsDAO dao = new SchemaDesignerSettingsDAO(cc);
+    SchemaDesignerSettings settings = dao.getSettings(configSet);
+    assertNotNull(settings);
+
+    Map<String, Object> expSettings = makeMap(
+        DESIGNER_KEY + ENABLE_DYNAMIC_FIELDS_PARAM, true,
+        AUTO_CREATE_FIELDS, true,
+        DESIGNER_KEY + ENABLE_NESTED_DOCS_PARAM, false,
+        DESIGNER_KEY + LANGUAGES_PARAM, Collections.emptyList());
+
+    assertDesignerSettings(expSettings, settings);
+    settings.setDisabled(false);
+    settings.setCopyFrom("foo");
+
+    assertTrue("updated settings should have changed in ZK", dao.persistIfChanged(configSet, settings));
+
+    settings = dao.getSettings(configSet);
+    assertNotNull(settings);
+
+    expSettings = makeMap(
+        DESIGNER_KEY + DISABLED, false,
+        DESIGNER_KEY + COPY_FROM_PARAM, "foo",
+        DESIGNER_KEY + ENABLE_DYNAMIC_FIELDS_PARAM, true,
+        AUTO_CREATE_FIELDS, true,
+        DESIGNER_KEY + ENABLE_NESTED_DOCS_PARAM, false,
+        DESIGNER_KEY + LANGUAGES_PARAM, Collections.emptyList());
+    assertDesignerSettings(expSettings, settings);
+    assertFalse("should not be disabled", dao.isDesignerDisabled(configSet));
+
+    settings.setDisabled(true);
+    settings.setCopyFrom("bar");
+    settings.setDynamicFieldsEnabled(false);
+    settings.setNestedDocsEnabled(true);
+    settings.setFieldGuessingEnabled(false);
+    settings.setLanguages(Collections.singletonList("en"));
+
+    assertTrue("updated settings should have changed in ZK", dao.persistIfChanged(configSet, settings));
+    settings = dao.getSettings(configSet);
+    assertNotNull(settings);
+
+    expSettings = makeMap(
+        DESIGNER_KEY + DISABLED, true,
+        DESIGNER_KEY + COPY_FROM_PARAM, "bar",
+        DESIGNER_KEY + ENABLE_DYNAMIC_FIELDS_PARAM, false,
+        AUTO_CREATE_FIELDS, false,
+        DESIGNER_KEY + ENABLE_NESTED_DOCS_PARAM, true,
+        DESIGNER_KEY + LANGUAGES_PARAM, Collections.singletonList("en"));
+    assertDesignerSettings(expSettings, settings);
+    assertTrue("should be disabled", dao.isDesignerDisabled(configSet));
+  }
+
+  protected void assertDesignerSettings(Map<String, Object> expectedMap, SchemaDesignerSettings actual) {
+    assertEquals(new SchemaDesignerSettings(expectedMap), actual);
+  }
+}
diff --git a/solr/solr-ref-guide/src/documents-fields-and-schema-design.adoc b/solr/solr-ref-guide/src/documents-fields-and-schema-design.adoc
index 125a024..22a0811 100644
--- a/solr/solr-ref-guide/src/documents-fields-and-schema-design.adoc
+++ b/solr/solr-ref-guide/src/documents-fields-and-schema-design.adoc
@@ -1,5 +1,5 @@
 = Documents, Fields, and Schema Design
-:page-children: overview-of-documents-fields-and-schema-design, solr-field-types, defining-fields, copying-fields, dynamic-fields, other-schema-elements, schema-api, putting-the-pieces-together, docvalues, schemaless-mode, luke-request-handler
+:page-children: overview-of-documents-fields-and-schema-design, solr-field-types, defining-fields, copying-fields, dynamic-fields, other-schema-elements, schema-api, putting-the-pieces-together, docvalues, schemaless-mode, luke-request-handler, schema-designer
 // 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
@@ -41,4 +41,6 @@ This section includes the following topics:
 
 <<schemaless-mode.adoc#,Schemaless Mode>>: Automatically add previously unknown schema fields using value-based field type guessing.
 
-<<luke-request-handler.adoc#,Luke Requst Handler>>: The request handler which provides access to information about fields in the index. This request handler powers the <<schema-browser-screen.adoc#,Schema Browser>> page of Solr's Admin UI.
+<<luke-request-handler.adoc#,Luke Request Handler>>: The request handler which provides access to information about fields in the index. This request handler powers the <<schema-browser-screen.adoc#,Schema Browser>> page of Solr's Admin UI.
+
+<<schema-designer.adoc#,Schema Designer>>: Provides an interactive experience to create a schema using sample data.
\ No newline at end of file
diff --git a/solr/solr-ref-guide/src/images/schema-designer/analyze-sample-docs.png b/solr/solr-ref-guide/src/images/schema-designer/analyze-sample-docs.png
new file mode 100644
index 0000000..debb102
Binary files /dev/null and b/solr/solr-ref-guide/src/images/schema-designer/analyze-sample-docs.png differ
diff --git a/solr/solr-ref-guide/src/images/schema-designer/field-filters.png b/solr/solr-ref-guide/src/images/schema-designer/field-filters.png
new file mode 100644
index 0000000..d6cbbb0
Binary files /dev/null and b/solr/solr-ref-guide/src/images/schema-designer/field-filters.png differ
diff --git a/solr/solr-ref-guide/src/images/schema-designer/incompat-change.png b/solr/solr-ref-guide/src/images/schema-designer/incompat-change.png
new file mode 100644
index 0000000..0dc2554
Binary files /dev/null and b/solr/solr-ref-guide/src/images/schema-designer/incompat-change.png differ
diff --git a/solr/solr-ref-guide/src/images/schema-designer/new-schema.png b/solr/solr-ref-guide/src/images/schema-designer/new-schema.png
new file mode 100644
index 0000000..8df41f0
Binary files /dev/null and b/solr/solr-ref-guide/src/images/schema-designer/new-schema.png differ
diff --git a/solr/solr-ref-guide/src/images/schema-designer/publish.png b/solr/solr-ref-guide/src/images/schema-designer/publish.png
new file mode 100644
index 0000000..adccf37
Binary files /dev/null and b/solr/solr-ref-guide/src/images/schema-designer/publish.png differ
diff --git a/solr/solr-ref-guide/src/images/schema-designer/query-tester.png b/solr/solr-ref-guide/src/images/schema-designer/query-tester.png
new file mode 100644
index 0000000..713d64b
Binary files /dev/null and b/solr/solr-ref-guide/src/images/schema-designer/query-tester.png differ
diff --git a/solr/solr-ref-guide/src/images/schema-designer/reload-schema.png b/solr/solr-ref-guide/src/images/schema-designer/reload-schema.png
new file mode 100644
index 0000000..7a88567
Binary files /dev/null and b/solr/solr-ref-guide/src/images/schema-designer/reload-schema.png differ
diff --git a/solr/solr-ref-guide/src/images/schema-designer/schema-designer.png b/solr/solr-ref-guide/src/images/schema-designer/schema-designer.png
new file mode 100644
index 0000000..5c87839
Binary files /dev/null and b/solr/solr-ref-guide/src/images/schema-designer/schema-designer.png differ
diff --git a/solr/solr-ref-guide/src/images/schema-designer/schema-editor-fields.png b/solr/solr-ref-guide/src/images/schema-designer/schema-editor-fields.png
new file mode 100644
index 0000000..ae1b837
Binary files /dev/null and b/solr/solr-ref-guide/src/images/schema-designer/schema-editor-fields.png differ
diff --git a/solr/solr-ref-guide/src/images/schema-designer/schema-editor-root.png b/solr/solr-ref-guide/src/images/schema-designer/schema-editor-root.png
new file mode 100644
index 0000000..b4d8bec
Binary files /dev/null and b/solr/solr-ref-guide/src/images/schema-designer/schema-editor-root.png differ
diff --git a/solr/solr-ref-guide/src/images/schema-designer/schema-editor-update.png b/solr/solr-ref-guide/src/images/schema-designer/schema-editor-update.png
new file mode 100644
index 0000000..1b27b11
Binary files /dev/null and b/solr/solr-ref-guide/src/images/schema-designer/schema-editor-update.png differ
diff --git a/solr/solr-ref-guide/src/images/schema-designer/show-changes.png b/solr/solr-ref-guide/src/images/schema-designer/show-changes.png
new file mode 100644
index 0000000..abcc4fc
Binary files /dev/null and b/solr/solr-ref-guide/src/images/schema-designer/show-changes.png differ
diff --git a/solr/solr-ref-guide/src/images/schema-designer/text-analysis.png b/solr/solr-ref-guide/src/images/schema-designer/text-analysis.png
new file mode 100644
index 0000000..8f6ffa6
Binary files /dev/null and b/solr/solr-ref-guide/src/images/schema-designer/text-analysis.png differ
diff --git a/solr/solr-ref-guide/src/overview-of-the-solr-admin-ui.adoc b/solr/solr-ref-guide/src/overview-of-the-solr-admin-ui.adoc
index 1eb9378..4fb25a8 100644
--- a/solr/solr-ref-guide/src/overview-of-the-solr-admin-ui.adoc
+++ b/solr/solr-ref-guide/src/overview-of-the-solr-admin-ui.adoc
@@ -59,6 +59,18 @@ Once logged in, the left-hand navigation will show the current user with an opti
 If Kerberos is enabled and the user has a valid ticket, the login screen will be skipped.
 However, if the user does not have a valid ticket, they will see a message that they need to obtain a valid ticket before continuing.
 
+== Schema Designer
+
+The <<schema-designer.adoc#,Schema Designer>> screen provides an interactive experience to create a schema using sample data.
+
+image::images/schema-designer/schema-designer.png[image]
+
+.Only Visible When Using SolrCloud
+[NOTE]
+====
+The Schema Designer is only available on Solr instances running in <<solrcloud.adoc#,SolrCloud>> mode.
+====
+
 == Getting Assistance
 
 At the bottom of each screen of the Admin UI is a set of links that can be used to get more assistance with configuring and using Solr.
diff --git a/solr/solr-ref-guide/src/schema-designer.adoc b/solr/solr-ref-guide/src/schema-designer.adoc
new file mode 100644
index 0000000..b48bec5
--- /dev/null
+++ b/solr/solr-ref-guide/src/schema-designer.adoc
@@ -0,0 +1,236 @@
+= Schema Designer
+:experimental:
+// 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.
+
+The Schema Designer screen lets you interactively design a new schema using sample data.
+
+.Schema Designer screen
+image::images/schema-designer/schema-designer.png[image]
+
+There are a number of panels on the Schema Designer screen to provide immediate feedback when you make changes to the schema, including:
+
+* Upload / paste sample documents to find fields and guess the correct field type and indexing strategy
+* Schema Editor tree to edit Fields, Dynamic Fields, Field Types, and supporting Files
+* Text Analysis panel to show the text analysis pipeline for sample text based on the selected field
+* Query Tester panel to see how schema change impact query matching, sorting, faceting, and hit highlighting
+* Show Changes dialog to view a report of all changes made by the designer before publishing
+
+The Schema Designer allows you to edit an existing schema, however its main purpose is to help you safely design a new schema from sample data.
+You can safely experiment with changes and see the impact on query results immediately.
+Once data is indexed using a published schema, there are severe restrictions on the type of changes you can make to the schema without needing a full re-index.
+When designing a new schema, the Schema Designer re-indexes your sample data automatically when you make changes. However, the designer does not re-index data in collections using a published schema.
+
+.Security Requirements
+[NOTE]
+====
+If the <<rule-based-authorization-plugin.adoc#,Rule-based Authorization Plugin>> is enabled for your Solr installation, then users need to have the `config-edit` and `config-read` permissions to use the Schema Designer.
+====
+
+== Getting Started
+
+Upon entering the Schema Designer for the first time, you'll be prompted to create a New Schema.
+
+image::images/schema-designer/new-schema.png[image,width=500]
+
+Choose a short name that reflects the intended use case for your new schema. You'll need to choose a source schema to copy as the starting point for your new schema.
+Solr includes a `_default` schema which provides a good starting place for building a custom schema for your search application.
+Once a schema is published, it can be used to create new schemas and will be listed in the *Copy from* drop-down list in the dialog.
+
+Once you create the new schema, the next step is to upload or paste a sample of the data you intend to index into Solr.
+The Schema Designer supports JSON, CSV, TSV, XML, and JSON lines (jsonl).
+
+image::images/schema-designer/analyze-sample-docs.png[image,width=400]
+
+The advantage of pasting sample documents into the text area is that you can edit the sample and see the impact of your changes immediately in the analyzed schema.
+The upload feature is useful if you have large or many sample documents; the Schema Designer API allows up to 1,000 sample documents or a max of 5MB upload, but in most cases you only need a handful of documents to get started.
+
+Click on the btn:[Analyze Documents] button to submit the sample documents to the Schema Designer API to generate your new schema.
+
+=== Temporary Configset and Collection
+
+Behind the scenes, the Schema Designer API creates a temporary <<config-sets.adoc#,Configset>> (schema + solrconfig.xml + supporting files) in Zookeeper.
+In addition, the Schema Designer API creates a temporary collection with a single shard and replica to hold sample documents.
+These temporary resources are persisted to disk and exist until the schema is published or manually deleted using the Schema Designer API cleanup endpoint (`/api/schema-designer/cleanup`).
+
+If you close your browser screen while designing a new schema, it will be available when you return.
+Simply choose the name of the schema you created previously in the select box and your schema will load into the designer UI.
+
+image::images/schema-designer/reload-schema.png[image,width=400]
+
+Previously uploaded sample documents are indexed in the temporary collection even though they do not display in the text area.
+
+[TIP]
+====
+Click on the btn:[Edit Documents] button on the *Query Results* panel to load a JSON representation of indexed documents into the text area.
+====
+
+=== Iteratively Post Sample Documents
+
+If you have sample documents spread across multiple files, you can POST them to the Schema Designer API and then load your schema in the Designer UI to design your schema.
+Here's an example of how to use the API to "prepare" a new schema and then iteratively post Solr's techproducts example files to the Schema Designer backend:
+
+[source,bash]
+----
+#!/bin/bash
+
+SOLR_INSTALL_DIR="path/to/solr/install"
+
+DIR_WITH_SAMPLE_FILES="$SOLR_INSTALL_DIR/example/exampledocs"
+
+SOLR_URL=http://localhost:8983
+
+MY_NEW_SCHEMA="myNewSchema"
+
+echo "Preparing new schema: ${MY_NEW_SCHEMA}"
+curl -s -o /dev/null -w "%{http_code}" -XPOST \
+  "$SOLR_URL/api/schema-designer/prep?configSet=${MY_NEW_SCHEMA}&copyFrom=_default"
+echo ""
+
+SAMPLE_FILES=( $(ls ${DIR_WITH_SAMPLE_FILES}/*.{xml,csv,json,jsonl}) )
+for f in "${SAMPLE_FILES[@]}"
+do
+  echo "POST'ing contents of $f to Schema Designer analyze endpoint ..."
+  curl -s -o /dev/null -w "%{http_code}" -XPOST \
+    "$SOLR_URL/api/schema-designer/analyze?configSet=${MY_NEW_SCHEMA}" -d @"$f"
+  echo ""
+done
+----
+
+After sending the sample documents to the Schema Designer backend, you can open the *prepared* schema in the Schema Designer screen in your browser.
+
+[NOTE]
+====
+The Schema Designer API is primarily intended to support an interactive experience in the UI vs. being used programmatically by developers.
+To create and manage Configsets and Schemas programmatically, see the <<configsets-api.adoc#,Configset>> and <<schema-api.adoc#,Schema>> APIs.
+====
+
+== Schema Editor
+
+After analyzing your sample documents, the Schema Designer loads the schema in the *Schema Editor* in the middle panel.
+The editor renders the schema as a tree component composed of Fields, Dynamic Fields, Field Types, and Files.
+For more information about schema objects, see <<documents-fields-and-schema-design.adoc#,Documents, Fields, and Schema Design>>.
+
+image::images/schema-designer/schema-editor-root.png[image,width=700]
+
+.Schema vs. Configset
+[NOTE]
+====
+A Configset includes a schema, so technically the Schema Designer works with a Configset behind the scenes.
+However, Configset is more of a technical implementation detail and your primary focus when designing a new search application should be on the fields and their types.
+Consequently, the Schema Designer focuses primarily on the schema aspects of a Configset vs. exposing complexities of a Configset in the UI.
+====
+
+When you click on the root node of the Schema Editor tree, you can refine top-level schema properties, including:
+
+* Languages: The `_default` schema includes text fields for a number of common languages. You can include all text analyzers in your schema or select a subset based on the languages your search application needs to support. The designer will remove all the unnecessary field types for languages you don't need. For more information about text analysis and languages, see: <<language-analysis.adoc#,Language Analysis>>
+* Dynamic fields allow Solr to index fields that you did not explicitly define in your schema. Dynamic fields can make your application less brittle by providing some flexibility in the documents you can add to Solr. It is recommended to keep the default set of dynamic fields enabled for your schema. Unchecking this option removes all dynamic fields from your schema. For more information about dynamic fields, see: <<dynamic-fields.adoc#,Dynamic Fields>>
+* Field guessing (aka "schemaless mode") allows Solr to detect the "best" field type for unknown fields encountered during indexing. Field guessing also performs some field transformations, such as removing spaces from field names. If you use the schema designer to create your schema based on sample documents, you may not need to enable this feature. However, with this feature disabled, you need to make sure the incoming data matches the schema exactly or indexing errors may occur. For m [...]
+* Enabling this feature adds the `+++_root_+++` and `+++_nest_path_+++` fields to your schema. For more information about indexing nested child documents, see: <<indexing-nested-documents.adoc#,Indexing Nested Documents>>
+
+Only make changes to these top-level schema properties when you fully understand how they impact the behavior of your search application.
+When first starting out, you can leave the default settings and focus your attention on the fields and field types in the schema.
+
+=== Schema Fields
+
+Click on the *Fields* node in the editor tree to see an overview of the fields in your schema,
+along with the <<field-type-definitions-and-properties.adoc#,properties>> that govern how the field will be indexed by Solr.
+
+image::images/schema-designer/schema-editor-fields.png[image,width=750]
+
+If there are many fields in your schema, you can filter the displayed fields by type or feature using the filters at the top of the tree.
+For instance, to filter fields with `docValues` enabled, choose `feature` and then `docValues` with `enabled` checked.
+
+image::images/schema-designer/field-filters.png[image,width=500]
+
+To edit a field, click on the name of the field in the tree to load the field information in the main editor panel to the right of the tree.
+
+image::images/schema-designer/schema-editor-update.png[image,width=750]
+
+[TIP]
+====
+Before changing properties for a specific field, consider if you should change the property on the *field type* instead, as changes applied to a field type will apply to all fields using that type.
+====
+
+After making the desired changes, click on the btn:[Update Field] button. The Schema Designer backend API will apply the changes to the schema and then re-index the sample documents into the temporary collection if needed.
+Some changes, such as changing a single-valued `docValues` field to multi-valued, may require the underlying Lucene index to be deleted and rebuilt from scratch.
+The Schema Designer will warn you when your change requires a full rebuild.
+
+image::images/schema-designer/incompat-change.png[image,width=450]
+
+Moreover, if you change a field to an incompatible type based on the sample data, the designer will undo the changes automatically.
+For instance, changing a field with text data to a numeric type will be rejected by the designer.
+
+[IMPORTANT]
+====
+Be careful changing field properties for schemas that are already being used by collections with indexed data.
+The designer cannot protect you from making an incompatible change for in-use schemas.
+Typically adding new fields and field types is a safe operation for existing schemas. Changing field and/or field type properties can lead to index corruption.
+====
+
+The Schema Designer does not support deleting fields from the schema.
+
+=== Text Analysis
+
+When you select a text-based field in the tree, the *Text Analysis* panel shows how the text from a sample document gets analyzed for indexing.
+
+image::images/schema-designer/text-analysis.png[image,width=600]
+
+If you need to change the text analysis strategy for a field, you need to edit the Field Type. For more information about text analysis, see: <<analyzers.adoc#,Analyzers>>.
+
+== Query Tester
+
+The *Query Tester* panel lets you experiment with queries executed against your sample document set using the current schema.
+Using the Query Tester, you can see how changes to the schema impact the behavior of queries, such as matching, sorting, faceting, and highlighting.
+The Query Tester form is not intended to demonstrate all possible <<query-syntax-and-parsing.adoc#,query features>> available in Solr.
+
+image::images/schema-designer/query-tester.png[image]
+
+Whenever you make a change to the schema, the query tester form is re-submitted to refresh the query results.
+This helps inform you about matching issues where a test query should find a sample document but doesn't return the correct results,
+which indicates that a field was not indexed correctly, such as using `string` instead of a text-analyzed field type.
+
+Matching issues may also indicate that a field was not copied into the default search field, such as the catch-all `+++_text_+++` field.
+
+The Query Tester form lists all fields that support sorting in the *Sort by* drop-down. If a field you need to sort by is not listed, then make sure it is single-valued and is either indexed or has docValues enabled.
+
+Similarly, the tester form lists all fields that you can compute facets for; if a field is not listed in the facet drop-down, then make sure it is indexed or has docValues enabled.
+
+== Show Changes
+
+Click on the btn:[Show Changes] button to view a report of the changes made to the schema during the current editing session.
+The un-published changes are compared to either the original schema you copied from (such as `_default`) or the published schema.
+
+image::images/schema-designer/show-changes.png[image,width=600]
+
+You need to take care when changing schemas that have already been published if there are collections with data indexed using the schema.
+
+== Publish
+
+Publishing a new schema makes it available for use when creating new collections. The Publish dialog shows if any existing collections
+will be affected by the publish action; of course there will not be any impacted collections for new schemas. You may also create a new
+collection during publishing that uses the new schema and choose whether to index your sample documents in the new collection.
+
+image::images/schema-designer/publish.png[image,width=450]
+
+You can also choose to prevent future changes to this schema by the Schema Designer. However, this setting only applies to the Schema Designer UI
+and does not prevent someone from changing the schema using the Schema API directly.
+
+Once the publish action completes, the temporary Configset and collection are deleted and the Schema Designer UI resets back to a fresh state.
+
+Alternatively, instead of publishing to Zookeeper, you can also download the Configset to a zip file containing the schema, solrconfig.xml, and supporting files.
+The zip file can be uploaded to other Solr instances using the <<configsets-api.adoc#,Configset API>> or saved in version control.
diff --git a/solr/webapp/web/css/angular/menu.css b/solr/webapp/web/css/angular/menu.css
index d5c2ec1..8789637 100644
--- a/solr/webapp/web/css/angular/menu.css
+++ b/solr/webapp/web/css/angular/menu.css
@@ -272,6 +272,8 @@ limitations under the License.
 #menu #cloud.global .zkstatus a { background-image: url( ../../img/ico/node-leader.png ); }
 #menu #cloud.global .graph a { background-image: url( ../../img/ico/molecule.png ); }
 
+#menu #schema-designer.global p a { background-image: url( ../../img/ico/book-open-text.png ); }
+
 .sub-menu .ping.error a
 {
 
diff --git a/solr/webapp/web/css/angular/schema-designer.css b/solr/webapp/web/css/angular/schema-designer.css
new file mode 100644
index 0000000..eb281d8
--- /dev/null
+++ b/solr/webapp/web/css/angular/schema-designer.css
@@ -0,0 +1,1603 @@
+/*
+
+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.
+
+*/
+#content #designer
+{
+  position: relative;
+}
+
+#content #designer .loader
+{
+  background-position: 0 50%;
+  padding-left: 21px;
+}
+
+#content #designer #designer-top
+{
+  padding: 0px;
+}
+
+#content #designer #tree-content
+{
+  overflow: auto;
+  float: left;
+  height: 520px;
+  width: 275px;
+  padding: 3px;
+}
+#content #designer #tree-node-content
+{
+  height: 560px;
+  margin: 2px;
+  overflow: hidden;
+}
+
+#content #designer .designer-left
+{
+  float: left;
+  width: 380px;
+  min-width: 380px; /* so the text area doesn't bleed over to the right */
+  padding: 7px;
+}
+
+#content #designer .designer-middle
+{
+  float: left;
+  width: 48%;
+  padding: 7px;
+  height: 585px;
+}
+
+#content #designer .designer-right
+{
+  padding: 7px;
+  overflow: hidden;
+}
+
+#content #designer .query-tester-left
+{
+  float: left;
+  width: 30%;
+  padding: 7px;
+}
+
+#content #designer .query-tester-right
+{
+  padding: 7px;
+  overflow: auto;
+}
+
+#content #designer .data
+{
+  padding-bottom: 12px;
+  overflow: hidden;
+}
+
+#content #designer .data:hover
+{
+  overflow-x: auto;
+}
+
+#content #designer .data li
+{
+  padding-top: 3px;
+  padding-bottom: 3px;
+}
+
+#content #designer .data li dt
+{
+  float: left;
+  white-space: nowrap;
+  width: 20%;
+}
+
+#content #designer .data li dd
+{
+  float: right;
+  text-overflow: ellipsis;
+  white-space: nowrap;
+  width: 80%
+}
+
+#content #designer .data li dd.odd
+{
+  background-color: #f0f0f0;
+}
+
+#content #designer .data dt span
+{
+  background-position: 1px 50%;
+  display: block;
+  padding-left: 22px;
+  font-weight: bold;
+  text-align: left;
+}
+
+#content #designer #sample-docs h2 { background-image: url( ../../img/ico/document-text.png ); }
+#content #designer #query-tester h2 { background-image: url( ../../img/ico/magnifier.png ); }
+#content #designer #query-results h2 { background-image: url( ../../img/ico/documents-stack.png ); }
+#content #designer #schema-editor h2 { background-image: url( ../../img/ico/book-open-text.png ); }
+#content #designer #analysis h2 { background-image: url( ../../img/ico/property.png ); }
+
+#content #designer #form
+{
+  margin-bottom: 10px;
+}
+
+#content #designer #form label
+{
+  cursor: pointer;
+  display: block;
+  margin-top: 5px;
+}
+
+#content #designer #field-form-left
+{
+  float: left;
+  width: 45%;
+  padding: 7px;
+}
+
+#content #designer #field-form-right
+{
+  margin-top: 8px;
+  padding: 7px;
+  overflow: hidden;
+}
+
+#content #designer .field-form
+{
+  display: block;
+  padding: 5px;
+}
+
+#content #designer .field-form-right
+{
+  display: block;
+  padding: 5px;
+}
+
+#content #designer .field-form-checkbox {
+  margin-top: 10px;
+  margin-left: 130px;
+  display: block;
+}
+
+#content #designer .field-form-right label
+{
+  float: left;
+  padding-top: 3px;
+  padding-bottom: 3px;
+  text-align: right;
+  width: 120px;
+  margin-right: 6px;
+}
+
+#content #designer #languages
+{
+  width: 125px;
+}
+
+#content #designer .checkbox
+{
+  text-align: left;
+  padding-left: 3px;
+}
+
+#content #designer .actions
+{
+  margin-bottom: 5px;
+}
+
+#content #designer .actions form .error
+{
+  background-image: url( ../../img/ico/cross-button.png );
+  background-position: 22% 1px;
+  color: #c00;
+  font-weight: bold;
+}
+
+#content #designer .actions form p
+{
+  padding-bottom: 8px;
+}
+
+#content #designer .actions form label
+{
+  float: left;
+  padding-top: 3px;
+  padding-bottom: 3px;
+  text-align: right;
+  width: 25%;
+  margin-right: 6px;
+}
+
+#content #designer .actions #download-button span
+{
+  background-image: url( ../../img/ico/download-cloud.png );
+}
+
+#content #designer #schema-selector .left
+{
+  float: left;
+  width: 405px;
+}
+
+#content #designer #schema-selector .middle
+{
+  float: left;
+  width: 48%;
+}
+
+#content #designer #schema-selector .right
+{
+  text-align: right;
+  overflow: hidden;
+  margin-right: 10px;
+}
+
+#content #designer #schema-selector #apply
+{
+  float: right;
+  text-align: right;
+  width: 300px;
+}
+
+#content #designer #schema-selector a
+{
+  padding: 0;
+  padding-left: 8px;
+}
+
+#content #designer #schema-selector select
+{
+  width: 150px;
+}
+
+#content #designer .actions form input,
+#content #designer .actions form select,
+#content #designer .actions form .buttons,
+#content #designer .actions form .note span
+{
+  float: right;
+  width: 71%;
+}
+
+#content #designer .actions form .note span
+{
+  padding-left: 3px;
+  padding-right: 3px;
+}
+
+#content #designer .actions form .buttons
+{
+  padding-top: 10px;
+}
+
+#content #designer .actions form button.submit
+{
+  margin-right: 20px;
+}
+
+#content #designer .actions form button.submit span
+{
+  background-image: url( ../../img/ico/tick.png );
+}
+
+#content #designer .actions form button.reset span
+{
+  background-image: url( ../../img/ico/cross.png );
+}
+
+#content #designer .actions #add span
+{
+  background-image: url( ../../img/ico/plus-button.png );
+}
+
+#content #designer .actions #unload
+{
+  margin-right: 20px;
+}
+
+#content #designer .actions #unload span
+{
+  background-image: url( ../../img/ico/cross.png );
+}
+
+#content #designer .actions #reload span
+{
+  background-image: url( ../../img/ico/arrow-circle.png );
+}
+
+#content #designer .actions #rename span
+{
+  background-image: url( ../../img/ico/ui-text-field-select.png );
+}
+
+#content #designer .actions #swap span
+{
+  background-image: url( ../../img/ico/arrow-switch.png );
+}
+
+#content #designer .actions div.action
+{
+  z-index: 100;
+  background-color: #fff;
+  border: 1px solid #f0f0f0;
+  box-shadow: 5px 5px 10px #c0c0c0;
+  -moz-box-shadow: 5px 5px 10px #c0c0c0;
+  -webkit-box-shadow: 5px 5px 10px #c0c0c0;
+  position: absolute;
+  left: 0px;
+  top: 36px;
+  padding: 10px;
+  width: 320px;
+}
+
+/* Sample Documents */
+
+#content #designer #analyze-buttons{
+  float: right;
+  margin-right: 12px;
+}
+
+#content #designer #query-buttons{
+  margin-top: 20px;
+  float: right;
+  margin-right: 8px;
+}
+
+#content #designer #queryButton span
+{
+  background-image: url( ../../img/ico/magnifier.png );
+}
+
+#content #designer #analyze span
+{
+  background-image: url( ../../img/ico/gear.png );
+}
+
+#content #designer #document-container
+{
+  padding: 5px;
+}
+
+#content #designer #file-upload
+{
+  margin-bottom: 20px;
+}
+
+/* Section header bars */
+
+#content #designer .bar-desc
+{
+  color: #4D4D4D;
+  font-weight: normal;
+  margin-left: 10px;
+  white-space: pre;
+}
+
+#content #designer .bar-holder
+{
+  box-shadow: 5px 5px 10px #c0c0c0;
+  -moz-box-shadow: 5px 5px 10px #c0c0c0;
+  -webkit-box-shadow: 5px 5px 10px #c0c0c0;
+  height: 35px;
+}
+
+#content #designer .bar-holder .bar
+{
+  height: 100%;
+  position: relative;
+}
+
+#content #designer .bar-holder div .val
+{
+  border-right: 1px solid #f00;
+  display: block;
+  padding-right: 5px;
+  position: absolute;
+  right: 0;
+  top: 35px;
+  white-space: nowrap;
+}
+
+#content #designer .bar-holder .bar-max.bar
+{
+  background-color: #f0f0f0;
+}
+
+#content #designer .bar-holder .bar-max.val
+{
+  border-color: #f0f0f0;
+  color: #8D8D8D;
+}
+
+#content #designer .bar-holder .bar-total.bar
+{
+  background-color: #c0c0c0;
+}
+
+#content #designer .bar-holder .bar-total.val
+{
+  border-color: #c0c0c0;
+  color: #4D4D4D;
+}
+
+#content #designer .bar-holder .bar-used.bar
+{
+  background-color: #969696;
+}
+
+#content #designer .bar-holder .bar-used.val
+{
+  border-color: #969696;
+  color: #969696;
+}
+
+#content #designer .bar-holder.bar-lvl-2 .bar-max.val { padding-top: 25px; }
+#content #designer .bar-holder.bar-lvl-2 .bar-total.val { padding-top: 5px; }
+#content #designer .bar-holder.bar-lvl-2 { margin-bottom: 45px; }
+
+#content #designer .bar-holder.bar-lvl-3 .bar-max.val { padding-top: 45px; }
+#content #designer .bar-holder.bar-lvl-3 .bar-total.val { padding-top: 25px; }
+#content #designer .bar-holder.bar-lvl-3 .bar-used.val { padding-top: 5px; }
+#content #designer .bar-holder.bar-lvl-3 { margin-bottom: 65px; }
+
+#content #designer .analyzer,
+#content #designer .analyzer li,
+#content #designer .analyzer ul,
+#content #designer .analyzer ul li
+{
+}
+
+#content #designer .analyzer p,
+#content #designer .analyzer dl
+{
+  float: left;
+}
+
+#content #designer .analyzer p
+{
+  margin-right: 5px;
+  text-align: left;
+  width: 100px;
+  white-space: pre;
+}
+
+#content #designer .analyzer p a
+{
+  cursor: auto;
+}
+
+#content #designer .analyzer p a.analysis
+{
+  cursor: pointer;
+  display: block;
+}
+
+#content #designer .analyzer p a.analysis span
+{
+  background-image: url( ../../img/ico/question-white.png );
+  background-position: 0 50%;
+  padding-left: 21px;
+}
+
+#content #designer .analyzer p a.analysis:hover span
+{
+  background-image: url( ../../img/ico/question.png );
+  color: #008;
+}
+
+#content #designer .analyzer a
+{
+  cursor: pointer;
+}
+
+#content #designer .file-link
+{
+  cursor: pointer;
+  color: #003eff;
+  text-decoration-line: underline;
+  text-decoration-color: #003eff;
+}
+
+#content #designer .analyzer .toggle
+{
+  background-image: url( ../../img/ico/chevron-small-expand.png );
+  background-position: 100% 50%;
+  cursor: pointer;
+  display: block;
+  padding-right: 21px;
+}
+
+#content #designer .analyzer .open .toggle
+{
+  background-image: url( ../../img/ico/chevron-small.png );
+}
+
+#content #designer .analyzer li
+{
+  border-top: 1px solid #f0f0f0;
+  margin-top: 10px;
+  padding-top: 10px;
+  margin-left: 10px;
+}
+
+#content #designer .analyzer ul
+{
+  clear: left;
+  margin-left: 25px;
+  padding-top: 5px;
+}
+
+#content #designer .analyzer .open ul
+{
+  display: block;
+}
+
+#content #designer .analyzer ul li
+{
+  border-top: 1px solid #f8f8f8;
+  margin-top: 5px;
+  padding-top: 5px;
+}
+
+#content #designer .analyzer ul p
+{
+  color: #4D4D4D;
+  margin-right: 5px;
+  text-align: left;
+  width: 80px;
+}
+
+#content #designer .analyzer ul dd
+{
+  margin-left: 10px;
+}
+
+#content #designer .analyzer ul dd
+{
+  background-image: url( ../../img/ico/document-list.png );
+  background-position: 0 50%;
+  color: #4D4D4D;
+  padding-left: 21px;
+}
+
+#content #designer .analyzer ul dd.ico-0
+{
+  background-image: url( ../../img/ico/slash.png );
+}
+
+#content #designer .analyzer ul dd.ico-1
+{
+  background-image: url( ../../img/ico/tick.png );
+}
+
+#content #designer #field-top
+{
+  width: 100%;
+}
+
+#content #designer #field-top-left
+{
+  float: left;
+  width: 240px;
+  padding: 7px;
+}
+
+#content #designer #field-top-right
+{
+  display: contents;
+}
+
+#content #designer #tree-node-content form button.submit
+{
+  margin-left: 140px;
+}
+
+#content #designer #tree-node-content form button.submit span
+{
+  background-image: url( ../../img/ico/tick.png );
+}
+
+#content #designer #field-bottom
+{
+  float: left;
+  width: 100%;
+}
+
+#content #designer #schema-actions {
+  float: left;
+  width: 99%;
+}
+
+#content #designer .schema-actions #addField { margin-right: 15px; }
+#content #designer .schema-actions #addFieldType { margin-right: 15px; }
+
+#content #designer .schema-actions #addField span { background-image: url( ../../img/ico/document-list.png ); }
+#content #designer .schema-actions #addCopyField span { background-image: url( ../../img/ico/document-import.png ); }
+#content #designer .schema-actions #addFieldType span { background-image: url( ../../img/ico/document-text.png ); }
+
+#content #designer #updateSelected { float: right; margin-top:5px; }
+#content #designer #updateSelected span { background-image: url( ../../img/ico/tick.png ); }
+#content #designer #deleteSelected { margin-right: 0px; }
+#content #designer #deleteSelected span { background-image: url( ../../img/ico/cross-button.png ); }
+
+#content #designer #updateStatusMessage { color: darkgreen; float: right; margin-top: 5px; background-image: url( ../../img/ico/tick.png ); }
+#content #designer #updateStatusMessage span { margin-left: 18px; }
+#content #designer .working { font-style: italic; color: black !important; float: right; margin-top: 5px; background-image: url( ../../img/loader.gif ) !important; }
+
+#content #designer .schema-actions div.action
+{
+  z-index: 100;
+  background-color: #fff;
+  border: 1px solid #f0f0f0;
+  box-shadow: 5px 5px 10px #c0c0c0;
+  -moz-box-shadow: 5px 5px 10px #c0c0c0;
+  -webkit-box-shadow: 5px 5px 10px #c0c0c0;
+  position: absolute;
+  left: 448px;
+  top: 31px;
+  padding: 10px;
+  width: 350px;
+}
+
+#content #designer .schema-actions div.copyfield
+{
+  z-index: 100;
+  background-color: #fff;
+  border: 1px solid #f0f0f0;
+  box-shadow: 5px 5px 10px #c0c0c0;
+  -moz-box-shadow: 5px 5px 10px #c0c0c0;
+  -webkit-box-shadow: 5px 5px 10px #c0c0c0;
+  position: absolute;
+  left: 700px;
+  top: 31px;
+  padding: 10px;
+  width: 350px;
+}
+
+
+#content #designer .schema-actions p
+{
+  padding-bottom: 8px;
+}
+
+#content #designer .schema-actions label
+{
+  margin-right: 6px;
+  float: left;
+  padding-top: 3px;
+  padding-bottom: 3px;
+  text-align: right;
+  width: 25%;
+}
+
+#content #designer .schema-actions input,
+#content #designer .schema-actions select,
+#content #designer .schema-actions .buttons,
+#content #designer .schema-actions .note span
+{
+  float: right;
+  width: 71%;
+}
+
+#content #designer #add-errors
+{
+  float: left;
+  margin-top:10px;
+  margin-bottom: 10px;
+  min-height: 30px;
+}
+
+#content #designer #add-errors .error
+{
+  background-image: url( ../../img/ico/cross-button.png );
+  background-position: 10px 1px;
+  color: #c00;
+  font-weight: bold;
+}
+
+#content #designer #add-errors .error span
+{
+  margin-left: 30px;
+}
+
+#content #designer .schema-actions .chosen-container
+{
+  width: 71% !important;
+}
+
+#content #designer .chosen-container {
+  width: 100%;
+}
+#content #designer .chosen-drop input,
+#content #designer .chosen-results {
+  width: 100% !important;
+}
+
+#content #designer .schema-actions label.checkbox {
+  margin-left: 27%;
+  text-align: left;
+  width: 73%;
+  padding: 0px;
+  margin-top: 0px;
+}
+#content #designer .schema-actions .checkbox input {
+  float: none;
+  width: auto;
+}
+
+#content #designer .add_showhide {
+  background-image: url( ../../img/ico/chevron-small-expand.png );
+  background-position: 100% 50%;
+  cursor: pointer;
+  padding-right: 21px;
+}
+
+#content #designer .add_showhide.open {
+  background-image: url( ../../img/ico/chevron-small.png );
+}
+
+#content #designer #schema-actions button span
+{
+  background-image: url( ../../img/ico/cross.png );
+}
+#content #designer #schema-actions button.submit span
+{
+  background-image: url( ../../img/ico/tick.png );
+}
+
+#content #designer #field-analysis
+{
+  margin-bottom: 0;
+  overflow: auto;
+  max-height: 135px;
+}
+
+#content #designer #analysis #field-analysis-holder
+{
+  width: 100%;
+  padding: 5px;
+  max-height: 480px;
+  overflow: auto;
+}
+
+#content #designer #analysis #field-analysis-holder #field-analysis-form
+{
+  width: 100%;
+  padding: 5px;
+}
+
+#content #designer #analysis .settings
+{
+  clear: both;
+  padding: 5px;
+}
+
+#content #designer #analysis select
+{
+  width: 200px;
+}
+
+#content #designer #analysis .settings select.loader
+{
+  background-position: 3px 50%;
+  padding-left: 21px;
+}
+
+#content #designer #analysis .settings select optgroup
+{
+  font-style: normal;
+  padding: 5px;
+}
+
+#content #designer #analysis .settings select option
+{
+  padding-left: 10px;
+}
+
+#content #designer #analysis .settings label
+{
+  display: block;
+  width: 150px;
+}
+
+#content #designer #analysis .settings textarea
+{
+  display: block;
+}
+
+#content #designer #analysis .settings
+{
+  float: left;
+  margin-right: 0.5%;
+}
+
+#content #designer #analysis .analysis-error
+{
+  background-color: #f00;
+  background-image: url( ../../img/ico/construction.png );
+  background-position: 10px 50%;
+  color: #fff;
+  font-weight: bold;
+  margin-bottom: 20px;
+  padding: 10px;
+  padding-left: 35px;
+}
+
+#content #designer #analysis .analysis-error .head a
+{
+  color: #fff;
+  cursor: auto;
+}
+
+#content #designer #analysis #analysis-result
+{
+  width: 100%;
+  float: left;
+  overflow: auto;
+}
+
+#content #designer #analysis #analysis-result .index
+{
+  background-color: #fff;
+  padding-top: 20px;
+}
+
+#content #designer #analysis #analysis-result table
+{
+  border-collapse: collapse;
+}
+
+#content #designer #analysis #analysis-result td
+{
+  vertical-align: top;
+  white-space: nowrap;
+}
+
+#content #designer #analysis #analysis-result td.part.analyzer div,
+#content #designer #analysis #analysis-result td.part.spacer .holder,
+#content #designer #analysis #analysis-result td td td
+{
+  padding-top: 1px;
+  padding-bottom: 1px;
+}
+
+#content #designer #analysis #analysis-result.verbose_output td.legend
+{
+  display: table-cell;
+}
+
+#content #designer #analysis #analysis-result.verbose_output td.data tr.verbose_output
+{
+  display: table-row;
+}
+
+#content #designer #analysis #analysis-result .match
+{
+  background-color: #F0D9C3;
+}
+
+#content #designer #analysis #analysis-result td.part
+{
+  padding-bottom: 10px;
+}
+
+#content #designer #analysis #analysis-result td.part.analyzer div
+{
+  border-right: 1px solid #f0f0f0;
+  padding-right: 10px;
+}
+
+#content #designer #analysis #analysis-result td.part.analyzer abbr
+{
+  color: #4D4D4D;
+}
+
+#content #designer #analysis #analysis-result td.part.legend .holder,
+#content #designer #analysis #analysis-result td.part.data .holder
+{
+  padding-left: 10px;
+  padding-right: 10px;
+  border-right: 1px solid #c0c0c0;
+}
+
+#content #designer #analysis #analysis-result td.part.legend td
+{
+  color: #4D4D4D;
+}
+
+#content #designer #analysis #analysis-result td.part.legend .holder
+{
+  border-right-color: #f0f0f0;
+}
+
+#content #designer #analysis #analysis-result td.part.data:last-child .holder
+{
+  padding-right: 0;
+  border-right: 0;
+}
+
+#content #designer #analysis #analysis-result td.details
+{
+  padding-left: 10px;
+  padding-right: 10px;
+  border-left: 1px solid #f0f0f0;
+  border-right: 1px solid #f0f0f0;
+}
+
+#content #designer #analysis #analysis-result td.details:first-child
+{
+  padding-left: 0;
+  border-left: 0;
+}
+
+#content #designer #analysis #analysis-result td.details:last-child
+{
+  padding-right: 0;
+  border-right: 0;
+}
+
+#content #designer #analysis #analysis-result td.details tr.empty td
+{
+  color: #f0f0f0;
+}
+
+#content #designer #analysis #analysis-result td.details tr.raw_bytes td
+{
+  letter-spacing: -1px;
+}
+
+#content #designer #analysis #analysis-result .part table table td
+{
+  border-top: 1px solid #f0f0f0;
+}
+
+#content #designer #analysis #analysis-result .part table table tr:first-child td
+{
+  border-top: 0;
+}
+
+#content #designer #analysis #field-analysis h2 { background-image: url( ../../img/ico/receipt.png ); }
+#content #designer #analysis .analysis-result h2 { background-image: url( ../../img/ico/receipt-invoice.png ); }
+
+#content #designer #schema-root-details
+{
+  min-height: 360px;
+  padding: 7px;
+}
+
+#content #designer #schema-root-details #unique_key
+{
+  width: 150px;
+}
+
+#content #designer #schema-root-details .note span
+{
+  float: right;
+  padding-left: 18px;
+}
+
+#content #designer #schema-root-details .error
+{
+  background-image: url( ../../img/ico/cross-button.png );
+  background-position: 0px 2px;
+  color: #c00;
+  font-weight: bold;
+}
+
+/* diff dialog */
+#content #designer div.diff
+{
+  z-index: 100;
+  position: absolute;
+  background-color: #fff;
+  border: 1px solid #f0f0f0;
+  box-shadow: 5px 5px 10px #c0c0c0;
+  -moz-box-shadow: 5px 5px 10px #c0c0c0;
+  -webkit-box-shadow: 5px 5px 10px #c0c0c0;
+  top: 31px;
+  left: 650px;
+  padding: 10px;
+  width: 600px;
+  overflow: auto;
+  min-height: 370px;
+  max-height: 600px;
+}
+
+#content #designer table.diff {
+  font-family: arial, sans-serif;
+  border-collapse: collapse;
+  width: 100%;
+  display: inline-block;
+  overflow: auto;
+}
+
+#content #designer table.diff th {
+  font-family: arial, sans-serif;
+  font-weight: bold;
+  min-width: 100px;
+}
+
+#content #designer p.diff-text {
+  margin-top: 15px;
+  margin-bottom: 5px;
+  font-size: medium;
+  font-weight: bold;
+}
+
+#content #designer p.diff-field-name {
+  margin-top: 5px;
+  margin-bottom: 15px;
+  font-size: small;
+  font-weight: bold;
+}
+
+#content #designer li.element
+{
+  border: 1px solid #f0f0f0;
+  box-shadow: 5px 5px 10px #c0c0c0;
+  -moz-box-shadow: 5px 5px 10px #c0c0c0;
+  -webkit-box-shadow: 5px 5px 10px #c0c0c0;
+  margin-top: 5px;
+  margin-bottom: 15px;
+}
+
+#content #designer .actions .diff-buttons {
+  margin-right: 15px;
+  float: right;
+  width: 48%;
+}
+
+#content #designer p.old-value {
+  color: indianred;
+  text-decoration: line-through;
+}
+
+#content #designer p.rem-value {
+  color: indianred;
+}
+
+#content #designer p.new-value {
+  color: green;
+}
+
+/* publish dialog */
+#content #designer .actions div.publish
+{
+  z-index: 100;
+  position: absolute;
+  background-color: #fff;
+  border: 1px solid #f0f0f0;
+  box-shadow: 5px 5px 10px #c0c0c0;
+  -moz-box-shadow: 5px 5px 10px #c0c0c0;
+  -webkit-box-shadow: 5px 5px 10px #c0c0c0;
+  top: 31px;
+  left: 1150px;
+  padding: 10px;
+  width: 380px;
+  height: 430px;
+}
+
+#content #designer .actions .publish-button {
+  margin-right: 15px;
+}
+
+#content #designer .actions .publish-button span
+{
+  background-image: url( ../../img/ico/disk-black.png );
+}
+
+#content #designer .actions .diff-button {
+  margin-right: 15px;
+}
+
+#content #designer .actions .diff-button span {
+  background-image: url( ../../img/ico/document-convert.png );
+}
+
+#content #designer .actions button.reset span
+{
+  background-image: url( ../../img/ico/cross.png );
+}
+
+#content #designer #publish-dialog-note {
+  margin-bottom: 15px;
+}
+
+#content #designer #publish-new-coll {
+  margin-top: 15px;
+  margin-bottom: 10px;
+}
+
+#content #designer .publish-buttons
+{
+  margin-top: 20px;
+  float: right;
+  width: 48%;
+}
+
+#content #designer #publish-affected {
+  overflow: auto;
+  height: 60px;
+}
+
+#content #designer #reload-form {
+  height: 20px;
+}
+
+#content #designer #file-node-text {
+  display:block;
+}
+
+#content #designer #file-node {
+  padding: 7px;
+}
+
+#content #designer #save-file span {
+  background-image: url( ../../img/ico/disk-black.png );
+}
+
+#content #designer #results-tree-left {
+  float: left;
+  width: 330px;
+  overflow: auto;
+  height: 350px;
+}
+
+#content #designer #results-tree-right {
+  max-width: 800px;
+  float: left;
+  overflow: auto;
+  height: 350px;
+}
+
+#content #designer #save-file-buttons {
+  margin-top: 15px;
+  margin-right: 5px;
+  display: block;
+  float: right;
+}
+
+#content #designer #copy_dest {
+  width: 300px;
+}
+
+#content #designer .hit-field {
+  padding: 3px;
+  width: 120px;
+}
+
+#content #designer .hit-value {
+  padding: 3px;
+  word-wrap: break-word;
+  max-width: 700px;
+}
+
+#content #designer #leaf-node .error
+{
+  background-image: url( ../../img/ico/cross-button.png );
+  background-position: 0px 2px;
+  color: #c00;
+  font-weight: bold;
+}
+
+#content #designer #leaf-node .error span
+{
+  padding-left: 18px;
+}
+
+#content #designer #sample-docs .note
+{
+  background-image: url( ../../img/ico/information.png );
+  background-position: 0px 2px;
+  color: #008;
+  font-weight: bold;
+}
+
+#content #designer #confirm-dialog
+{
+  width: 350px;
+  height: 250px;
+}
+
+#content #designer #confirm-dialog .warn span
+{
+  padding-left: 18px;
+}
+
+#content #designer #confirm-dialog .warn
+{
+  background-image: url( ../../img/ico/exclamation-button.png );
+  background-position: 0px 2px;
+  font-weight: bold;
+}
+
+#content #designer #confirm-dialog #confirm-edit span {
+  background-image: url( ../../img/ico/tick.png );
+}
+
+#content #designer #confirm-dialog #confirm-colls {
+  margin-top: 15px;
+  margin-bottom: 15px;
+}
+
+
+#content #designer #sample-docs .note span
+{
+  padding-left: 18px;
+}
+
+#content #designer .information
+{
+  display: block;
+  height: 40px;
+}
+
+#content #designer #field-bottom fieldset.common
+{
+  margin-top: 10px;
+}
+
+#content #designer #field-bottom fieldset legend,
+#content #designer #field-bottom .optional.expanded legend
+{
+  display: block;
+  margin-left: 10px;
+  padding: 0px 5px;
+}
+
+#content #designer #field-bottom fieldset legend label
+{
+  margin-top: 0;
+}
+
+#content #designer #query-form select
+{
+  width: 150px;
+}
+
+#content #designer #selected-hit {
+  padding: 5px;
+}
+
+#content #designer tr.odd
+{
+  background-color: #f8f8f8;
+}
+
+#content #designer .help
+{
+  font-weight: normal !important;
+  position: relative;
+}
+
+#content #designer .help div.help
+{
+  z-index: 200;
+  background-color: #FCF0AD;
+  border: 1px solid #f0f0f0;
+  box-shadow: 5px 5px 10px #c0c0c0;
+  -moz-box-shadow: 5px 5px 10px #c0c0c0;
+  -webkit-box-shadow: 5px 5px 10px #c0c0c0;
+  position: absolute;
+  left: 0px;
+  top: 0px;
+  padding: 6px;
+  width: 340px;
+}
+
+#content #designer .help div.help-wide
+{
+  z-index: 200;
+  background-color: #FCF0AD;
+  border: 1px solid #f0f0f0;
+  box-shadow: 5px 5px 10px #c0c0c0;
+  -moz-box-shadow: 5px 5px 10px #c0c0c0;
+  -webkit-box-shadow: 5px 5px 10px #c0c0c0;
+  position: absolute;
+  left: -90px;
+  top: 0px;
+  padding: 6px;
+  width: 420px;
+}
+
+#content #designer .help div.help-wider
+{
+  z-index: 200;
+  background-color: #FCF0AD;
+  border: 1px solid #f0f0f0;
+  box-shadow: 5px 5px 10px #c0c0c0;
+  -moz-box-shadow: 5px 5px 10px #c0c0c0;
+  -webkit-box-shadow: 5px 5px 10px #c0c0c0;
+  position: absolute;
+  left: 0px;
+  top: 0px;
+  padding: 6px;
+  width: 430px;
+}
+
+#content #designer .help div.help-top
+{
+  z-index: 200;
+  background-color: #FCF0AD;
+  border: 1px solid #f0f0f0;
+  box-shadow: 5px 5px 10px #c0c0c0;
+  -moz-box-shadow: 5px 5px 10px #c0c0c0;
+  -webkit-box-shadow: 5px 5px 10px #c0c0c0;
+  position: absolute;
+  left: 0px;
+  top: -200px;
+  padding: 6px;
+  width: 420px;
+}
+
+#content #designer .help-anchor
+{
+  margin-top: 7px;
+  margin-left: 18px;
+  margin-bottom: 10px;
+}
+
+#content #designer .help-anchor a
+{
+  color: #003eff;
+  text-decoration-line: underline;
+  text-decoration-color: #003eff;
+}
+
+#content #designer .help-ico {
+  margin-left: 3px;
+  margin-top: 3px;
+}
+
+#content #designer #analysis .verbose_output
+{
+  padding: 3px;
+  float: left;
+  margin-left: 260px;
+}
+
+#content #designer #analysis .verbose_output a
+{
+  background-image: url( ../../img/ico/ui-check-box-uncheck.png );
+  background-position: 0 50%;
+  color: #4D4D4D;
+  display: block;
+  padding-left: 21px;
+}
+
+#content #designer #analysis .verbose_output.active a
+{
+  background-image: url( ../../img/ico/ui-check-box.png );
+}
+
+#content #designer #file-node .error
+{
+  background-image: url( ../../img/ico/cross-button.png );
+  color: #c00;
+  font-weight: bold;
+}
+
+#content #designer #file-node .error span
+{
+  margin-left: 20px;
+}
+
+#content #designer .field-top-form {
+  margin-bottom: 7px;
+  width: 300px;
+}
+
+#content #designer .field-top-form label {
+  text-align: right;
+  margin-right: 5px;
+  width: 150px;
+}
+
+#content #designer .field-top-form select
+{
+  display: block;
+  width: 150px;
+}
+
+#content #designer .warn-dialog
+{
+  z-index: 200;
+  background-color: #f0f0f0;
+  border: 1px solid #a0a0a0;
+  box-shadow: 5px 5px 10px #c0c0c0;
+  -moz-box-shadow: 5px 5px 10px #c0c0c0;
+  -webkit-box-shadow: 5px 5px 10px #c0c0c0;
+  position: absolute;
+  left: 350px;
+  top: 95px;
+  padding: 20px;
+  width: 450px;
+}
+
+#content #designer #warn-dialog #warn-dialog-buttons {
+  float: right;
+}
+
+#content #designer #warn-dialog .warn-button {
+  margin-right: 15px;
+}
+
+#content #designer #warn-dialog .warn-button span
+{
+  background-image: url( ../../img/ico/tick.png );
+}
+
+#content #designer .error-dialog
+{
+  z-index: 200;
+  background-color: #f0f0f0;
+  border: 1px solid #c00;
+  box-shadow: 5px 5px 10px #c0c0c0;
+  -moz-box-shadow: 5px 5px 10px #c0c0c0;
+  -webkit-box-shadow: 5px 5px 10px #c0c0c0;
+  position: absolute;
+  left: 350px;
+  top: 95px;
+  padding: 20px;
+  width: 450px;
+}
+
+#content #designer #error-dialog #error-dialog-buttons {
+  float: right;
+}
+
+#content #designer #error-dialog .error-button {
+  margin-right: 15px;
+}
+
+#content #designer #error-dialog .error-button span
+{
+  background-image: url( ../../img/ico/tick.png );
+}
+
+#content #designer #error-dialog .reload-error-button {
+  margin-right: 15px;
+}
+
+#content #designer #error-dialog .reload-error-button span
+{
+  background-image: url( ../../img/ico/arrow-circle.png );
+}
+
+#content #designer #error-dialog-note {
+  color: #c00;
+  font-weight: bold;
+  margin-bottom: 15px;
+}
+
+#content #designer #error-dialog-details {
+  min-height: 80px;
+  margin-bottom: 15px;
+}
+
+#content #designer #edit-analysis-json-link {
+  margin-top: 10px;
+  float: right;
+}
+
+#content #designer #edit-analysis-json-link a {
+  cursor: pointer;
+  color: #003eff;
+  text-decoration-line: underline;
+  text-decoration-color: #003eff;
+}
+
+#content #designer #tree-filters {
+  font-size: 11px;
+  max-width: 400px;
+  overflow: hidden;
+  margin-top: 2px;
+  margin-bottom: 6px;
+}
+
+#content #designer #tree-filters-cat {
+  max-width: 70px;
+  font-size: 11px;
+}
+
+#content #designer #tree-filters-options {
+  max-width: 90px;
+  font-size: 11px;
+}
+
+#content #designer #copy_schema {
+  width: 200px;
+}
+
+#content #designer #currentSchemaLabel {
+  font-weight: bold;
+  margin-top: 2px;
+}
+
+#content #designer #treeFilterFeatureEnabledSpan {
+  display: inline;
+  font-size: 11px;
+}
+
+#content #designer #treeFilterFeatureEnabledSpan input {
+  padding: 0px;
+  margin-top: 4px;
+}
+
+#content #designer .container-node {
+  max-height: 540px;
+  overflow: auto;
+}
+
+
+#content #designer .prop {
+  border-bottom: 1px solid #dddddd;
+  padding: 3px;
+  text-align: left;
+  min-width: 80px;
+  word-wrap: break-word;
+  max-width: 140px;
+}
+
+#content #designer .prop-flag {
+  border-bottom: 1px solid #dddddd;
+  padding: 3px;
+  text-align: center;
+}
+
+#content #designer .prop-hdr {
+  border-bottom: 1px solid #dddddd;
+  padding: 3px;
+  font-weight: bold;
+  text-align: left;
+  min-width: 80px;
+  word-wrap: break-word;
+  max-width: 140px;
+}
+
+#content #designer .prop-hdr-flag {
+  border-bottom: 1px solid #dddddd;
+  padding: 3px;
+  font-weight: bold;
+  text-align: center;
+  min-width: 65px;
+}
+
+#content #designer #editDocumentsToolbar { float: right; margin-top:0px; margin-left:20px; }
+#content #designer #editDocuments span { background-image: url( ../../img/ico/pencil.png ); }
+
+#content #designer .help div.help-small
+{
+  z-index: 200;
+  background-color: #FCF0AD;
+  border: 1px solid #f0f0f0;
+  box-shadow: 5px 5px 10px #c0c0c0;
+  -moz-box-shadow: 5px 5px 10px #c0c0c0;
+  -webkit-box-shadow: 5px 5px 10px #c0c0c0;
+  position: absolute;
+  left: -120px;
+  top: 0px;
+  padding: 6px;
+  width: 250px;
+}
+
+
diff --git a/solr/webapp/web/index.html b/solr/webapp/web/index.html
index 818cdd4..e75f65e 100644
--- a/solr/webapp/web/index.html
+++ b/solr/webapp/web/index.html
@@ -26,6 +26,7 @@ limitations under the License.
   <link rel="stylesheet" type="text/css" href="css/angular/angular-csp.css?_=${version}">
   <link rel="stylesheet" type="text/css" href="css/angular/common.css?_=${version}">
   <link rel="stylesheet" type="text/css" href="css/angular/analysis.css?_=${version}">
+  <link rel="stylesheet" type="text/css" href="css/angular/schema-designer.css?_=${version}">
   <link rel="stylesheet" type="text/css" href="css/angular/cloud.css?_=${version}">
   <link rel="stylesheet" type="text/css" href="css/angular/cores.css?_=${version}">
   <link rel="stylesheet" type="text/css" href="css/angular/collections.css?_=${version}">
@@ -88,6 +89,7 @@ limitations under the License.
   <script src="js/angular/controllers/plugins.js"></script>
   <script src="js/angular/controllers/replication.js"></script>
   <script src="js/angular/controllers/schema.js"></script>
+  <script src="js/angular/controllers/schema-designer.js"></script>
   <script src="js/angular/controllers/segments.js"></script>
   <script src="js/angular/controllers/unknown.js"></script>
   <script src="js/angular/controllers/cluster-suggestions.js"></script>
@@ -167,6 +169,8 @@ limitations under the License.
                 </ul>
               </li>
 
+              <li ng-show="isCloudEnabled && isSchemaDesignerEnabled" id="schema-designer" class="global" ng-class="{active:page=='schema-designer'}"><p><a href="#/~schema-designer">Schema Designer</a></p></li>
+
               <li ng-show="isCloudEnabled" id="collections" class="global" ng-class="{active:page=='collections'}"><p><a href="#/~collections">Collections</a></p></li>
               <li ng-hide="isCloudEnabled" id="cores" class="global" ng-class="{active:page=='cores'}"><p><a href="#/~cores">Core Admin</a></p></li>
 
diff --git a/solr/webapp/web/js/angular/app.js b/solr/webapp/web/js/angular/app.js
index d2db740..80e3aab 100644
--- a/solr/webapp/web/js/angular/app.js
+++ b/solr/webapp/web/js/angular/app.js
@@ -188,6 +188,10 @@ solrAdminApp.config([
         templateUrl: 'partials/segments.html',
         controller: 'SegmentsController'
       }).
+      when('/~schema-designer', {
+        templateUrl: 'partials/schema-designer.html',
+        controller: 'SchemaDesignerController'
+      }).
       otherwise({
         templateUrl: 'partials/unknown.html',
         controller: 'UnknownController'
@@ -383,7 +387,9 @@ solrAdminApp.config([
     if (sessionStorage.getItem("auth.header")) {
       config.headers['Authorization'] = sessionStorage.getItem("auth.header");
     }
-    config.timeout = 10000;
+    if (!config.timeout) {
+      config.timeout = 10000;
+    }
     return config || $q.when(config);
   };
 
@@ -445,7 +451,11 @@ solrAdminApp.config([
         $location.path('/login');
       }
     } else {
-      $rootScope.exceptions[rejection.config.url] = rejection.data.error;
+      // schema designer prefers to handle errors itselft
+      var isHandledBySchemaDesigner = rejection.config.url && rejection.config.url.startsWith("/api/schema-designer/");
+      if (!isHandledBySchemaDesigner) {
+        $rootScope.exceptions[rejection.config.url] = rejection.data.error;
+      }
     }
     return $q.reject(rejection);
   };
@@ -473,7 +483,7 @@ solrAdminApp.config([
     };
 });
 
-solrAdminApp.controller('MainController', function($scope, $route, $rootScope, $location, Cores, Collections, System, Ping, Constants) {
+solrAdminApp.controller('MainController', function($scope, $route, $rootScope, $location, Cores, Collections, System, Ping, Constants, SchemaDesigner) {
 
   $rootScope.exceptions={};
 
@@ -495,6 +505,9 @@ solrAdminApp.controller('MainController', function($scope, $route, $rootScope, $
       delete $scope.currentCore;
       for (key in data.status) {
         var core = data.status[key];
+        if (core.name.startsWith("._designer_")) {
+          continue;
+        }
         $scope.cores.push(core);
         if ((!$scope.isSolrCloud || pageType == Constants.IS_CORE_PAGE) && core.name == currentCoreName) {
             $scope.currentCore = core;
@@ -504,6 +517,7 @@ solrAdminApp.controller('MainController', function($scope, $route, $rootScope, $
       $scope.initFailures = data.initFailures;
     });
 
+    $scope.isSchemaDesignerEnabled = true;
     System.get(function(data) {
       $scope.isCloudEnabled = data.mode.match( /solrcloud/i );
 
@@ -526,6 +540,9 @@ solrAdminApp.controller('MainController', function($scope, $route, $rootScope, $
             }
             $scope.collections = [];
             for (key in cdata.collections) {
+              if (cdata.collections[key].startsWith("._designer_")) {
+                continue; // ignore temp designer collections
+              }
               var collection = {name: cdata.collections[key], type: 'collection'};
               $scope.collections.push(collection);
               if (pageType == Constants.IS_COLLECTION_PAGE && collection.name == currentCollectionName) {
@@ -538,6 +555,14 @@ solrAdminApp.controller('MainController', function($scope, $route, $rootScope, $
               $scope.aliases_and_collections = $scope.aliases_and_collections.concat({name:'-----'});
             }
             $scope.aliases_and_collections = $scope.aliases_and_collections.concat($scope.collections);
+
+            SchemaDesigner.get({path: "configs"}, function (ignore) {
+              // no-op, just checking if we have access to this path
+            }, function(e) {
+              if (e.status === 403) {
+                $scope.isSchemaDesignerEnabled = false;
+              }
+            });
           });
         });
       }
diff --git a/solr/webapp/web/js/angular/controllers/cloud.js b/solr/webapp/web/js/angular/controllers/cloud.js
index 27be1ad..0566f36 100644
--- a/solr/webapp/web/js/angular/controllers/cloud.js
+++ b/solr/webapp/web/js/angular/controllers/cloud.js
@@ -562,14 +562,12 @@ var treeSubController = function($scope, Zookeeper) {
         var path = decodeURIComponent(link.replace(/.*[\\?&]path=([^&#]*).*/, "$1"));
         Zookeeper.detail({path: path}, function(data) {
             $scope.znode = data.znode;
-            var path = data.znode.path.split( '.' );
-            if(path.length >1) {
-              $scope.lang = path.pop();
+            if (data.znode.path.endsWith("/managed-schema") || data.znode.path.endsWith(".xml.bak")) {
+              $scope.lang = "xml";
             } else {
               var lastPathElement = data.znode.path.split( '/' ).pop();
-              if (lastPathElement == "managed-schema") {
-                  $scope.lang = "xml";
-              }
+              var lastDotAt = lastPathElement ? lastPathElement.lastIndexOf('.') : -1;
+              $scope.lang = lastDotAt != -1 ? lastPathElement.substring(lastDotAt+1) : "txt";
             }
             $scope.showData = true;
         });
diff --git a/solr/webapp/web/js/angular/controllers/collections.js b/solr/webapp/web/js/angular/controllers/collections.js
index c7ee006..6dbe2d7 100644
--- a/solr/webapp/web/js/angular/controllers/collections.js
+++ b/solr/webapp/web/js/angular/controllers/collections.js
@@ -26,6 +26,9 @@ solrAdminApp.controller('CollectionsController',
           Collections.status(function (data) {
               $scope.collections = [];
               for (var name in data.cluster.collections) {
+                  if (name.startsWith("._designer_")) {
+                      continue;
+                  }
                   var collection = data.cluster.collections[name];
                   collection.name = name;
                   collection.type = 'collection';
diff --git a/solr/webapp/web/js/angular/controllers/schema-designer.js b/solr/webapp/web/js/angular/controllers/schema-designer.js
new file mode 100644
index 0000000..ecc128d
--- /dev/null
+++ b/solr/webapp/web/js/angular/controllers/schema-designer.js
@@ -0,0 +1,1994 @@
+/*
+ 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.
+*/
+
+solrAdminApp.controller('SchemaDesignerController', function ($scope, $timeout, $cookies, $window, Constants, SchemaDesigner, Luke) {
+  $scope.resetMenu("schema-designer", Constants.IS_ROOT_PAGE);
+
+  $scope.onWarning = function (warnMsg, warnDetails) {
+    $scope.updateWorking = false;
+    delete $scope.updateStatusMessage;
+    $scope.apiWarning = warnMsg;
+    $scope.apiWarningDetails = warnDetails;
+  };
+  
+  $scope.onError = function (errorMsg, errorCode, errorDetails) {
+    $scope.updateWorking = false;
+    delete $scope.updateStatusMessage;
+    $scope.designerAPIError = errorMsg;
+    if (errorDetails) {
+      var errorDetailsStr = "";
+      if (errorDetails["error"]) {
+        errorDetailsStr = errorDetails["error"];
+      } else {
+        for (var id in errorDetails) {
+          var msg = errorDetails[id];
+          var at = msg.indexOf("ERROR: ");
+          if (at !== -1) {
+            msg = msg.substring(at+7);
+          }
+          if (!msg.includes(id)) {
+            msg = id+": "+msg;
+          }
+          errorDetailsStr += msg+"\n\n";
+        }
+      }
+      $scope.designerAPIErrorDetails = errorDetailsStr;
+    } else {
+      delete $scope.designerAPIErrorDetails;
+    }
+
+    if (errorCode === 409) {
+      $scope.schemaVersion = -1; // reset to get the latest
+      $scope.isVersionMismatch = true;
+      $scope.errorButton = "Reload Schema";
+    } else if (errorCode < 500) {
+      $scope.isVersionMismatch = false;
+      $scope.errorButton = "OK";
+    } // else 500 errors get the top-level error message
+  };
+
+  $scope.errorHandler = function (e) {
+    var error = e.data && e.data.error ? e.data.error : null;
+    if (error) {
+      $scope.onError(error.msg, error.code, e.data.errorDetails);
+    } else {
+      // when a timeout occurs, the error details are sparse so just give the user a hint that something was off
+      var path = e.config && e.config.url ? e.config.url : "/api/schema-designer";
+      var reloadMsg = "";
+      if (path.includes("/analyze")) {
+        reloadMsg = " Re-try analyzing your sample docs by clicking on 'Analyze Documents' again."
+      }
+      $scope.onError("Request to "+path+" failed!", 408,
+          {"error":"Most likely the request timed out; check server log for more details."+reloadMsg});
+    }
+  };
+
+  $scope.closeWarnDialog = function () {
+    delete $scope.apiWarning;
+    delete $scope.apiWarningDetails;
+  };
+
+  $scope.closeErrorDialog = function () {
+    delete $scope.designerAPIError;
+    delete $scope.designerAPIErrorDetails;
+    if ($scope.isVersionMismatch) {
+      $scope.isVersionMismatch = false;
+      var nodeId = "/";
+      if ($scope.selectedNode) {
+        nodeId = $scope.selectedNode.href;
+      }
+      $scope.doAnalyze(nodeId);
+    }
+  };
+
+  $scope.refresh = function () {
+    $scope.isSchemaDesignerEnabled = true;
+
+    delete $scope.helpId;
+
+    $scope.updateStatusMessage = "";
+    $scope.analysisVerbose = false;
+    $scope.updateWorking = false;
+    $scope.currentSchema = "";
+
+    delete $scope.hasDocsOnServer;
+    delete $scope.queryResultsTree;
+
+    $scope.languages = ["*"];
+    $scope.copyFrom = "_default";
+    delete $scope.sampleMessage;
+    delete $scope.sampleDocuments;
+
+    $scope.schemaVersion = -1;
+    $scope.schemaTree = {};
+    $scope.showSchemaActions = false;
+    $scope.sampleDocIds = [];
+    $scope.isSchemaRoot = false;
+
+    delete $scope.enableNestedDocs;
+    delete $scope.enableDynamicFields;
+    delete $scope.enableFieldGuessing;
+
+    // schema editor
+    $scope.showFieldDetails = false;
+    $scope.selectedNode = null;
+    $scope.selectedUpdated = false;
+    delete $scope.updateStatusMessage;
+
+    // text field analysis
+    $scope.showAnalysis = false;
+    $scope.sampleDocId = null;
+    $scope.indexText = "";
+    $scope.result = {};
+
+    // publish vars
+    delete $scope.newCollection;
+    $scope.reloadOnPublish = "true";
+
+    // query form
+    $scope.query = {q: '*:*', sortBy: 'score', sortDir: 'desc'};
+
+    SchemaDesigner.get({path: "configs"}, function (data) {
+
+      $scope.schemas = [];
+      $scope.publishedSchemas = ["_default"];
+
+      for (var s in data.configSets) {
+        // 1 means published but not editable
+        if (data.configSets[s] !== 1) {
+          $scope.schemas.push(s);
+        }
+
+        // 0 means not published yet (so can't copy from it yet)
+        if (data.configSets[s] > 0) {
+          $scope.publishedSchemas.push(s);
+        }
+      }
+
+      $scope.schemas.sort();
+      $scope.publishedSchemas.sort();
+
+      // if no schemas available to select, open the pop-up immediately
+      if ($scope.schemas.length === 0) {
+        $scope.firstSchemaMessage = true;
+        $scope.showNewSchemaDialog();
+      }
+    }, function(e) {
+      if (e.status === 403) {
+        $scope.isSchemaDesignerEnabled = false;
+        $scope.hideAll();
+      }
+    });
+  };
+
+  $scope.selectNodeInTree = function(nodeId) {
+    nodeId = stripAnchorSuffix(nodeId);
+    if (!nodeId) return;
+
+    var jst = $('#schemaJsTree').jstree(true);
+    if (jst) {
+      var selectedId = null;
+      var selected_node = jst.get_selected();
+      if (selected_node && selected_node.length > 0) {
+        selectedId = selected_node[0];
+      }
+      if (selectedId) {
+        try {
+          jst.deselect_node(selectedId);
+        } catch (err) {
+          // just ignore
+          //console.log("error deselecting "+selectedId);
+        }
+      }
+
+      try {
+        jst.select_node(nodeId, true);
+      } catch (err) {
+        // just ignore, some low-level tree issue
+        //console.log("error selecting "+nodeId);
+      }
+    }
+  };
+
+  $scope.loadFile = function (event) {
+    var t = event.target || event.srcElement || event.currentTarget;
+    if (t && t.text) {
+      $scope.onSelectFileNode("files/" + t.text, true);
+    }
+  };
+
+  $scope.confirmEditSchema = function () {
+    $scope.showConfirmEditSchema = false;
+    if ($scope.hasDocsOnServer || $scope.published) {
+      $scope.doAnalyze();
+    } else {
+      $scope.sampleMessage = "Please upload or paste some sample documents to analyze for building the '" + $scope.currentSchema + "' schema.";
+    }
+  };
+
+  $scope.cancelEditSchema = function () {
+    $scope.currentSchema = "";
+    $scope.showConfirmEditSchema = false;
+  };
+
+  $scope.loadSchema = function () {
+
+    if (!$scope.currentSchema) {
+      return;
+    }
+
+    $scope.resetSchema();
+    var params = {path: "info", configSet: $scope.currentSchema};
+    SchemaDesigner.get(params, function (data) {
+      $scope.currentSchema = data.configSet;
+      $("#select-schema").trigger("chosen:updated");
+
+      $scope.confirmSchema = data.configSet;
+      $scope.collectionsForConfig = data.collections;
+      $scope.hasDocsOnServer = data.numDocs > 0;
+      $scope.published = data.published;
+      $scope.initDesignerSettingsFromResponse(data);
+      if ($scope.collectionsForConfig && $scope.collectionsForConfig.length > 0) {
+        $scope.showConfirmEditSchema = true;
+      } else {
+        if ($scope.hasDocsOnServer || $scope.published) {
+          $scope.doAnalyze();
+        } else {
+          $scope.sampleMessage = "Please upload or paste some sample documents to build the '" + $scope.currentSchema + "' schema.";
+        }
+      }
+    });
+  };
+
+  $scope.showNewSchemaDialog = function () {
+    $scope.hideAll();
+    $scope.showNewSchema = true;
+    $scope.newSchema = "";
+  };
+
+  $scope.addSchema = function () {
+    $scope.firstSchemaMessage = false;
+    delete $scope.addMessage;
+
+    if (!$scope.newSchema) {
+      $scope.addMessage = "Please provide a schema name!";
+      return;
+    }
+
+    $scope.newSchema = $scope.newSchema.trim();
+    if ($scope.newSchema.length > 50) {
+      $scope.addMessage = "Schema name be 50 characters or less";
+      return;
+    }
+
+    if ($scope.newSchema.indexOf(" ") !== -1 || $scope.newSchema.indexOf("/") !== -1) {
+      $scope.addMessage = "Schema name should not contain spaces or /";
+      return;
+    }
+
+    if ($scope.publishedSchemas.includes($scope.newSchema) || $scope.schemas.includes($scope.newSchema)) {
+      $scope.addMessage = "Schema '" + $scope.newSchema + "' already exists!";
+      return;
+    }
+
+    delete $scope.addMessage;
+    if (!$scope.copyFrom) {
+      $scope.copyFrom = "_default";
+    }
+
+    $scope.resetSchema();
+    $scope.schemas.push($scope.newSchema);
+    $scope.showNewSchema = false;
+    $scope.currentSchema = $scope.newSchema;
+    $scope.sampleMessage = "Please upload or paste some sample documents to analyze for building the '" + $scope.currentSchema + "' schema.";
+
+    SchemaDesigner.post({path: "prep", configSet: $scope.newSchema, copyFrom: $scope.copyFrom}, null, function (data) {
+      $scope.initDesignerSettingsFromResponse(data);
+    }, $scope.errorHandler);
+  };
+
+  $scope.cancelAddSchema = function () {
+    delete $scope.addMessage;
+    delete $scope.sampleMessage;
+
+    $scope.showNewSchema = false
+  };
+
+  $scope.hideAll = function () {
+    delete $scope.helpId;
+    $scope.showPublish = false;
+    $scope.showDiff = false;
+    $scope.showNewSchema = false;
+    $scope.showAddField = false;
+    $scope.showAddDynamicField = false;
+    $scope.showAddCopyField = false;
+    $scope.showAnalysis = false;
+    // add more dialogs here
+  };
+
+  $scope.showHelp = function (id) {
+    if ($scope.helpId && ($scope.helpId === id || id === '')) {
+      delete $scope.helpId;
+    } else {
+      $scope.helpId = id;
+    }
+  };
+
+  $scope.hideData = function () {
+    $scope.showData = false;
+  };
+
+  $scope.rootChanged = function () {
+    $scope.selectedUpdated = true;
+    $scope.selectedType = "Schema";
+  };
+
+  $scope.updateUniqueKey = function () {
+    delete $scope.schemaRootMessage;
+    var jst = $('#schemaJsTree').jstree();
+    if (jst) {
+      var node = jst.get_node("field/" + $scope.updateUniqueKeyField);
+      if (node && node.a_attr) {
+        var attrs = node.a_attr;
+        if (attrs.multiValued || attrs.tokenized || !attrs.stored || !attrs.indexed) {
+          $scope.schemaRootMessage = "Field '" + $scope.updateUniqueKeyField +
+              "' cannot be used as the uniqueKey field! Must be single-valued, stored, indexed, and not tokenized.";
+          $scope.updateUniqueKeyField = $scope.uniqueKeyField;
+          return;
+        }
+      }
+    }
+    $scope.uniqueKeyField = $scope.updateUniqueKeyField;
+    $scope.selectedUpdated = true;
+    $scope.selectedType = "Schema";
+  };
+
+  $scope.resetSchema = function () {
+    $scope.hideAll();
+    $scope.analysisVerbose = false;
+    $scope.showSchemaActions = false;
+    $scope.showAnalysis = false;
+    $scope.showFieldDetails = false;
+    $scope.hasDocsOnServer = false;
+    $scope.query = {q: '*:*', sortBy: 'score', sortDir: 'desc'};
+    $scope.schemaVersion = -1;
+
+    $scope.updateWorking = false;
+    $scope.isVersionMismatch = false;
+    delete $scope.updateStatusMessage;
+    delete $scope.designerAPIError;
+    delete $scope.designerAPIErrorDetails;
+    delete $scope.selectedFacets;
+    delete $scope.sampleDocuments;
+    delete $scope.selectedNode;
+    delete $scope.queryResultsTree;
+  };
+
+  $scope.onSchemaUpdated = function (schema, data, nodeId) {
+    $scope.hasDocsOnServer = data.numDocs && data.numDocs > 0;
+    $scope.uniqueKeyField = data.uniqueKeyField;
+    $scope.updateUniqueKeyField = $scope.uniqueKeyField;
+    $scope.initDesignerSettingsFromResponse(data);
+
+    var fieldTypes = fieldTypesToTree(data.fieldTypes);
+    var files = filesToTree(data.files);
+
+    var rootChildren = [];
+    $scope.fieldsSrc = fieldsToTree(data.fields);
+    $scope.fieldsNode = {
+      "id": "fields",
+      "text": "Fields",
+      "state": {"opened": true},
+      "a_attr": {"href": "fields"},
+      "children": $scope.fieldsSrc
+    };
+    rootChildren.push($scope.fieldsNode);
+
+    if ($scope.enableDynamicFields === "true") {
+      $scope.dynamicFieldsSrc = fieldsToTree(data.dynamicFields);
+      $scope.dynamicFieldsNode = {
+        "id": "dynamicFields",
+        "text": "Dynamic Fields",
+        "a_attr": {"href": "dynamicFields"},
+        "children": $scope.dynamicFieldsSrc
+      };
+      rootChildren.push($scope.dynamicFieldsNode);
+    } else {
+      delete $scope.dynamicFieldsNode;
+      delete $scope.dynamicFieldsSrc;
+    }
+
+    rootChildren.push({"id":"fieldTypes", "text": "Field Types", "a_attr": {"href": "fieldTypes"}, "children": fieldTypes});
+    rootChildren.push(files);
+
+    var tree = [{"id":"/", "text": schema, "a_attr": {"href": "/"}, "state":{"opened": true}, "children": rootChildren}];
+
+    $scope.fields = data.fields;
+    $scope.fieldNames = data.fields.map(f => f.name).sort();
+    $scope.possibleIdFields = data.fields.filter(f => f.indexed && f.stored && !f.tokenized).map(f => f.name).sort();
+    $scope.sortableFields = data.fields.filter(f => (f.indexed || f.docValues) && !f.multiValued && !f.tokenized).map(f => f.name).sort();
+    $scope.sortableFields.push("score");
+
+    $scope.facetFields = data.fields.filter(f => (f.indexed || f.docValues) && !f.tokenized && f.name !== '_version_').map(f => f.name).sort();
+    $scope.hlFields = data.fields.filter(f => f.stored && f.tokenized).map(f => f.name).sort();
+
+    $scope.schemaVersion = data.schemaVersion;
+    $scope.currentSchema = data.configSet;
+    $scope.fieldTypes = fieldTypes;
+    $scope.core = data.core;
+    $scope.schemaTree = tree;
+    $scope.refreshTree();
+
+    $scope.collectionsForConfig = data.collectionsForConfig;
+
+    if (data.docIds) {
+      $scope.sampleDocIds = data.docIds;
+    }
+
+    // Load the Luke schema
+    Luke.schema({core: data.core}, function (schema) {
+      Luke.raw({core: data.core}, function (index) {
+        $scope.luke = mergeIndexAndSchemaData(index, schema.schema);
+        $scope.types = Object.keys(schema.schema.types);
+        $scope.showSchemaActions = true;
+        if (!nodeId) {
+          nodeId = "/";
+        }
+        // re-apply the filters on the updated schema
+        $scope.applyTreeFilterOption();
+        $scope.onSelectSchemaTreeNode(nodeId);
+
+        $scope.updateWorking = false;
+
+        if (data.updateError != null) {
+          $scope.onError(data.updateError, data.updateErrorCode, data.errorDetails);
+        } else {
+          if ($scope.selectedUpdated) {
+            $scope.selectedUpdated = false;
+            $scope.updateStatusMessage = "Changes applied successfully.";
+            var waitMs = 3000;
+            if (data.rebuild) {
+              $scope.updateStatusMessage += " Did full re-index of sample docs due to incompatible update.";
+              waitMs = 5000; // longer message, more time to read
+            }
+
+            if (data.analysisError) {
+              var updateType = data["updateType"];
+              var updatedObject = data[updateType];
+              var updatedName = updatedObject && updatedObject.name ? updatedObject.name : "";
+              var warnMsg = "Changes to "+updateType+" "+updatedName+" applied, but required the temp collection to be deleted " +
+                  "and re-created due to an incompatible Lucene change, see details below.";
+              $scope.onWarning(warnMsg, data.analysisError);
+            } else {
+              $timeout(function () { delete $scope.updateStatusMessage; }, waitMs);
+            }
+          } else {
+            var source = data.sampleSource;
+            if (source) {
+              if (source === "paste") {
+                source = "pasted sample"
+              } else if (source === "blob") {
+                source = "previous upload stored on the server"
+              }
+              if (data.numDocs > 0) {
+                $scope.updateStatusMessage = "Analyzed "+data.numDocs+" docs from "+source;
+              } else {
+                $scope.updateStatusMessage = "Schema '"+$scope.currentSchema+"' loaded.";
+              }
+            }
+            $timeout(function () {
+              delete $scope.updateStatusMessage;
+            }, 5000);
+          }
+        }
+
+        // re-fire the current query to reflect the updated schema
+        $scope.doQuery();
+        $scope.selectNodeInTree(nodeId);
+      });
+    });
+  };
+
+  $scope.toggleAddField = function (type) {
+    if ($scope.showAddField) {
+      $scope.hideAll();
+    } else {
+      $scope.hideAll();
+      $scope.showAddField = true;
+      $scope.adding = type;
+
+      $scope.newField = {
+        stored: "true",
+        indexed: "true",
+        uninvertible: "true",
+        docValues: "true"
+      }
+
+      if (type === "field") {
+        $scope.newField.type = "string";
+      }
+
+      delete $scope.addErrors;
+    }
+  };
+
+  function applyConstraintsOnField(f) {
+
+    if (!f.docValues) {
+      delete f.useDocValuesAsStored;
+    }
+
+    if (!f.docValues && !f.uninvertible) {
+      delete f.sortMissingLast; // remove this setting if no docValues / uninvertible
+    }
+
+    if (f.indexed) {
+      if (f.omitTermFreqAndPositions && !f.omitPositions) {
+        delete f.omitPositions; // :shrug ~ see SchemaField ln 295
+      }
+      if (!f.termVectors) {
+        delete f.termPositions;
+        delete f.termOffsets;
+        delete f.termPayloads;
+      }
+    } else {
+      // if not indexed, a bunch of fields are false
+      f.tokenized = false;
+      f.uninvertible = false;
+
+      // drop these from the request
+      delete f.termVectors;
+      delete f.termPositions;
+      delete f.termOffsets;
+      delete f.termPayloads;
+      delete f.omitNorms;
+      delete f.omitPositions;
+      delete f.omitTermFreqAndPositions;
+      delete f.storeOffsetsWithPositions;
+    }
+
+    return f;
+  }
+
+  $scope.addField = function () {
+
+    // validate the form input
+    $scope.addErrors = [];
+    if (!$scope.newField.name) {
+      $scope.addErrors.push($scope.adding + " name is required!");
+    }
+
+    if ($scope.newField.name.indexOf(" ") != -1) {
+      $scope.addErrors.push($scope.adding + " name should not have whitespace");
+    }
+
+    var command = "add-field-type";
+    if ("field" === $scope.adding) {
+
+      if ($scope.fieldNames.includes($scope.newField.name)) {
+        $scope.addErrors.push("Field '" + $scope.newField.name + "' already exists!");
+        return;
+      }
+
+      // TODO: is this the correct logic for detecting dynamic? Probably good enough for the designer
+      var isDynamic = $scope.newField.name.startsWith("*") || $scope.newField.name.endsWith("*");
+      if (isDynamic) {
+        if ($scope.luke && $scope.luke.dynamic_fields[$scope.newField.name]) {
+          $scope.addErrors.push("dynamic field '" + $scope.newField.name + "' already exists!");
+        }
+      } else {
+        if ($scope.luke && $scope.luke.fields[$scope.newField.name]) {
+          $scope.addErrors.push("field '" + $scope.newField.name + "' already exists!");
+        }
+      }
+
+      if (!$scope.newField.type) {
+        $scope.addErrors.push("field type is required!");
+      }
+
+      command = isDynamic ? "add-dynamic-field" : "add-field";
+    } else if ("type" === $scope.adding) {
+      if ($scope.types.includes($scope.newField.name)) {
+        $scope.addErrors.push("Type '" + $scope.newField.name + "' already exists!");
+      }
+
+      if (!$scope.newField.class) {
+        $scope.addErrors.push("class is required when creating a new field type!");
+      }
+    }
+
+    var addData = {};
+    addData[command] = applyConstraintsOnField($scope.newField);
+    if ($scope.textAnalysisJson) {
+      var text = $scope.textAnalysisJson.trim();
+      if (text.length > 0) {
+        text = text.replace(/\s+/g, ' ');
+        if (!text.startsWith("{")) {
+          text = "{ " + text + " }";
+        }
+        try {
+          var textJson = JSON.parse(text);
+          if (textJson.analyzer) {
+            addData[command].analyzer = textJson.analyzer;
+          } else {
+            if (!textJson.indexAnalyzer || !textJson.queryAnalyzer) {
+              $scope.addErrors.push("Text analysis JSON should define either an 'analyzer' or an 'indexAnalyzer' and 'queryAnalyzer'");
+              return;
+            }
+            addData[command].indexAnalyzer = textJson.indexAnalyzer;
+            addData[command].queryAnalyzer = textJson.queryAnalyzer;
+          }
+        } catch (e) {
+          $scope.addErrors.push("Failed to parse analysis as JSON due to: " + e.message +
+              "; expected JSON object with either an 'analyzer' or 'indexAnalyzer' and 'queryAnalyzer'");
+          return;
+        }
+      }
+    }
+
+    if ($scope.addErrors.length > 0) {
+      return;
+    }
+    delete $scope.addErrors; // no errors!
+
+    SchemaDesigner.post({
+      path: "add",
+      configSet: $scope.currentSchema,
+      schemaVersion: $scope.schemaVersion
+    }, addData, function (data) {
+      if (data.errors) {
+        $scope.addErrors = data.errors[0].errorMessages;
+        if (typeof $scope.addErrors === "string") {
+          $scope.addErrors = [$scope.addErrors];
+        }
+      } else {
+        delete $scope.textAnalysisJson;
+        $scope.added = true;
+        $timeout(function () {
+          $scope.showAddField = false;
+          $scope.added = false;
+          var nodeId = "/";
+          if ("field" === $scope.adding) {
+            nodeId = "field/" + data[command];
+          } else if ("type" === $scope.adding) {
+            nodeId = "type/" + data[command];
+          }
+          $scope.onSchemaUpdated(data.configSet, data, nodeId);
+        }, 500);
+      }
+    }, $scope.errorHandler);
+  }
+
+  function toSortedNameAndTypeList(fields, typeAttr) {
+    var list = [];
+    var keys = Object.keys(fields);
+    for (var f in keys) {
+      var field = fields[keys[f]];
+      var type = field[typeAttr];
+      if (type) {
+        list.push(field.name + ": "+type);
+      } else {
+        list.push(field.name);
+      }
+    }
+    return list.sort();
+  }
+
+  function toSortedFieldList(fields) {
+    var list = [];
+    var keys = Object.keys(fields);
+    for (var f in keys) {
+      list.push(fields[keys[f]]);
+    }
+    return list.sort((a, b) => (a.name > b.name) ? 1 : -1);
+  }
+
+  $scope.toggleDiff = function (event) {
+    if ($scope.showDiff) {
+      // toggle, close dialog
+      $scope.showDiff = false;
+      return;
+    }
+
+    if (event) {
+      var t = event.target || event.currentTarget;
+      var leftPos = t.getBoundingClientRect().left - 600;
+      if (leftPos < 0) leftPos = 0;
+      $('#show-diff-dialog').css({left: leftPos});
+    }
+
+    SchemaDesigner.get({ path: "diff", configSet: $scope.currentSchema }, function (data) {
+      var diff = data.diff;
+
+      var dynamicFields = diff.dynamicFields;
+      var enableDynamicFields = data.enableDynamicFields !== null ? data.enableDynamicFields : true;
+      if (!enableDynamicFields) {
+        dynamicFields = null;
+      }
+
+      $scope.diffSource = data["diff-source"];
+      $scope.schemaDiff = {
+        "fieldsDiff": diff.fields,
+        "addedFields": [],
+        "removedFields": [],
+        "fieldTypesDiff": diff.fieldTypes,
+        "removedTypes": [],
+        "dynamicFieldsDiff": dynamicFields,
+        "copyFieldsDiff": diff.copyFields
+      }
+      if (diff.fields && diff.fields.added) {
+        $scope.schemaDiff.addedFields = toSortedFieldList(diff.fields.added);
+      }
+      if (diff.fields && diff.fields.removed) {
+        $scope.schemaDiff.removedFields = toSortedNameAndTypeList(diff.fields.removed, "type");
+      }
+      if (diff.fieldTypes && diff.fieldTypes.removed) {
+        $scope.schemaDiff.removedTypes = toSortedNameAndTypeList(diff.fieldTypes.removed, "class");
+      }
+
+      $scope.schemaDiffExists = !(diff.fields == null && diff.fieldTypes == null && dynamicFields == null && diff.copyFields == null);
+      $scope.showDiff = true;
+    });
+  }
+
+  $scope.togglePublish = function (event) {
+    if (event) {
+      var t = event.target || event.currentTarget;
+      var leftPos = t.getBoundingClientRect().left - 515;
+      if (leftPos < 0) leftPos = 0;
+      $('#publish-dialog').css({left: leftPos});
+    }
+
+    $scope.showDiff = false;
+    $scope.showPublish = !$scope.showPublish;
+    delete $scope.publishErrors;
+
+    $scope.disableDesigner = "false";
+
+    if ($scope.showPublish && !$scope.newCollection) {
+      $scope.newCollection = {numShards: 1, replicationFactor: 1, indexToCollection: "true"};
+    }
+  };
+
+  $scope.toggleAddCopyField = function () {
+    if ($scope.showAddCopyField) {
+      $scope.hideAll();
+      $scope.showFieldDetails = true;
+    } else {
+      $scope.hideAll();
+      $scope.showAddCopyField = true;
+      $scope.showFieldDetails = false;
+
+      $scope.copyField = {};
+      delete $scope.addCopyFieldErrors;
+    }
+  }
+  $scope.addCopyField = function () {
+    delete $scope.addCopyFieldErrors;
+    var data = {"add-copy-field": $scope.copyField};
+    SchemaDesigner.post({
+      path: "add",
+      configSet: $scope.currentSchema,
+      schemaVersion: $scope.schemaVersion
+    }, data, function (data) {
+      if (data.errors) {
+        $scope.addCopyFieldErrors = data.errors[0].errorMessages;
+        if (typeof $scope.addCopyFieldErrors === "string") {
+          $scope.addCopyFieldErrors = [$scope.addCopyFieldErrors];
+        }
+      } else {
+        $scope.showAddCopyField = false;
+        // TODO:
+        //$timeout($scope.refresh, 1500);
+      }
+    }, $scope.errorHandler);
+  }
+
+  $scope.toggleAnalyzer = function (analyzer) {
+    analyzer.show = !analyzer.show;
+  }
+
+  $scope.initTypeAnalysisInfo = function (typeName) {
+    $scope.analysis = getAnalysisInfo($scope.luke, {type: true}, typeName);
+    if ($scope.analysis && $scope.analysis.data) {
+      $scope.className = $scope.analysis.data.className
+    }
+    $scope.editAnalysis = "Edit JSON";
+    $scope.showAnalysisJson = false;
+    delete $scope.analysisJsonText;
+  };
+
+  $scope.toggleVerbose = function () {
+    $scope.analysisVerbose = !$scope.analysisVerbose;
+  };
+
+  $scope.updateSampleDocId = function () {
+    $scope.indexText = "";
+    $scope.result = {};
+
+    if (!$scope.selectedNode) {
+      return;
+    }
+
+    var field = $scope.selectedNode.name;
+    var params = {path: "sample"};
+    params.configSet = $scope.currentSchema;
+    params.uniqueKeyField = $scope.uniqueKeyField;
+    params.field = field;
+
+    if ($scope.sampleDocId) {
+      params.docId = $scope.sampleDocId;
+    } // else the server will pick the first doc with a non-empty text value for the desired field
+
+    SchemaDesigner.get(params, function (data) {
+      $scope.sampleDocId = data[$scope.uniqueKeyField];
+      $scope.indexText = data[field];
+      if (data.analysis && data.analysis["field_names"]) {
+        $scope.result = processFieldAnalysisData(data.analysis["field_names"][field]);
+      }
+    });
+  };
+
+  $scope.changeLanguages = function () {
+    $scope.selectedUpdated = true;
+    $scope.selectedType = "Schema";
+  };
+
+  function getType(typeName) {
+    if ($scope.fieldTypes) {
+      for (i in $scope.fieldTypes) {
+        if ($scope.fieldTypes[i].text === typeName) {
+          return $scope.fieldTypes[i];
+        }
+      }
+    }
+    return null;
+  }
+
+  $scope.refreshTree = function() {
+    var jst = $('#schemaJsTree').jstree(true);
+    if (jst) {
+      jst.refresh();
+    }
+  };
+
+  $scope.onSchemaTreeLoaded = function (id) {
+    //console.log(">> on tree loaded");
+  };
+
+  $scope.updateFile = function () {
+    var nodeId = "files/" + $scope.selectedFile;
+    var params = {path: "file", file: $scope.selectedFile, configSet: $scope.currentSchema};
+
+    $scope.updateWorking = true;
+    $scope.updateStatusMessage = "Updating file ...";
+
+    SchemaDesigner.post(params, $scope.fileNodeText, function (data) {
+      if (data.updateFileError) {
+        if (data[$scope.selectedFile]) {
+          $scope.fileNodeText = data[$scope.selectedFile];
+        }
+        $scope.updateFileError = data.updateFileError;
+      } else {
+        delete $scope.updateFileError;
+        $scope.updateStatusMessage = "File '"+$scope.selectedFile+"' updated.";
+        $scope.onSchemaUpdated(data.configSet, data, nodeId);
+      }
+    }, $scope.errorHandler);
+  };
+
+  $scope.onSelectFileNode = function (id, doSelectOnTree) {
+    $scope.selectedFile = id.startsWith("files/") ? id.substring("files/".length) : id;
+
+    var params = {path: "file", file: $scope.selectedFile, configSet: $scope.currentSchema};
+    SchemaDesigner.get(params, function (data) {
+      $scope.fileNodeText = data[$scope.selectedFile];
+      $scope.isLeafNode = false;
+      if (doSelectOnTree) {
+        delete $scope.selectedNode;
+        $scope.isLeafNode = false;
+        $scope.showFieldDetails = true;
+        delete $scope.sampleDocId;
+        $scope.showAnalysis = false;
+        $scope.selectNodeInTree(id);
+      }
+    });
+  };
+  
+  function fieldNodes(src, type) {
+    var nodes = [];
+    for (var c in src) {
+      var childNode = src[c];
+      if (childNode && childNode.a_attr) {
+        var a = childNode.a_attr;
+        var stored = a.stored || (a.docValues && a.useDocValuesAsStored);
+        var obj = {"name":a.name, "indexed":a.indexed, "docValues": a.docValues,
+          "multiValued":a.multiValued, "stored":stored, "tokenized": a.tokenized};
+        if (type === "field" || type === "dynamicField") {
+          obj.type = a.type;
+        } else if (type === "type") {
+          obj.class = a.class;
+        }
+        nodes.push(obj);
+      }
+    }
+    return nodes;
+  }
+
+  function stripAnchorSuffix(id) {
+    if (id && id.endsWith("_anchor")) {
+      id = id.substring(0, id.length - "_anchor".length);
+    }
+    return id;
+  }
+
+  $scope.onSelectSchemaTreeNode = function (id) {
+    id = stripAnchorSuffix(id);
+    $scope.showFieldDetails = false;
+    $scope.isSchemaRoot = false;
+    $scope.isLeafNode = false;
+    delete $scope.containerNodeLabel;
+    delete $scope.containerNode;
+    delete $scope.containerNodes;
+    delete $scope.selectedFile;
+
+    if (id === "/") {
+      $scope.selectedType = "Schema";
+      $scope.selectedNode = null;
+      $scope.isSchemaRoot = true;
+      $scope.isLeafNode = false;
+      $scope.isContainerNode = false;
+      $scope.showFieldDetails = true;
+      delete $scope.sampleDocId;
+      $scope.showAnalysis = false;
+
+      if (!$scope.treeFilter) {
+        $scope.treeFilter = "type";
+        $scope.treeFilterOption = "*";
+        $scope.initTreeFilters();
+      }
+      return;
+    }
+
+    var jst = $('#schemaJsTree').jstree(true);
+    if (!jst) {
+      return;
+    }
+
+    var node = jst.get_node(id);
+    if (!node) {
+      return;
+    }
+
+    if (id === "files") {
+      $scope.selectedNode = null;
+      $scope.isLeafNode = false;
+      return;
+    }
+
+    if (id.indexOf("/") === -1) {
+      $scope.selectedNode = null;
+      $scope.isLeafNode = false;
+      $scope.containerNode = id;
+
+      if (id === "fields") {
+        $scope.containerNodes = fieldNodes($scope.fieldsNode ? $scope.fieldsNode.children : $scope.fieldsSrc, "field");
+      } else if (id === "dynamicFields") {
+        $scope.containerNodes = fieldNodes($scope.dynamicFieldsNode ? $scope.dynamicFieldsNode.children : $scope.dynamicFieldsSrc, "dynamicField");
+      } else if (id === "fieldTypes") {
+        $scope.containerNodes = fieldNodes($scope.fieldTypes, "type");
+      }
+
+      $scope.containerNodeLabel = node.text;
+      $scope.showFieldDetails = true;
+      delete $scope.sampleDocId;
+      $scope.showAnalysis = false;
+      return;
+    }
+
+    if (id.startsWith("files/")) {
+      $scope.selectedNode = null;
+      $scope.isLeafNode = false;
+      delete $scope.sampleDocId;
+      $scope.showAnalysis = false;
+      if (node.children.length === 0) {
+        // file
+        $scope.showFieldDetails = true;
+        $scope.onSelectFileNode(id, false);
+      } else {
+        // folder
+        $scope.showFieldDetails = false;
+        delete $scope.selectedFile;
+      }
+      return;
+    }
+
+    delete $scope.selectedFile;
+    $scope.selectedNode = node["a_attr"]; // all the info we need is in the a_attr object
+    if (!$scope.selectedNode) {
+      // a node in the tree that isn't a field was selected, just ignore
+      return;
+    }
+
+    $scope.selectedNode.fieldType = getType($scope.selectedNode.type);
+    $scope.isLeafNode = true;
+
+    var nodeType = id.substring(0, id.indexOf("/"));
+    var name = null;
+    if (nodeType === "field") {
+      $scope.selectedType = "Field";
+      name = $scope.selectedNode.type;
+    } else if (nodeType === "dynamic") {
+      $scope.selectedType = "Dynamic Field";
+    } else if (nodeType === "type") {
+      $scope.selectedType = "Type";
+      name = $scope.selectedNode.name;
+    }
+
+    if (name) {
+      $scope.initTypeAnalysisInfo(name, "type");
+    }
+
+    // apply some sanity to the checkboxes
+    $scope.selectedNode = applyConstraintsOnField($scope.selectedNode);
+    $scope.showFieldDetails = true;
+
+    if (nodeType === "field" && $scope.selectedNode.tokenized && $scope.selectedNode.stored) {
+      $scope.showAnalysis = true;
+      $scope.updateSampleDocId();
+    } else {
+      $scope.showAnalysis = false;
+      $scope.indexText = "";
+      $scope.result = {};
+    }
+  };
+
+  function addFileNode(dirs, parent, f) {
+    var path = f.split("/");
+    if (path.length === 1) {
+      if (!parent.children) {
+        parent.children = [];
+        dirs.push(parent); // now parent has children, so track in dirs ...
+      }
+      var nodeId = parent.id + "/" + f;
+      parent.children.push({"text": f, "id": nodeId, "a_attr": {"href": nodeId}});
+    } else {
+      // get the parent for this path
+      var parentId = "files/" + path.slice(0, path.length - 1).join("/");
+      var dir = null;
+      for (var d in dirs) {
+        if (dirs[d].id === parentId) {
+          dir = dirs[d];
+          break;
+        }
+      }
+      if (!dir) {
+        dir = {"text": path[0], "id": parentId, "a_attr": {"href": parentId}, "children": []};
+        dirs.push(dir);
+        parent.children.push(dir);
+      }
+
+      // walk down the next level in this path
+      addFileNode(dirs, dir, path.slice(1).join("/"));
+    }
+  }
+
+  // transform a flat list structure into the nested tree structure
+  function filesToTree(files) {
+    var filesNode = {"text": "Files", "a_attr": {"href": "files"}, "id": "files", "children": []};
+    if (files) {
+      var dirs = []; // lookup for known dirs by path since nodes don't keep a ref to their parent node
+      for (var i in files) {
+        // hide the configoverlay.json from the UI
+        if (files[i] === "configoverlay.json") {
+          continue;
+        }
+
+        addFileNode(dirs, filesNode, files[i]);
+      }
+      delete dirs;
+    }
+    return filesNode;
+  }
+
+  function fieldsToTree(fields) {
+    var children = [];
+    if (fields) {
+      for (var i in fields) {
+        var id = "field/" + fields[i].name;
+        fields[i].href = id;
+        var text = fields[i].name;
+        if (fields[i].name === $scope.uniqueKeyField) {
+          text += "*"; // unique key field
+        }
+        children.push({"text": text, "a_attr": fields[i], "id": id});
+      }
+    }
+    return children;
+  }
+
+  function fieldTypesToTree(types) {
+    var children = [];
+    for (var i in types) {
+      var ft = types[i]
+      var id = "type/" + ft.name;
+      ft.href = id;
+      children.push({"text": ft.name, "a_attr": ft, "id": id});
+    }
+    return children;
+  }
+
+  $scope.onSampleDocumentsChanged = function () {
+    $scope.hasDocsOnServer = false; // so the updates get sent on next analyze action
+  };
+
+  $scope.initDesignerSettingsFromResponse = function (data) {
+    $scope.enableDynamicFields = data.enableDynamicFields !== null ? "" + data.enableDynamicFields : "true";
+    $scope.enableFieldGuessing = data.enableFieldGuessing !== null ? "" + data.enableFieldGuessing : "true";
+    $scope.enableNestedDocs = data.enableNestedDocs !== null ? "" + data.enableNestedDocs : "false";
+    $scope.languages = data.languages !== null && data.languages.length > 0 ? data.languages : ["*"];
+    $scope.copyFrom = data.copyFrom !== null ? data.copyFrom : "_default";
+  };
+
+  $scope.doAnalyze = function (nodeId) {
+    delete $scope.sampleMessage;
+
+    var schema = $scope.currentSchema;
+    if (schema) {
+      delete $scope.copyFrom;
+    } else {
+      schema = $scope.newSchema;
+      if (!$scope.copyFrom) {
+        $scope.copyFrom = "_default";
+      }
+    }
+
+    if (!schema) {
+      return;
+    }
+
+    var params = {path: "analyze", configSet: schema};
+    if ($scope.schemaVersion && $scope.schemaVersion !== -1) {
+      params.schemaVersion = $scope.schemaVersion;
+    }
+
+    if ($scope.enableDynamicFields) {
+      params.enableDynamicFields = $scope.enableDynamicFields;
+    }
+    if ($scope.enableFieldGuessing) {
+      params.enableFieldGuessing = $scope.enableFieldGuessing;
+    }
+    if ($scope.enableNestedDocs) {
+      params.enableNestedDocs = $scope.enableNestedDocs;
+    }
+
+    if ($scope.languages && $scope.languages.length > 0) {
+      params.languages = $scope.languages;
+    }
+
+    if ($scope.copyFrom) {
+      params.copyFrom = $scope.copyFrom;
+    }
+
+    $scope.updateWorking = true;
+    if ($scope.selectedUpdated) {
+      $scope.updateStatusMessage = "Applying " + $scope.selectedType + " updates ..."
+    } else {
+      $scope.updateStatusMessage = "Analyzing your sample data, schema will load momentarily ..."
+    }
+
+    if (!nodeId && $scope.selectedNode) {
+      nodeId = $scope.selectedNode.id;
+    }
+
+    // a bit tricky ...
+    // so users can upload a file or paste in docs
+    // if they upload a file containing a small (<15K) of text data, then we'll show it in the textarea
+    // they can change the text and re-analyze too
+    // if no changes or nothing uploaded, the server-side uses the latest sample data stored in the blob store
+    if ($scope.fileUpload) {
+      var file = $scope.fileUpload;
+      var fd = new FormData();
+      fd.append('file', file);
+      SchemaDesigner.upload(params, fd, function (data) {
+        $("#upload-file").val("");
+        delete $scope.fileUpload;
+        $scope.onSchemaUpdated(schema, data, nodeId);
+      }, $scope.errorHandler);
+    } else {
+      // don't need to keep re-posting the same sample if already stored in the blob store
+      var postData = null;
+      if (!$scope.hasDocsOnServer) {
+        postData = $scope.sampleDocuments;
+        if (!postData && !$scope.published) {
+          return;
+        }
+      }
+
+      var respHandler = function (data) {
+        $scope.onSchemaUpdated(schema, data, nodeId);
+      };
+
+      // TODO: need a better approach to detecting the content type from text content
+      var contentType = "text/plain";
+      if (postData != null) {
+        var txt = postData.trim();
+        if ((txt.startsWith("[") && txt.includes("]")) || (txt.startsWith("{") && txt.includes("}"))) {
+          contentType = "application/json"
+        } else if (txt.startsWith("<") || txt.includes("<add>") || txt.includes("<!--")) {
+          contentType = "text/xml";
+        } else {
+          contentType = "application/csv";
+        }
+      }
+
+      if (contentType === "text/xml") {
+        SchemaDesigner.postXml(params, postData, respHandler, $scope.errorHandler);
+      } else if (contentType === "application/csv") {
+        SchemaDesigner.postCsv(params, postData, respHandler, $scope.errorHandler);
+      } else {
+        SchemaDesigner.post(params, postData, respHandler, $scope.errorHandler);
+      }
+    }
+  };
+
+  $scope.onFieldTypeChanged = function () {
+
+    var copyFromType = ["stored", "indexed", "multiValued", "docValues", "useDocValuesAsStored", "tokenized", "uninvertible", "termVectors", "termPositions", "termOffsets",
+      "omitNorms", "omitTermFreqAndPositions", "omitPositions", "storeOffsetsWithPositions"];
+
+    var type = $scope.selectedNode.type
+
+    // when the user updates the selected field's type, we go apply the
+    // new field type's properties to the selected node
+    for (var i in $scope.fieldTypes) {
+      if ($scope.fieldTypes[i].text == type) {
+        var ft = $scope.fieldTypes[i];
+        $scope.selectedNode.fieldType = ft;
+        for (var i in copyFromType) {
+          var x = copyFromType[i];
+          if (ft.a_attr[x] !== null) {
+            $scope.selectedNode[x] = ft.a_attr[x];
+          } else {
+            delete $scope.selectedNode[x];
+          }
+        }
+        $scope.selectedUpdated = true;
+        break;
+      }
+    }
+
+    $scope.selectedNode = applyConstraintsOnField($scope.selectedNode);
+    if ($scope.selectedUpdated) {
+      // for luke analysis, we need the type info here, not the specific field into b/c it just changed.
+      $scope.initTypeAnalysisInfo(type, "type");
+    }
+  };
+
+  $scope.isDisabled = function (dep) {
+    if (!$scope.selectedNode) {
+      return false;
+    }
+
+    if (dep === "termVectors") {
+      // termVectors dependency depends on indexed
+      return !($scope.selectedNode.indexed && $scope.selectedNode.termVectors);
+    }
+
+    if (dep === "not-text" || dep === "docValues") {
+      if ($scope.selectedNode.fieldType && $scope.selectedNode.fieldType.a_attr.class === "solr.TextField") {
+        // no doc values for TextField
+        return true;
+      }
+    }
+
+    return $scope.selectedNode[dep] === false;
+  };
+
+  // this updates checkboxes based on the current settings
+  $scope.markSelectedUpdated = function (event) {
+    delete $scope.updateStatusMessage;
+    $scope.selectedUpdated = true; // enable the update button for this field
+  };
+
+  $scope.deleteSelected = function () {
+    // console.log(">> deleteSelected");
+  };
+
+  $scope.updateSelected = function () {
+    if (!$scope.selectedNode) {
+
+      if ($scope.selectedUpdated) {
+        // some root level property changed ... re-analyze
+        $scope.doAnalyze("/");
+      }
+
+      return;
+    }
+
+    delete $scope.updateSelectedError;
+
+    // make a copy for the PUT
+    var href = $scope.selectedNode.href;
+    var id = $scope.selectedNode.id;
+
+    var putData = JSON.parse(JSON.stringify($scope.selectedNode));
+    if ($scope.selectedType === "Field" && putData.copyDest) {
+      var fields = putData.copyDest.split(",");
+      for (var f in fields) {
+        var name = fields[f].trim();
+        if (!$scope.fieldNames.includes(name)) {
+          $scope.updateSelectedError = "Copy to field '" + name + "' doesn't exist!";
+          return;
+        }
+        if (name === $scope.selectedNode.name) {
+          $scope.updateSelectedError = "Cannot copy a field to itself!";
+          return;
+        }
+      }
+    } else {
+      delete putData.copyDest;
+    }
+
+    delete putData.fieldType;
+    delete putData.href;
+    delete putData.id;
+
+    putData = applyConstraintsOnField(putData);
+
+    if ($scope.analysisJsonText && !$scope.selectedNode.type) {
+      var text = $scope.analysisJsonText.trim();
+      if (text.length > 0) {
+        text = text.replace(/\s+/g, ' ');
+        if (!text.startsWith("{")) {
+          text = "{ " + text + " }";
+        }
+        try {
+          var textJson = JSON.parse(text);
+          if (textJson.analyzer) {
+            putData.analyzer = textJson.analyzer;
+          } else {
+            if (textJson.indexAnalyzer && textJson.queryAnalyzer) {
+              putData.indexAnalyzer = textJson.indexAnalyzer;
+              putData.queryAnalyzer = textJson.queryAnalyzer;
+            }
+          }
+        } catch (e) {
+          $scope.updateSelectedError = "Failed to parse analysis as JSON due to: " + e.message +
+              "; expected JSON object with either an 'analyzer' or 'indexAnalyzer' and 'queryAnalyzer'";
+          return;
+        }
+      }
+    }
+
+    $scope.updateWorking = true;
+    $scope.updateStatusMessage = "Updating " + $scope.selectedType + " ...";
+
+    SchemaDesigner.put({
+      path: "update",
+      configSet: $scope.currentSchema,
+      schemaVersion: $scope.schemaVersion
+    }, putData, function (data) {
+
+      var nodeType = data.updateType;
+      $scope.schemaVersion = data.schemaVersion;
+      $scope.currentSchema = data.configSet;
+      $scope.core = data.core;
+
+      $scope.selectedNode = data[nodeType];
+      $scope.selectedNode.href = href;
+      $scope.selectedNode.id = id;
+
+      var name = nodeType === "field" ? $scope.selectedNode.type : $scope.selectedNode.name;
+      $scope.initTypeAnalysisInfo(name, "type");
+      $scope.showFieldDetails = true;
+
+      if (nodeType === "field" && $scope.selectedNode.tokenized) {
+        $scope.showAnalysis = true;
+        $scope.updateSampleDocId();
+      }
+
+      $scope.onSchemaUpdated($scope.currentSchema, data, href);
+    }, $scope.errorHandler);
+  };
+
+  // TODO: These are copied from analysis.js, so move to a shared location for both vs. duplicating
+  var getShortComponentName = function (longname) {
+    var short = -1 !== longname.indexOf('$')
+        ? longname.split('$')[1]
+        : longname.split('.').pop();
+    return short.match(/[A-Z]/g).join('');
+  };
+
+  var getCaptionsForComponent = function (data) {
+    var captions = [];
+    for (var key in data[0]) {
+      key = key.replace(/.*#/, '');
+      if (key != "match" && key != "positionHistory") {
+        captions.push(key.replace(/.*#/, ''));
+      }
+    }
+    return captions;
+  };
+
+  var getTokensForComponent = function (data) {
+    var tokens = [];
+    var previousPosition = 0;
+    var index = 0;
+    for (var i in data) {
+      var tokenhash = data[i];
+      var positionDifference = tokenhash.position - previousPosition;
+      for (var j = positionDifference; j > 1; j--) {
+        tokens.push({position: tokenhash.position - j + 1, blank: true, index: index++});
+      }
+
+      var token = {position: tokenhash.position, keys: [], index: index++};
+
+      for (key in tokenhash) {
+        if (key == "match" || key == "positionHistory") {
+          //skip, to not display these keys in the UI
+        } else {
+          var tokenInfo = new Object();
+          tokenInfo.name = key;
+          tokenInfo.value = tokenhash[key];
+          if ('text' === key || 'raw_bytes' === key) {
+            if (tokenhash.match) {
+              tokenInfo.extraclass = 'match'; //to highlight matching text strings
+            }
+          }
+          token.keys.push(tokenInfo);
+        }
+      }
+      tokens.push(token);
+      previousPosition = tokenhash.position;
+    }
+    return tokens;
+  };
+
+  var extractComponents = function (data, result, name) {
+    if (data) {
+      result[name] = [];
+      for (var i = 0; i < data.length; i += 2) {
+        var component = {
+          name: data[i],
+          short: getShortComponentName(data[i]),
+          captions: getCaptionsForComponent(data[i + 1]),
+          tokens: getTokensForComponent(data[i + 1])
+        };
+        result[name].push(component);
+      }
+    }
+  };
+
+  var processFieldAnalysisData = function (analysis) {
+    var response = {};
+    extractComponents(analysis.index, response, "index");
+    return response;
+  };
+
+  $scope.doPublish = function () {
+    var params = {
+      path: "publish",
+      configSet: $scope.currentSchema,
+      schemaVersion: $scope.schemaVersion,
+      reloadCollections: $scope.reloadOnPublish,
+      cleanupTemp: true,
+      disableDesigner: $scope.disableDesigner
+    };
+    if ($scope.newCollection && $scope.newCollection.name) {
+      params.newCollection = $scope.newCollection.name;
+      params.numShards = $scope.newCollection.numShards;
+      params.replicationFactor = $scope.newCollection.replicationFactor;
+      params.indexToCollection = $scope.newCollection.indexToCollection;
+    }
+    SchemaDesigner.put(params, null, function (data) {
+      $scope.schemaVersion = data.schemaVersion;
+      $scope.currentSchema = data.configSet;
+
+      delete $scope.selectedNode;
+      $scope.currentSchema = "";
+      delete $scope.newSchema;
+      $scope.showPublish = false;
+      $scope.refresh();
+
+      if (data.newCollection) {
+        $window.location.href = "#/" + data.newCollection + "/collection-overview";
+      }
+    }, $scope.errorHandler);
+  };
+
+  $scope.downloadConfig = function () {
+    location.href = "/api/schema-designer/download/"+$scope.currentSchema+"_configset.zip?wt=raw&configSet=" + $scope.currentSchema;
+  };
+
+  function docsToTree(docs) {
+    var children = [];
+    for (var i in docs) {
+      var id = docs[i][$scope.uniqueKeyField];
+      if (!id) {
+        id = "" + i; // id not in results so use the position in results as the value
+      }
+      var nodeId = "doc/" + id;
+      docs[i].href = nodeId;
+      children.push({"text": id, "a_attr": docs[i], "id": nodeId});
+    }
+    return children;
+  }
+
+  function debugToTree(debugObj) {
+    var children = [];
+    for (var x in debugObj) {
... 1521 lines suppressed ...