You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rj...@apache.org on 2015/03/31 07:22:50 UTC

svn commit: r1670257 [28/39] - in /lucene/dev/branches/lucene6271: ./ dev-tools/ dev-tools/idea/.idea/libraries/ dev-tools/scripts/ lucene/ lucene/analysis/ lucene/analysis/common/ lucene/analysis/common/src/java/org/apache/lucene/analysis/miscellaneou...

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java Tue Mar 31 05:22:40 2015
@@ -26,7 +26,6 @@ import org.apache.solr.analysis.Tokenize
 import org.apache.solr.client.solrj.SolrClient;
 import org.apache.solr.client.solrj.SolrRequest;
 import org.apache.solr.client.solrj.SolrResponse;
-import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
@@ -69,6 +68,7 @@ import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.Iterator;
 import java.util.List;
 import java.util.Map;
 import java.util.Set;
@@ -363,9 +363,10 @@ public final class ManagedIndexSchema ex
     }
 
     @Override
-    public SolrResponse process(SolrClient server) throws SolrServerException, IOException {
+    protected SolrResponse createResponse(SolrClient client) {
       return null;
     }
+
   }
 
 
@@ -386,7 +387,7 @@ public final class ManagedIndexSchema ex
   public ManagedIndexSchema addFields(Collection<SchemaField> newFields,
                                       Map<String, Collection<String>> copyFieldNames,
                                       boolean persist) {
-    ManagedIndexSchema newSchema = null;
+    ManagedIndexSchema newSchema;
     if (isMutable) {
       boolean success = false;
       if (copyFieldNames == null){
@@ -440,11 +441,136 @@ public final class ManagedIndexSchema ex
     return newSchema;
   }
 
+  @Override
+  public ManagedIndexSchema deleteFields(Collection<String> names) {
+    ManagedIndexSchema newSchema;
+    if (isMutable) {
+      newSchema = shallowCopy(true);
+      for (String name : names) {
+        SchemaField field = getFieldOrNull(name); 
+        if (null != field) {
+          String message = "Can't delete field '" + name
+              + "' because it's referred to by at least one copy field directive.";
+          if (newSchema.copyFieldsMap.containsKey(name) || newSchema.isCopyFieldTarget(field)) {
+            throw new SolrException(ErrorCode.BAD_REQUEST, message);
+          }
+          for (int i = 0 ; i < newSchema.dynamicCopyFields.length ; ++i) {
+            DynamicCopy dynamicCopy = newSchema.dynamicCopyFields[i];
+            if (name.equals(dynamicCopy.getRegex())) {
+              throw new SolrException(ErrorCode.BAD_REQUEST, message);
+            }
+          }
+          newSchema.fields.remove(name);
+          newSchema.fieldsWithDefaultValue.remove(field);
+          newSchema.requiredFields.remove(field);
+        } else {
+          String msg = "The field '" + name + "' is not present in this schema, and so cannot be deleted.";
+          throw new SolrException(ErrorCode.BAD_REQUEST, msg);
+        }
+      }
+      // Run the callbacks on SchemaAware now that everything else is done
+      for (SchemaAware aware : newSchema.schemaAware) {
+        aware.inform(newSchema);
+      }
+      newSchema.refreshAnalyzers();
+    } else {
+      String msg = "This ManagedIndexSchema is not mutable.";
+      log.error(msg);
+      throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+    }
+    return newSchema;
+  }
+
+  @Override
+  public ManagedIndexSchema replaceField
+      (String fieldName, FieldType replacementFieldType, Map<String,?> replacementArgs) {
+    ManagedIndexSchema newSchema;
+    if (isMutable) {
+      SchemaField oldField = fields.get(fieldName);
+      if (null == oldField) {
+        String msg = "The field '" + fieldName + "' is not present in this schema, and so cannot be replaced.";
+        throw new SolrException(ErrorCode.BAD_REQUEST, msg);
+      }
+      newSchema = shallowCopy(true);
+      // clone data structures before modifying them
+      newSchema.copyFieldsMap = cloneCopyFieldsMap(copyFieldsMap);
+      newSchema.copyFieldTargetCounts
+          = (Map<SchemaField,Integer>)((HashMap<SchemaField,Integer>)copyFieldTargetCounts).clone();
+      newSchema.dynamicCopyFields = new DynamicCopy[dynamicCopyFields.length];
+      System.arraycopy(dynamicCopyFields, 0, newSchema.dynamicCopyFields, 0, dynamicCopyFields.length);
+
+      // Drop the old field
+      newSchema.fields.remove(fieldName);
+      newSchema.fieldsWithDefaultValue.remove(oldField);
+      newSchema.requiredFields.remove(oldField);
+
+      // Add the replacement field
+      SchemaField replacementField = SchemaField.create(fieldName, replacementFieldType, replacementArgs);
+      newSchema.fields.put(fieldName, replacementField);
+      if (null != replacementField.getDefaultValue()) {
+        log.debug(replacementField.getName() + " contains default value: " + replacementField.getDefaultValue());
+        newSchema.fieldsWithDefaultValue.add(replacementField);
+      }
+      if (replacementField.isRequired()) {
+        log.debug("{} is required in this schema", replacementField.getName());
+        newSchema.requiredFields.add(replacementField);
+      }
+
+      List<CopyField> copyFieldsToRebuild = new ArrayList<>();
+      newSchema.removeCopyFieldSource(fieldName, copyFieldsToRebuild);
+
+      newSchema.copyFieldTargetCounts.remove(oldField); // zero out target count for this field
+
+      // Remove copy fields where the target is this field; remember them to rebuild
+      for (Map.Entry<String,List<CopyField>> entry : newSchema.copyFieldsMap.entrySet()) {
+        List<CopyField> perSourceCopyFields = entry.getValue();
+        Iterator<CopyField> checkDestCopyFieldsIter = perSourceCopyFields.iterator();
+        while (checkDestCopyFieldsIter.hasNext()) {
+          CopyField checkDestCopyField = checkDestCopyFieldsIter.next();
+          if (fieldName.equals(checkDestCopyField.getDestination().getName())) {
+            checkDestCopyFieldsIter.remove();
+            copyFieldsToRebuild.add(checkDestCopyField);
+          }
+        }
+      }
+      newSchema.rebuildCopyFields(copyFieldsToRebuild);
+
+      // Find dynamic copy fields where the source or destination is this field; remember them to rebuild
+      List<DynamicCopy> dynamicCopyFieldsToRebuild = new ArrayList<>();
+      List<DynamicCopy> newDynamicCopyFields = new ArrayList<>();
+      for (int i = 0 ; i < newSchema.dynamicCopyFields.length ; ++i) {
+        DynamicCopy dynamicCopy = newSchema.dynamicCopyFields[i];
+        SchemaField destinationPrototype = dynamicCopy.getDestination().getPrototype();
+        if (fieldName.equals(dynamicCopy.getRegex()) || fieldName.equals(destinationPrototype.getName())) {
+          dynamicCopyFieldsToRebuild.add(dynamicCopy);
+        } else {
+          newDynamicCopyFields.add(dynamicCopy);
+        }
+      }
+      // Rebuild affected dynamic copy fields
+      if (dynamicCopyFieldsToRebuild.size() > 0) {
+        newSchema.dynamicCopyFields = newDynamicCopyFields.toArray(new DynamicCopy[newDynamicCopyFields.size()]);
+        for (DynamicCopy dynamicCopy : dynamicCopyFieldsToRebuild) {
+          newSchema.registerCopyField(dynamicCopy.getRegex(), dynamicCopy.getDestFieldName(), dynamicCopy.getMaxChars());
+        }
+      }
 
+      for (SchemaAware aware : newSchema.schemaAware) {
+        aware.inform(newSchema);
+      }
+      newSchema.refreshAnalyzers();
+    } else {
+      String msg = "This ManagedIndexSchema is not mutable.";
+      log.error(msg);
+      throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+    }
+    return newSchema;
+  }
+  
   @Override
   public ManagedIndexSchema addDynamicFields(Collection<SchemaField> newDynamicFields, 
                                              Map<String,Collection<String>> copyFieldNames, boolean persist) {
-    ManagedIndexSchema newSchema = null;
+    ManagedIndexSchema newSchema;
     if (isMutable) {
       boolean success = false;
       if (copyFieldNames == null){
@@ -474,7 +600,7 @@ public final class ManagedIndexSchema ex
         aware.inform(newSchema);
       }
       newSchema.refreshAnalyzers();
-      if(persist) {
+      if (persist) {
         success = newSchema.persistManagedSchema(false); // don't just create - update it if it already exists
         if (success) {
           log.debug("Added dynamic field(s): {}", newDynamicFields);
@@ -491,8 +617,152 @@ public final class ManagedIndexSchema ex
   }
 
   @Override
+  public ManagedIndexSchema deleteDynamicFields(Collection<String> fieldNamePatterns) {
+    ManagedIndexSchema newSchema;
+    if (isMutable) {
+      newSchema = shallowCopy(true);
+
+      newSchema.dynamicCopyFields = new DynamicCopy[dynamicCopyFields.length];
+      System.arraycopy(dynamicCopyFields, 0, newSchema.dynamicCopyFields, 0, dynamicCopyFields.length);
+
+      List<DynamicCopy> dynamicCopyFieldsToRebuild = new ArrayList<>();
+      List<DynamicCopy> newDynamicCopyFields = new ArrayList<>();
+
+      for (String fieldNamePattern : fieldNamePatterns) {
+        DynamicField dynamicField = null;
+        int dfPos = 0;
+        for ( ; dfPos < newSchema.dynamicFields.length ; ++dfPos) {
+          DynamicField df = newSchema.dynamicFields[dfPos];
+          if (df.getRegex().equals(fieldNamePattern)) {
+            dynamicField = df;
+            break;
+          }
+        }
+        if (null == dynamicField) {
+          String msg = "The dynamic field '" + fieldNamePattern
+              + "' is not present in this schema, and so cannot be deleted.";
+          throw new SolrException(ErrorCode.BAD_REQUEST, msg);
+        }          
+        for (int i = 0 ; i < newSchema.dynamicCopyFields.length ; ++i) {
+          DynamicCopy dynamicCopy = newSchema.dynamicCopyFields[i];
+          DynamicField destDynamicBase = dynamicCopy.getDestDynamicBase();
+          DynamicField sourceDynamicBase = dynamicCopy.getSourceDynamicBase();
+          if ((null != destDynamicBase && fieldNamePattern.equals(destDynamicBase.getRegex()))
+              || (null != sourceDynamicBase && fieldNamePattern.equals(sourceDynamicBase.getRegex()))
+              || dynamicField.matches(dynamicCopy.getRegex())
+              || dynamicField.matches(dynamicCopy.getDestFieldName())) {
+            dynamicCopyFieldsToRebuild.add(dynamicCopy);
+            newSchema.decrementCopyFieldTargetCount(dynamicCopy.getDestination().getPrototype());
+            // don't add this dynamic copy field to newDynamicCopyFields - effectively removing it
+          } else {
+            newDynamicCopyFields.add(dynamicCopy);
+          }
+        }
+        if (newSchema.dynamicFields.length > 1) {
+          DynamicField[] temp = new DynamicField[newSchema.dynamicFields.length - 1];
+          System.arraycopy(newSchema.dynamicFields, 0, temp, 0, dfPos);
+          // skip over the dynamic field to be deleted
+          System.arraycopy(newSchema.dynamicFields, dfPos + 1, temp, dfPos, newSchema.dynamicFields.length - dfPos - 1);
+          newSchema.dynamicFields = temp;
+        } else {
+          newSchema.dynamicFields = new DynamicField[0];
+        }
+      }
+      // After removing all dynamic fields, rebuild affected dynamic copy fields.
+      // This may trigger an exception, if one of the deleted dynamic fields was the only matching source or target.
+      if (dynamicCopyFieldsToRebuild.size() > 0) {
+        newSchema.dynamicCopyFields = newDynamicCopyFields.toArray(new DynamicCopy[newDynamicCopyFields.size()]);
+        for (DynamicCopy dynamicCopy : dynamicCopyFieldsToRebuild) {
+          newSchema.registerCopyField(dynamicCopy.getRegex(), dynamicCopy.getDestFieldName(), dynamicCopy.getMaxChars());
+        }
+      }
+
+      // Run the callbacks on SchemaAware now that everything else is done
+      for (SchemaAware aware : newSchema.schemaAware) {
+        aware.inform(newSchema);
+      }
+      newSchema.refreshAnalyzers();
+    } else {
+      String msg = "This ManagedIndexSchema is not mutable.";
+      log.error(msg);
+      throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+    }
+    return newSchema;
+  }
+
+  @Override
+  public ManagedIndexSchema replaceDynamicField
+    (String fieldNamePattern, FieldType replacementFieldType, Map<String,?> replacementArgs) {
+    ManagedIndexSchema newSchema;
+    if (isMutable) {
+      DynamicField oldDynamicField = null;
+      int dfPos = 0;
+      for ( ; dfPos < dynamicFields.length ; ++dfPos) {
+        DynamicField dynamicField = dynamicFields[dfPos];
+        if (dynamicField.getRegex().equals(fieldNamePattern)) {
+          oldDynamicField = dynamicField;
+          break;
+        }
+      }
+      if (null == oldDynamicField) {
+        String msg = "The dynamic field '" + fieldNamePattern 
+            + "' is not present in this schema, and so cannot be replaced.";
+        throw new SolrException(ErrorCode.BAD_REQUEST, msg);
+      }
+
+      newSchema = shallowCopy(true);
+
+      // clone data structures before modifying them
+      newSchema.copyFieldTargetCounts
+          = (Map<SchemaField,Integer>)((HashMap<SchemaField,Integer>)copyFieldTargetCounts).clone();
+      newSchema.dynamicCopyFields = new DynamicCopy[dynamicCopyFields.length];
+      System.arraycopy(dynamicCopyFields, 0, newSchema.dynamicCopyFields, 0, dynamicCopyFields.length);
+
+      // Put the replacement dynamic field in place
+      SchemaField prototype = SchemaField.create(fieldNamePattern, replacementFieldType, replacementArgs);
+      newSchema.dynamicFields[dfPos] = new DynamicField(prototype);
+
+      // Find dynamic copy fields where this dynamic field is the source or target base; remember them to rebuild
+      List<DynamicCopy> dynamicCopyFieldsToRebuild = new ArrayList<>();
+      List<DynamicCopy> newDynamicCopyFields = new ArrayList<>();
+      for (int i = 0 ; i < newSchema.dynamicCopyFields.length ; ++i) {
+        DynamicCopy dynamicCopy = newSchema.dynamicCopyFields[i];
+        DynamicField destDynamicBase = dynamicCopy.getDestDynamicBase();
+        DynamicField sourceDynamicBase = dynamicCopy.getSourceDynamicBase();
+        if (fieldNamePattern.equals(dynamicCopy.getRegex())
+            || fieldNamePattern.equals(dynamicCopy.getDestFieldName())
+            || (null != destDynamicBase && fieldNamePattern.equals(destDynamicBase.getRegex()))
+            || (null != sourceDynamicBase && fieldNamePattern.equals(sourceDynamicBase.getRegex()))) {
+          dynamicCopyFieldsToRebuild.add(dynamicCopy);
+          newSchema.decrementCopyFieldTargetCount(dynamicCopy.getDestination().getPrototype());
+          // don't add this dynamic copy field to newDynamicCopyFields - effectively removing it
+        } else {
+          newDynamicCopyFields.add(dynamicCopy);
+        }
+      }
+      // Rebuild affected dynamic copy fields
+      if (dynamicCopyFieldsToRebuild.size() > 0) {
+        newSchema.dynamicCopyFields = newDynamicCopyFields.toArray(new DynamicCopy[newDynamicCopyFields.size()]);
+        for (DynamicCopy dynamicCopy : dynamicCopyFieldsToRebuild) {
+          newSchema.registerCopyField(dynamicCopy.getRegex(), dynamicCopy.getDestFieldName(), dynamicCopy.getMaxChars());
+        }
+      }
+
+      for (SchemaAware aware : newSchema.schemaAware) {
+        aware.inform(newSchema);
+      }
+      newSchema.refreshAnalyzers();
+    } else {
+      String msg = "This ManagedIndexSchema is not mutable.";
+      log.error(msg);
+      throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+    }
+    return newSchema;
+  }
+
+  @Override
   public ManagedIndexSchema addCopyFields(Map<String, Collection<String>> copyFields, boolean persist) {
-    ManagedIndexSchema newSchema = null;
+    ManagedIndexSchema newSchema;
     if (isMutable) {
       boolean success = false;
       newSchema = shallowCopy(true);
@@ -517,10 +787,153 @@ public final class ManagedIndexSchema ex
           log.error("Failed to add copy fields for {} sources", copyFields.size());
         }
       }
+    } else {
+      String msg = "This ManagedIndexSchema is not mutable.";
+      log.error(msg);
+      throw new SolrException(ErrorCode.SERVER_ERROR, msg);
     }
     return newSchema;
   }
 
+  @Override
+  public ManagedIndexSchema deleteCopyFields(Map<String,Collection<String>> copyFields) {
+    ManagedIndexSchema newSchema;
+    if (isMutable) {
+      newSchema = shallowCopy(true);
+      // clone data structures before modifying them
+      newSchema.copyFieldsMap = cloneCopyFieldsMap(copyFieldsMap);
+      newSchema.copyFieldTargetCounts
+          = (Map<SchemaField,Integer>)((HashMap<SchemaField,Integer>)copyFieldTargetCounts).clone();
+      newSchema.dynamicCopyFields = new DynamicCopy[dynamicCopyFields.length];
+      System.arraycopy(dynamicCopyFields, 0, newSchema.dynamicCopyFields, 0, dynamicCopyFields.length);
+
+      for (Map.Entry<String,Collection<String>> entry : copyFields.entrySet()) {
+        // Key is the source, values are the destinations
+        for (String destination : entry.getValue()) {
+          newSchema.deleteCopyField(entry.getKey(), destination);
+        }
+      }
+      //TODO: move this common stuff out to shared methods
+      // Run the callbacks on SchemaAware now that everything else is done
+      for (SchemaAware aware : newSchema.schemaAware) {
+        aware.inform(newSchema);
+      }
+      newSchema.refreshAnalyzers();
+    } else {
+      String msg = "This ManagedIndexSchema is not mutable.";
+      log.error(msg);
+      throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+    }
+    return newSchema;
+  }
+  
+  private void deleteCopyField(String source, String dest) {
+    // Assumption: a copy field directive will exist only if the source & destination (dynamic) fields exist
+    SchemaField destSchemaField = fields.get(dest);
+    SchemaField sourceSchemaField = fields.get(source);
+
+    final String invalidGlobMessage = "is an invalid glob: either it contains more than one asterisk,"
+        + " or the asterisk occurs neither at the start nor at the end.";
+    if (source.contains("*") && ! isValidFieldGlob(source)) {
+      String msg = "copyField source '" + source + "' " + invalidGlobMessage;
+      throw new SolrException(ErrorCode.BAD_REQUEST, msg);
+    }
+    if (dest.contains("*") && ! isValidFieldGlob(dest)) {
+      String msg = "copyField dest '" + dest + "' " + invalidGlobMessage;
+      throw new SolrException(ErrorCode.BAD_REQUEST, msg);
+    }
+
+    boolean found = false;
+
+    if (null == destSchemaField || null == sourceSchemaField) { // Must be dynamic copy field
+      if (dynamicCopyFields != null) {
+        for (int i = 0 ; i < dynamicCopyFields.length ; ++i) {
+          DynamicCopy dynamicCopy = dynamicCopyFields[i];
+          if (source.equals(dynamicCopy.getRegex()) && dest.equals(dynamicCopy.getDestFieldName())) {
+            found = true;
+            SchemaField destinationPrototype = dynamicCopy.getDestination().getPrototype();
+            if (copyFieldTargetCounts.containsKey(destinationPrototype)) {
+              decrementCopyFieldTargetCount(destinationPrototype);
+            }
+            if (dynamicCopyFields.length > 1) {
+              DynamicCopy[] temp = new DynamicCopy[dynamicCopyFields.length - 1];
+              System.arraycopy(dynamicCopyFields, 0, temp, 0, i);
+              // skip over the dynamic copy field to be deleted
+              System.arraycopy(dynamicCopyFields, i + 1, temp, i, dynamicCopyFields.length - i - 1);
+              dynamicCopyFields = temp;
+            } else {
+              dynamicCopyFields = null;
+            }
+            break;
+          }
+        }
+      }
+    } else { // non-dynamic copy field directive
+      List<CopyField> copyFieldList = copyFieldsMap.get(source);
+      if (copyFieldList != null) {
+        for (Iterator<CopyField> iter = copyFieldList.iterator() ; iter.hasNext() ; ) {
+          CopyField copyField = iter.next();
+          if (dest.equals(copyField.getDestination().getName())) {
+            found = true;
+            decrementCopyFieldTargetCount(copyField.getDestination());
+            iter.remove();
+            if (copyFieldList.isEmpty()) {
+              copyFieldsMap.remove(source);
+            }
+            break;
+          }
+        }
+      }
+    }
+    if ( ! found) {
+      throw new SolrException(ErrorCode.BAD_REQUEST,
+          "Copy field directive not found: '" + source + "' -> '" + dest + "'");
+    }
+  }
+
+  /**
+   * Removes all copy fields with the given source field name, decrements the count for the copy field target,
+   * and adds the removed copy fields to removedCopyFields.
+   */
+  private void removeCopyFieldSource(String sourceFieldName, List<CopyField> removedCopyFields) {
+    List<CopyField> sourceCopyFields = copyFieldsMap.remove(sourceFieldName);
+    if (null != sourceCopyFields) {
+      for (CopyField sourceCopyField : sourceCopyFields) {
+        decrementCopyFieldTargetCount(sourceCopyField.getDestination());
+        removedCopyFields.add(sourceCopyField);
+      }
+    }
+  }
+
+  /**
+   * Registers new copy fields with the source, destination and maxChars taken from each of the oldCopyFields.
+   * 
+   * Assumption: the fields in oldCopyFields still exist in the schema. 
+   */
+  private void rebuildCopyFields(List<CopyField> oldCopyFields) {
+    if (oldCopyFields.size() > 0) {
+      for (CopyField copyField : oldCopyFields) {
+        SchemaField source = fields.get(copyField.getSource().getName());
+        SchemaField destination = fields.get(copyField.getDestination().getName());
+        registerExplicitSrcAndDestFields
+            (copyField.getSource().getName(), copyField.getMaxChars(), destination, source);
+      }
+    }
+  }
+
+  /**
+   * Decrements the count for the given destination field in copyFieldTargetCounts.
+   */
+  private void decrementCopyFieldTargetCount(SchemaField dest) {
+    Integer count = copyFieldTargetCounts.get(dest);
+    assert count != null;
+    if (count <= 1) {
+      copyFieldTargetCounts.remove(dest);
+    } else {
+      copyFieldTargetCounts.put(dest, count - 1);
+    }
+  }
+
   public ManagedIndexSchema addFieldTypes(List<FieldType> fieldTypeList, boolean persist) {
     if (!isMutable) {
       String msg = "This ManagedIndexSchema is not mutable.";
@@ -579,6 +992,182 @@ public final class ManagedIndexSchema ex
     return newSchema;
   }
 
+  @Override
+  public ManagedIndexSchema deleteFieldTypes(Collection<String> names) {
+    ManagedIndexSchema newSchema;
+    if (isMutable) {
+      for (String name : names) {
+        if ( ! fieldTypes.containsKey(name)) {
+          String msg = "The field type '" + name + "' is not present in this schema, and so cannot be deleted.";
+          throw new SolrException(ErrorCode.BAD_REQUEST, msg);
+        }
+        for (SchemaField field : fields.values()) {
+          if (field.getType().getTypeName().equals(name)) {
+            throw new SolrException(ErrorCode.BAD_REQUEST, "Can't delete '" + name
+                + "' because it's the field type of field '" + field.getName() + "'.");
+          }
+        }
+        for (DynamicField dynamicField : dynamicFields) {
+          if (dynamicField.getPrototype().getType().getTypeName().equals(name)) {
+            throw new SolrException(ErrorCode.BAD_REQUEST, "Can't delete '" + name
+                + "' because it's the field type of dynamic field '" + dynamicField.getRegex() + "'.");
+          }
+        }
+      }
+      newSchema = shallowCopy(true);
+      for (String name : names) {
+        newSchema.fieldTypes.remove(name);
+      }
+      for (SchemaAware aware : newSchema.schemaAware) {
+        aware.inform(newSchema);
+      }
+      for (FieldType fieldType : newSchema.fieldTypes.values()) {
+        informResourceLoaderAwareObjectsForFieldType(fieldType);
+      }
+      newSchema.refreshAnalyzers();
+    } else {
+      String msg = "This ManagedIndexSchema is not mutable.";
+      log.error(msg);
+      throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+    }
+    return newSchema;
+  }
+  
+  private Map<String,List<CopyField>> cloneCopyFieldsMap(Map<String,List<CopyField>> original) {
+    Map<String,List<CopyField>> clone = new HashMap<>(original.size());
+    Iterator<Map.Entry<String,List<CopyField>>> iterator = original.entrySet().iterator();
+    while (iterator.hasNext()) {
+      Map.Entry<String,List<CopyField>> entry = iterator.next();
+      clone.put(entry.getKey(), new ArrayList<>(entry.getValue()));
+    }
+    return clone;
+  }
+
+  @Override
+  public ManagedIndexSchema replaceFieldType(String typeName, String replacementClassName, Map<String,Object> replacementArgs) {
+    ManagedIndexSchema newSchema;
+    if (isMutable) {
+      if ( ! fieldTypes.containsKey(typeName)) {
+        String msg = "The field type '" + typeName + "' is not present in this schema, and so cannot be replaced.";
+        throw new SolrException(ErrorCode.BAD_REQUEST, msg);
+      }
+      newSchema = shallowCopy(true);
+      // clone data structures before modifying them
+      newSchema.fieldTypes = (Map<String,FieldType>)((HashMap<String,FieldType>)fieldTypes).clone();
+      newSchema.copyFieldsMap = cloneCopyFieldsMap(copyFieldsMap);
+      newSchema.copyFieldTargetCounts
+          = (Map<SchemaField,Integer>)((HashMap<SchemaField,Integer>)copyFieldTargetCounts).clone();
+      newSchema.dynamicCopyFields = new DynamicCopy[dynamicCopyFields.length];
+      System.arraycopy(dynamicCopyFields, 0, newSchema.dynamicCopyFields, 0, dynamicCopyFields.length);
+      newSchema.dynamicFields = new DynamicField[dynamicFields.length];
+      System.arraycopy(dynamicFields, 0, newSchema.dynamicFields, 0, dynamicFields.length);
+      
+      newSchema.fieldTypes.remove(typeName);
+      FieldType replacementFieldType = newSchema.newFieldType(typeName, replacementClassName, replacementArgs);
+      newSchema.fieldTypes.put(typeName, replacementFieldType);
+
+      // Rebuild fields of the type being replaced
+      List<CopyField> copyFieldsToRebuild = new ArrayList<>();
+      List<SchemaField> replacementFields = new ArrayList<>();
+      Iterator<Map.Entry<String,SchemaField>> fieldsIter = newSchema.fields.entrySet().iterator();
+      while (fieldsIter.hasNext()) {
+        Map.Entry<String,SchemaField> entry = fieldsIter.next();
+        SchemaField oldField = entry.getValue();
+        if (oldField.getType().getTypeName().equals(typeName)) {
+          String fieldName = oldField.getName();
+          
+          // Drop the old field
+          fieldsIter.remove();
+          newSchema.fieldsWithDefaultValue.remove(oldField);
+          newSchema.requiredFields.remove(oldField);
+          
+          // Add the replacement field
+          SchemaField replacementField = SchemaField.create(fieldName, replacementFieldType, oldField.getArgs());
+          replacementFields.add(replacementField); // Save the new field to be added after iteration is finished
+          if (null != replacementField.getDefaultValue()) {
+            log.debug(replacementField.getName() + " contains default value: " + replacementField.getDefaultValue());
+            newSchema.fieldsWithDefaultValue.add(replacementField);
+          }
+          if (replacementField.isRequired()) {
+            log.debug("{} is required in this schema", replacementField.getName());
+            newSchema.requiredFields.add(replacementField);
+          }
+          newSchema.removeCopyFieldSource(fieldName, copyFieldsToRebuild);
+        }
+      }
+      for (SchemaField replacementField : replacementFields) {
+        newSchema.fields.put(replacementField.getName(), replacementField);
+      }
+      // Remove copy fields where the target is of the type being replaced; remember them to rebuild
+      Iterator<Map.Entry<String,List<CopyField>>> copyFieldsMapIter = newSchema.copyFieldsMap.entrySet().iterator();
+      while (copyFieldsMapIter.hasNext()) {
+        Map.Entry<String,List<CopyField>> entry = copyFieldsMapIter.next();
+        List<CopyField> perSourceCopyFields = entry.getValue();
+        Iterator<CopyField> checkDestCopyFieldsIter = perSourceCopyFields.iterator();
+        while (checkDestCopyFieldsIter.hasNext()) {
+          CopyField checkDestCopyField = checkDestCopyFieldsIter.next();
+          SchemaField destination = checkDestCopyField.getDestination();
+          if (typeName.equals(destination.getType().getTypeName())) {
+            checkDestCopyFieldsIter.remove();
+            copyFieldsToRebuild.add(checkDestCopyField);
+            newSchema.copyFieldTargetCounts.remove(destination); // zero out target count
+          }
+        }
+        if (perSourceCopyFields.isEmpty()) {
+          copyFieldsMapIter.remove();
+        }
+      }
+      // Rebuild dynamic fields of the type being replaced
+      for (int i = 0; i < newSchema.dynamicFields.length; ++i) {
+        SchemaField prototype = newSchema.dynamicFields[i].getPrototype();
+        if (typeName.equals(prototype.getType().getTypeName())) {
+          newSchema.dynamicFields[i] = new DynamicField
+              (SchemaField.create(prototype.getName(), replacementFieldType, prototype.getArgs()));
+        }
+      }
+      // Find dynamic copy fields where the destination field's type is being replaced
+      // or the source dynamic base's type is being replaced; remember them to rebuild
+      List<DynamicCopy> dynamicCopyFieldsToRebuild = new ArrayList<>();
+      List<DynamicCopy> newDynamicCopyFields = new ArrayList<>();
+      for (int i = 0 ; i < newSchema.dynamicCopyFields.length ; ++i) {
+        DynamicCopy dynamicCopy = newSchema.dynamicCopyFields[i];
+        DynamicField sourceDynamicBase = dynamicCopy.getSourceDynamicBase();
+        SchemaField destinationPrototype = dynamicCopy.getDestination().getPrototype();
+        if (typeName.equals(destinationPrototype.getType().getTypeName())
+            || (null != sourceDynamicBase && typeName.equals(sourceDynamicBase.getPrototype().getType().getTypeName()))) {
+          dynamicCopyFieldsToRebuild.add(dynamicCopy);
+          if (newSchema.copyFieldTargetCounts.containsKey(destinationPrototype)) {
+            newSchema.decrementCopyFieldTargetCount(destinationPrototype);
+          }
+          // don't add this dynamic copy field to newDynamicCopyFields - effectively removing it
+        } else {
+          newDynamicCopyFields.add(dynamicCopy);
+        }
+      }
+      // Rebuild affected dynamic copy fields
+      if (dynamicCopyFieldsToRebuild.size() > 0) {
+        newSchema.dynamicCopyFields = newDynamicCopyFields.toArray(new DynamicCopy[newDynamicCopyFields.size()]);
+        for (DynamicCopy dynamicCopy : dynamicCopyFieldsToRebuild) {
+          newSchema.registerCopyField(dynamicCopy.getRegex(), dynamicCopy.getDestFieldName(), dynamicCopy.getMaxChars());
+        }
+      }
+      newSchema.rebuildCopyFields(copyFieldsToRebuild);
+
+      for (SchemaAware aware : newSchema.schemaAware) {
+        aware.inform(newSchema);
+      }
+      for (FieldType fieldType : newSchema.fieldTypes.values()) {
+        newSchema.informResourceLoaderAwareObjectsForFieldType(fieldType);
+      }
+      newSchema.refreshAnalyzers();
+    } else {
+      String msg = "This ManagedIndexSchema is not mutable.";
+      log.error(msg);
+      throw new SolrException(ErrorCode.SERVER_ERROR, msg);
+    }
+    return newSchema;
+  }
+
   /**
    * Informs analyzers used by a fieldType.
    */
@@ -696,8 +1285,8 @@ public final class ManagedIndexSchema ex
     // build the new FieldType using the existing FieldTypePluginLoader framework
     // which expects XML, so we use a JSON to XML adapter to transform the JSON object
     // provided in the request into the XML format supported by the plugin loader
-    Map<String, FieldType> newFieldTypes = new HashMap<String, FieldType>();
-    List<SchemaAware> schemaAwareList = new ArrayList<SchemaAware>();
+    Map<String,FieldType> newFieldTypes = new HashMap<>();
+    List<SchemaAware> schemaAwareList = new ArrayList<>();
     FieldTypePluginLoader typeLoader = new FieldTypePluginLoader(this, newFieldTypes, schemaAwareList);
     typeLoader.loadSingle(loader, FieldTypeXmlAdapter.toNode(options));
     FieldType ft = newFieldTypes.get(typeName);
@@ -749,56 +1338,6 @@ public final class ManagedIndexSchema ex
     }
   }
   
-
-  /** 
-   * Called from ZkIndexSchemaReader to merge the fields from the serialized managed schema
-   * on ZooKeeper with the local managed schema.
-   * 
-   * @param inputSource The serialized content of the managed schema from ZooKeeper
-   * @param schemaZkVersion The ZK version of the managed schema on ZooKeeper
-   * @return The new merged schema
-   */
-  ManagedIndexSchema reloadFields(InputSource inputSource, int schemaZkVersion) {
-    ManagedIndexSchema newSchema;
-    try {
-      newSchema = shallowCopy(false);
-      Config schemaConf = new Config(loader, SCHEMA, inputSource, SLASH+SCHEMA+SLASH);
-      Document document = schemaConf.getDocument();
-      final XPath xpath = schemaConf.getXPath();
-
-      // create a unified collection of field types from zk and in the local
-      newSchema.mergeFieldTypesFromZk(document, xpath);
-
-      newSchema.loadFields(document, xpath);
-      // let's completely rebuild the copy fields from the schema in ZK.
-      // create new copyField-related objects so we don't affect the
-      // old schema
-      newSchema.copyFieldsMap = new HashMap<>();
-      newSchema.dynamicCopyFields = new DynamicCopy[] {};
-      newSchema.copyFieldTargetCounts = new HashMap<>();
-      newSchema.loadCopyFields(document, xpath);
-      if (null != uniqueKeyField) {
-        newSchema.requiredFields.add(uniqueKeyField);
-      }
-      //Run the callbacks on SchemaAware now that everything else is done
-      for (SchemaAware aware : newSchema.schemaAware) {
-        aware.inform(newSchema);
-      }
-
-      // notify analyzers and other objects for our fieldTypes
-      for (FieldType fieldType : newSchema.fieldTypes.values())
-        informResourceLoaderAwareObjectsForFieldType(fieldType);
-
-      newSchema.refreshAnalyzers();
-      newSchema.schemaZkVersion = schemaZkVersion;
-    } catch (SolrException e) {
-      throw e;
-    } catch (Exception e) {
-      throw new SolrException(ErrorCode.SERVER_ERROR, "Schema Parsing Failed: " + e.getMessage(), e);
-    }
-    return newSchema;
-  }
-  
   private ManagedIndexSchema(final SolrConfig solrConfig, final SolrResourceLoader loader, boolean isMutable,
                              String managedSchemaResourceName, int schemaZkVersion, Object schemaUpdateLock) 
       throws KeeperException, InterruptedException {
@@ -872,23 +1411,4 @@ public final class ManagedIndexSchema ex
   public Object getSchemaUpdateLock() {
     return schemaUpdateLock;
   }
-
-  /**
-   * Loads FieldType objects defined in the schema.xml document.
-   *
-   * @param document Schema XML document where field types are defined.
-   * @param xpath Used for evaluating xpath expressions to find field types defined in the schema.xml.
-   * @throws javax.xml.xpath.XPathExpressionException if an error occurs when finding field type elements in the document.
-   */
-  protected synchronized void mergeFieldTypesFromZk(Document document, XPath xpath)
-      throws XPathExpressionException
-  {
-    Map<String, FieldType> newFieldTypes = new HashMap<String, FieldType>();
-    FieldTypePluginLoader typeLoader = new FieldTypePluginLoader(this, newFieldTypes, schemaAware);
-    String expression = getFieldTypeXPathExpressions();
-    NodeList nodes = (NodeList) xpath.evaluate(expression, document, XPathConstants.NODESET);
-    typeLoader.load(loader, nodes);
-    for (String newTypeName : newFieldTypes.keySet())
-      fieldTypes.put(newTypeName, newFieldTypes.get(newTypeName));
-  }
 }

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchemaFactory.java Tue Mar 31 05:22:40 2015
@@ -404,4 +404,12 @@ public class ManagedIndexSchemaFactory e
     this.schema = schema;
     core.setLatestSchema(schema);
   }
+  
+  public boolean isMutable() {
+    return isMutable;
+  }
+
+  public SolrConfig getConfig() {
+    return config;
+  }
 }

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/PreAnalyzedField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/PreAnalyzedField.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/PreAnalyzedField.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/PreAnalyzedField.java Tue Mar 31 05:22:40 2015
@@ -28,12 +28,11 @@ import java.util.Map;
 import org.apache.lucene.analysis.Analyzer;
 import org.apache.lucene.analysis.Tokenizer;
 import org.apache.lucene.document.Field;
-import org.apache.lucene.index.StorableField;
 import org.apache.lucene.index.IndexOptions;
+import org.apache.lucene.index.StorableField;
 import org.apache.lucene.queries.function.ValueSource;
 import org.apache.lucene.queries.function.valuesource.SortedSetFieldSource;
 import org.apache.lucene.search.SortField;
-import org.apache.lucene.search.SortedSetSortField;
 import org.apache.lucene.uninverting.UninvertingReader.Type;
 import org.apache.lucene.util.AttributeFactory;
 import org.apache.lucene.util.AttributeSource;
@@ -178,6 +177,7 @@ public class PreAnalyzedField extends Fi
     newType.setStoreTermVectors(field.storeTermVector());
     newType.setStoreTermVectorOffsets(field.storeTermOffsets());
     newType.setStoreTermVectorPositions(field.storeTermPositions());
+    newType.setStoreTermVectorPayloads(field.storeTermPayloads());
     return newType;
   }
   

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/SchemaField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/SchemaField.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/SchemaField.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/SchemaField.java Tue Mar 31 05:22:40 2015
@@ -17,19 +17,18 @@
 
 package org.apache.solr.schema;
 
-import org.apache.solr.common.SolrException;
-import org.apache.lucene.index.StorableField;
-import org.apache.lucene.search.SortField;
-import org.apache.solr.common.util.SimpleOrderedMap;
-import org.apache.solr.search.QParser;
-
-import org.apache.solr.response.TextResponseWriter;
-
+import java.io.IOException;
 import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.io.IOException;
+
+import org.apache.lucene.index.StorableField;
+import org.apache.lucene.search.SortField;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.util.SimpleOrderedMap;
+import org.apache.solr.response.TextResponseWriter;
+import org.apache.solr.search.QParser;
 
 /**
  * Encapsulates all information about a Field in a Solr Schema
@@ -94,6 +93,7 @@ public final class SchemaField extends F
   public boolean storeTermVector() { return (properties & STORE_TERMVECTORS)!=0; }
   public boolean storeTermPositions() { return (properties & STORE_TERMPOSITIONS)!=0; }
   public boolean storeTermOffsets() { return (properties & STORE_TERMOFFSETS)!=0; }
+  public boolean storeTermPayloads() { return (properties & STORE_TERMPAYLOADS)!=0; }
   public boolean omitNorms() { return (properties & OMIT_NORMS)!=0; }
 
   public boolean omitTermFreqAndPositions() { return (properties & OMIT_TF_POSITIONS)!=0; }
@@ -104,6 +104,7 @@ public final class SchemaField extends F
   public boolean sortMissingFirst() { return (properties & SORT_MISSING_FIRST)!=0; }
   public boolean sortMissingLast() { return (properties & SORT_MISSING_LAST)!=0; }
   public boolean isRequired() { return required; } 
+  public Map<String,?> getArgs() { return Collections.unmodifiableMap(args); }
 
   // things that should be determined by field type, not set as options
   boolean isTokenized() { return (properties & TOKENIZED)!=0; }
@@ -235,7 +236,7 @@ public final class SchemaField extends F
 
     if (on(falseProps,INDEXED)) {
       int pp = (INDEXED 
-              | STORE_TERMVECTORS | STORE_TERMPOSITIONS | STORE_TERMOFFSETS);
+              | STORE_TERMVECTORS | STORE_TERMPOSITIONS | STORE_TERMOFFSETS | STORE_TERMPAYLOADS);
       if (on(pp,trueProps)) {
         throw new RuntimeException("SchemaField: " + name + " conflicting 'true' field options for non-indexed field:" + props);
       }
@@ -268,7 +269,7 @@ public final class SchemaField extends F
     }
 
     if (on(falseProps,STORE_TERMVECTORS)) {
-      int pp = (STORE_TERMVECTORS | STORE_TERMPOSITIONS | STORE_TERMOFFSETS);
+      int pp = (STORE_TERMVECTORS | STORE_TERMPOSITIONS | STORE_TERMOFFSETS | STORE_TERMPAYLOADS);
       if (on(pp,trueProps)) {
         throw new RuntimeException("SchemaField: " + name + " conflicting termvector field options:" + props);
       }
@@ -322,6 +323,7 @@ public final class SchemaField extends F
       properties.add(getPropertyName(STORE_TERMVECTORS), storeTermVector());
       properties.add(getPropertyName(STORE_TERMPOSITIONS), storeTermPositions());
       properties.add(getPropertyName(STORE_TERMOFFSETS), storeTermOffsets());
+      properties.add(getPropertyName(STORE_TERMPAYLOADS), storeTermPayloads());
       properties.add(getPropertyName(OMIT_NORMS), omitNorms());
       properties.add(getPropertyName(OMIT_TF_POSITIONS), omitTermFreqAndPositions());
       properties.add(getPropertyName(OMIT_POSITIONS), omitPositions());

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/SchemaManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/SchemaManager.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/SchemaManager.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/SchemaManager.java Tue Mar 31 05:22:40 2015
@@ -21,7 +21,6 @@ package org.apache.solr.schema;
 import org.apache.solr.cloud.ZkController;
 import org.apache.solr.cloud.ZkSolrResourceLoader;
 import org.apache.solr.common.SolrException;
-import org.apache.solr.core.ConfigOverlay;
 import org.apache.solr.core.CoreDescriptor;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
@@ -38,15 +37,13 @@ import java.io.InputStream;
 import java.io.Reader;
 import java.io.StringWriter;
 import java.nio.charset.StandardCharsets;
-import java.util.Collection;
 import java.util.Collections;
-import java.util.HashSet;
+import java.util.HashMap;
 import java.util.List;
-import java.util.Set;
+import java.util.Map;
 import java.util.concurrent.TimeUnit;
 
-import static java.util.Collections.EMPTY_LIST;
-import static java.util.Collections.EMPTY_MAP;
+import static java.util.Collections.singleton;
 import static java.util.Collections.singletonList;
 import static java.util.Collections.singletonMap;
 import static org.apache.solr.schema.FieldType.CLASS_NAME;
@@ -55,9 +52,10 @@ import static org.apache.solr.schema.Ind
 import static org.apache.solr.schema.IndexSchema.SOURCE;
 import static org.apache.solr.schema.IndexSchema.TYPE;
 
-/**A utility class to manipulate schema using the bulk mode.
- * This class takes in all the commands and process them completely. It is an all or none
- * operation
+/**
+ * A utility class to manipulate schema using the bulk mode.
+ * This class takes in all the commands and processes them completely.
+ * It is an all or nothing operation.
  */
 public class SchemaManager {
   private static final Logger log = LoggerFactory.getLogger(SchemaManager.class);
@@ -65,50 +63,35 @@ public class SchemaManager {
   final SolrQueryRequest req;
   ManagedIndexSchema managedIndexSchema;
 
-  public static final String ADD_FIELD = "add-field";
-  public static final String ADD_COPY_FIELD = "add-copy-field";
-  public static final String ADD_DYNAMIC_FIELD = "add-dynamic-field";
-  public static final String ADD_FIELD_TYPE = "add-field-type";
-
-  private static final Set<String> KNOWN_OPS = new HashSet<>();
-  static {
-    KNOWN_OPS.add(ADD_COPY_FIELD);
-    KNOWN_OPS.add(ADD_FIELD);
-    KNOWN_OPS.add(ADD_DYNAMIC_FIELD);
-    KNOWN_OPS.add(ADD_FIELD_TYPE);
-  }
-
   public SchemaManager(SolrQueryRequest req){
     this.req = req;
-
   }
 
-  /**Take in a JSON command set and execute them . It tries to capture as many errors
-   * as possible instead of failing at the frst error it encounters
-   * @param rdr The input as a Reader
-   * @return Lis of errors . If the List is empty then the operation is successful.
+  /**
+   * Take in a JSON command set and execute them. It tries to capture as many errors
+   * as possible instead of failing at the first error it encounters
+   * @param reader The input as a Reader
+   * @return List of errors. If the List is empty then the operation was successful.
    */
-  public List performOperations(Reader rdr) throws Exception {
-    List<CommandOperation> ops = null;
+  public List performOperations(Reader reader) throws Exception {
+    List<CommandOperation> ops;
     try {
-      ops = CommandOperation.parse(rdr);
+      ops = CommandOperation.parse(reader);
     } catch (Exception e) {
-      String msg= "Error parsing schema operations ";
-      log.warn(msg  ,e );
+      String msg = "Error parsing schema operations ";
+      log.warn(msg, e);
       return Collections.singletonList(singletonMap(CommandOperation.ERR_MSGS, msg + ":" + e.getMessage()));
     }
     List errs = CommandOperation.captureErrors(ops);
-    if(!errs.isEmpty()) return errs;
+    if (!errs.isEmpty()) return errs;
 
     IndexSchema schema = req.getCore().getLatestSchema();
     if (!(schema instanceof ManagedIndexSchema)) {
-      return singletonList( singletonMap(CommandOperation.ERR_MSGS,"schema is not editable"));
+      return singletonList(singletonMap(CommandOperation.ERR_MSGS, "schema is not editable"));
     }
-
     synchronized (schema.getSchemaUpdateLock()) {
       return doOperations(ops);
     }
-
   }
 
   private List doOperations(List<CommandOperation> operations) throws InterruptedException, IOException, KeeperException {
@@ -116,16 +99,12 @@ public class SchemaManager {
     long startTime = System.nanoTime();
     long endTime = timeout > 0 ? System.nanoTime() + (timeout * 1000 * 1000) : Long.MAX_VALUE;
     SolrCore core = req.getCore();
-    for (; System.nanoTime() < endTime; ) {
+    while (System.nanoTime() < endTime) {
       managedIndexSchema = getFreshManagedSchema();
       for (CommandOperation op : operations) {
-        if (ADD_FIELD.equals(op.name) || ADD_DYNAMIC_FIELD.equals(op.name)) {
-          applyAddField(op);
-        } else if(ADD_COPY_FIELD.equals(op.name)) {
-          applyAddCopyField(op);
-        } else if(ADD_FIELD_TYPE.equals(op.name)) {
-          applyAddType(op);
-
+        OpType opType = OpType.get(op.name);
+        if (opType != null) {
+          opType.perform(op, this);
         } else {
           op.addError("No such operation : " + op.name);
         }
@@ -150,28 +129,24 @@ public class SchemaManager {
               managedIndexSchema.getResourceName(),
               sw.toString().getBytes(StandardCharsets.UTF_8),
               true);
-          return EMPTY_LIST;
+          return Collections.emptyList();
         } catch (ZkController.ResourceModifiedInZkException e) {
           log.info("Race condition schema modified by another node");
-          continue;
         } catch (Exception e) {
           String s = "Exception persisting schema";
           log.warn(s, e);
           return singletonList(s + e.getMessage());
         }
-
-      }else {
-
+      } else {
         try {
           //only for non cloud stuff
           managedIndexSchema.persistManagedSchema(false);
           core.setLatestSchema(managedIndexSchema);
           waitForOtherReplicasToUpdate(timeout, startTime);
-          return EMPTY_LIST;
+          return Collections.emptyList();
         } catch (ManagedIndexSchema.SchemaChangedInZkException e) {
           String s = "Failed to update schema because schema is modified";
           log.warn(s, e);
-          continue;
         } catch (Exception e) {
           String s = "Exception persisting schema";
           log.warn(s, e);
@@ -179,120 +154,281 @@ public class SchemaManager {
         }
       }
     }
-
     return singletonList("Unable to persist schema");
-
   }
 
   private void waitForOtherReplicasToUpdate(int timeout, long startTime) {
-    if(timeout > 0 && managedIndexSchema.getResourceLoader()instanceof ZkSolrResourceLoader){
+    if (timeout > 0 && managedIndexSchema.getResourceLoader() instanceof ZkSolrResourceLoader) {
       CoreDescriptor cd = req.getCore().getCoreDescriptor();
       String collection = cd.getCollectionName();
       if (collection != null) {
         ZkSolrResourceLoader zkLoader = (ZkSolrResourceLoader) managedIndexSchema.getResourceLoader();
-        long timeLeftSecs = timeout -   TimeUnit.SECONDS.convert(System.nanoTime() - startTime, TimeUnit.NANOSECONDS);
-        if(timeLeftSecs<=0) throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Not enough time left to update replicas. However the schema is updated already");
+        long timeLeftSecs = timeout - TimeUnit.SECONDS.convert(System.nanoTime() - startTime, TimeUnit.NANOSECONDS);
+        if (timeLeftSecs <= 0) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+              "Not enough time left to update replicas. However, the schema is updated already.");
+        }
         ManagedIndexSchema.waitForSchemaZkVersionAgreement(collection,
             cd.getCloudDescriptor().getCoreNodeName(),
             (managedIndexSchema).getSchemaZkVersion(),
             zkLoader.getZkController(),
             (int) timeLeftSecs);
       }
-
     }
   }
 
-  private boolean applyAddType(CommandOperation op) {
-    String name = op.getStr(NAME);
-    String clz = op.getStr(CLASS_NAME);
-    if(op.hasError())
-      return false;
-    try {
-      FieldType fieldType = managedIndexSchema.newFieldType(name, clz, op.getDataMap());
-      managedIndexSchema = managedIndexSchema.addFieldTypes(singletonList(fieldType), false);
-      return true;
-    } catch (Exception e) {
-      op.addError(getErrorStr(e));
-      return false;
+  public enum OpType {
+    ADD_FIELD_TYPE("add-field-type") {
+      @Override public boolean perform(CommandOperation op, SchemaManager mgr) {
+        String name = op.getStr(NAME);
+        String className = op.getStr(CLASS_NAME);
+        if (op.hasError())
+          return false;
+        try {
+          FieldType fieldType = mgr.managedIndexSchema.newFieldType(name, className, op.getDataMap());
+          mgr.managedIndexSchema = mgr.managedIndexSchema.addFieldTypes(singletonList(fieldType), false);
+          return true;
+        } catch (Exception e) {
+          op.addError(getErrorStr(e));
+          return false;
+        }
+      }
+    },
+    ADD_COPY_FIELD("add-copy-field") {
+      @Override public boolean perform(CommandOperation op, SchemaManager mgr) {
+        String src  = op.getStr(SOURCE);
+        List<String> dests = op.getStrs(DESTINATION);
+        if (op.hasError())
+          return false;
+        if ( ! op.getValuesExcluding(SOURCE, DESTINATION).isEmpty()) {
+          op.addError("Only the '" + SOURCE + "' and '" + DESTINATION
+              + "' params are allowed with the 'add-copy-field' operation");
+          return false;
+        }
+        try {
+          mgr.managedIndexSchema = mgr.managedIndexSchema.addCopyFields(singletonMap(src, dests), false);
+          return true;
+        } catch (Exception e) {
+          op.addError(getErrorStr(e));
+          return false;
+        }
+      }
+    },
+    ADD_FIELD("add-field") {
+      @Override public boolean perform(CommandOperation op, SchemaManager mgr) {
+        String name = op.getStr(NAME);
+        String type = op.getStr(TYPE);
+        if (op.hasError())
+          return false;
+        FieldType ft = mgr.managedIndexSchema.getFieldTypeByName(type);
+        if (ft == null) {
+          op.addError("No such field type '" + type + "'");
+          return false;
+        }
+        try {
+          SchemaField field = SchemaField.create(name, ft, op.getValuesExcluding(NAME, TYPE));
+          mgr.managedIndexSchema 
+              = mgr.managedIndexSchema.addFields(singletonList(field), Collections.emptyMap(), false);
+          return true;
+        } catch (Exception e) {
+          op.addError(getErrorStr(e));
+          return false;
+        }
+      }
+    },
+    ADD_DYNAMIC_FIELD("add-dynamic-field") {
+      @Override public boolean perform(CommandOperation op, SchemaManager mgr) {
+        String name = op.getStr(NAME);
+        String type = op.getStr(TYPE);
+        if (op.hasError())
+          return false;
+        FieldType ft = mgr.managedIndexSchema.getFieldTypeByName(type);
+        if (ft == null) {
+          op.addError("No such field type '" + type + "'");
+          return  false;
+        }
+        try {
+          SchemaField field = SchemaField.create(name, ft, op.getValuesExcluding(NAME, TYPE)); 
+          mgr.managedIndexSchema 
+              = mgr.managedIndexSchema.addDynamicFields(singletonList(field), Collections.emptyMap(), false);
+          return true;
+        } catch (Exception e) {
+          op.addError(getErrorStr(e));
+          return false;
+        }
+      }
+    },
+    DELETE_FIELD_TYPE("delete-field-type") {
+      @Override public boolean perform(CommandOperation op, SchemaManager mgr) {
+        String name = op.getStr(NAME);
+        if (op.hasError())
+          return false;
+        if ( ! op.getValuesExcluding(NAME).isEmpty()) {
+          op.addError("Only the '" + NAME + "' param is allowed with the 'delete-field-type' operation");
+          return false;
+        }
+        try {
+          mgr.managedIndexSchema = mgr.managedIndexSchema.deleteFieldTypes(singleton(name));
+          return true;
+        } catch (Exception e) {
+          op.addError(getErrorStr(e));
+          return false;
+        }
+      }
+    },
+    DELETE_COPY_FIELD("delete-copy-field") {
+      @Override public boolean perform(CommandOperation op, SchemaManager mgr) {
+        String source = op.getStr(SOURCE);
+        List<String> dests = op.getStrs(DESTINATION);
+        if (op.hasError())
+          return false;
+        if ( ! op.getValuesExcluding(SOURCE, DESTINATION).isEmpty()) {
+          op.addError("Only the '" + SOURCE + "' and '" + DESTINATION 
+              + "' params are allowed with the 'delete-copy-field' operation");
+          return false;
+        }
+        try {
+          mgr.managedIndexSchema = mgr.managedIndexSchema.deleteCopyFields(singletonMap(source, dests));
+          return true;
+        } catch (Exception e) {
+          op.addError(getErrorStr(e));
+          return false;
+        }
+      }
+    },
+    DELETE_FIELD("delete-field") {
+      @Override public boolean perform(CommandOperation op, SchemaManager mgr) {
+        String name = op.getStr(NAME);
+        if (op.hasError())
+          return false;
+        if ( ! op.getValuesExcluding(NAME).isEmpty()) {
+          op.addError("Only the '" + NAME + "' param is allowed with the 'delete-field' operation");
+          return false;
+        }                                                            
+        try {
+          mgr.managedIndexSchema = mgr.managedIndexSchema.deleteFields(singleton(name));
+          return true;
+        } catch (Exception e) {
+          op.addError(getErrorStr(e));
+          return false;
+        }                                                             
+      }
+    },
+    DELETE_DYNAMIC_FIELD("delete-dynamic-field") {
+      @Override public boolean perform(CommandOperation op, SchemaManager mgr) {
+        String name = op.getStr(NAME);
+        if (op.hasError())
+          return false;
+        if ( ! op.getValuesExcluding(NAME).isEmpty()) {
+          op.addError("Only the '" + NAME + "' param is allowed with the 'delete-dynamic-field' operation");
+          return false;
+        }
+        try {
+          mgr.managedIndexSchema = mgr.managedIndexSchema.deleteDynamicFields(singleton(name));
+          return true;
+        } catch (Exception e) {
+          op.addError(getErrorStr(e));
+          return false;
+        }
+      }
+    },
+    REPLACE_FIELD_TYPE("replace-field-type") {
+      @Override public boolean perform(CommandOperation op, SchemaManager mgr) {
+        String name = op.getStr(NAME);
+        String className = op.getStr(CLASS_NAME);
+        if (op.hasError())
+          return false;
+        try {
+          mgr.managedIndexSchema = mgr.managedIndexSchema.replaceFieldType(name, className, op.getDataMap());
+          return true;
+        } catch (Exception e) {
+          op.addError(getErrorStr(e));
+          return false;
+        }
+      }
+    },
+    REPLACE_FIELD("replace-field") {
+      @Override public boolean perform(CommandOperation op, SchemaManager mgr) {
+        String name = op.getStr(NAME);
+        String type = op.getStr(TYPE);
+        if (op.hasError())
+          return false;
+        FieldType ft = mgr.managedIndexSchema.getFieldTypeByName(type);
+        if (ft == null) {
+          op.addError("No such field type '" + type + "'");
+          return false;
+        }
+        try {
+          mgr.managedIndexSchema = mgr.managedIndexSchema.replaceField(name, ft, op.getValuesExcluding(NAME, TYPE));
+          return true;
+        } catch (Exception e) {
+          op.addError(getErrorStr(e));
+          return false;
+        }
+      }
+    },
+    REPLACE_DYNAMIC_FIELD("replace-dynamic-field") {
+      @Override public boolean perform(CommandOperation op, SchemaManager mgr) {
+        String name = op.getStr(NAME);
+        String type = op.getStr(TYPE);
+        if (op.hasError())
+          return false;
+        FieldType ft = mgr.managedIndexSchema.getFieldTypeByName(type);
+        if (ft == null) {
+          op.addError("No such field type '" + type + "'");
+          return  false;
+        }
+        try {
+          mgr.managedIndexSchema = mgr.managedIndexSchema.replaceDynamicField(name, ft, op.getValuesExcluding(NAME, TYPE));
+          return true;
+        } catch (Exception e) {
+          op.addError(getErrorStr(e));
+          return false;
+        }
+      }
+    };
+
+    public abstract boolean perform(CommandOperation op, SchemaManager mgr);
+
+    public static OpType get(String label) {
+      return Nested.OP_TYPES.get(label);
+    }
+
+    private static class Nested { // Initializes contained static map before any enum ctor
+      static final Map<String,OpType> OP_TYPES = new HashMap<>();
+    }
+
+    private OpType(String label) {
+      Nested.OP_TYPES.put(label, this);
     }
   }
 
   public static String getErrorStr(Exception e) {
     StringBuilder sb = new StringBuilder();
-    Throwable cause= e;
-    for(int i =0;i<5;i++) {
+    Throwable cause = e;
+    for (int i = 0 ; i < 5 ; i++) {
       sb.append(cause.getMessage()).append("\n");
-      if(cause.getCause() == null || cause.getCause() == cause) break;
+      if (cause.getCause() == null || cause.getCause() == cause) break;
       cause = cause.getCause();
     }
     return sb.toString();
   }
 
-  private boolean applyAddCopyField(CommandOperation op) {
-    String src  = op.getStr(SOURCE);
-    List<String> dest = op.getStrs(DESTINATION);
-    if(op.hasError())
-      return false;
-    try {
-      managedIndexSchema = managedIndexSchema.addCopyFields(Collections.<String,Collection<String>>singletonMap(src,dest), false);
-      return true;
-    } catch (Exception e) {
-      op.addError(getErrorStr(e));
-      return false;
-    }
-  }
-
-
-  private boolean applyAddField( CommandOperation op) {
-    String name = op.getStr(NAME);
-    String type = op.getStr(TYPE);
-    if(op.hasError())
-      return false;
-    FieldType ft = managedIndexSchema.getFieldTypeByName(type);
-    if(ft==null){
-      op.addError("No such field type '"+type+"'");
-      return  false;
-    }
-    try {
-      if(ADD_DYNAMIC_FIELD.equals(op.name)){
-        managedIndexSchema = managedIndexSchema.addDynamicFields(
-            singletonList(SchemaField.create(name, ft, op.getValuesExcluding(NAME, TYPE))),
-            EMPTY_MAP,false);
-      } else {
-        managedIndexSchema = managedIndexSchema.addFields(
-            singletonList( SchemaField.create(name, ft, op.getValuesExcluding(NAME, TYPE))),
-            EMPTY_MAP,
-            false);
-      }
-    } catch (Exception e) {
-      op.addError(getErrorStr(e));
-      return false;
-    }
-    return true;
-  }
-
   public ManagedIndexSchema getFreshManagedSchema() throws IOException, KeeperException, InterruptedException {
     SolrResourceLoader resourceLoader = req.getCore().getResourceLoader();
     if (resourceLoader instanceof ZkSolrResourceLoader) {
-      ZkSolrResourceLoader loader = (ZkSolrResourceLoader) resourceLoader;
       InputStream in = resourceLoader.openResource(req.getSchema().getResourceName());
       if (in instanceof ZkSolrResourceLoader.ZkByteArrayInputStream) {
         int version = ((ZkSolrResourceLoader.ZkByteArrayInputStream) in).getStat().getVersion();
         log.info("managed schema loaded . version : {} ", version);
-        return new ManagedIndexSchema(req.getCore().getSolrConfig(),
-            req.getSchema().getResourceName() ,new InputSource(in),
-            true,
-            req.getSchema().getResourceName(),
-            version,new Object());
-      }else {
+        return new ManagedIndexSchema
+            (req.getCore().getSolrConfig(), req.getSchema().getResourceName(), new InputSource(in), 
+                true, req.getSchema().getResourceName(), version, req.getSchema().getSchemaUpdateLock());
+      } else {
         return (ManagedIndexSchema) req.getCore().getLatestSchema();
       }
-
     } else {
       return (ManagedIndexSchema) req.getCore().getLatestSchema();
     }
-
-
-
   }
 }

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/schema/ZkIndexSchemaReader.java Tue Mar 31 05:22:40 2015
@@ -103,10 +103,13 @@ public class ZkIndexSchemaReader impleme
       if (expectedZkVersion == -1 || oldSchema.schemaZkVersion < expectedZkVersion) {
         byte[] data = zkClient.getData(managedSchemaPath, watcher, stat, true);
         if (stat.getVersion() != oldSchema.schemaZkVersion) {
-          log.info("Retrieved schema version "+stat.getVersion()+" from ZooKeeper");
+          log.info("Retrieved schema version "+ stat.getVersion() + " from ZooKeeper");
           long start = System.nanoTime();
           InputSource inputSource = new InputSource(new ByteArrayInputStream(data));
-          ManagedIndexSchema newSchema = oldSchema.reloadFields(inputSource, stat.getVersion());
+          String resourceName = managedIndexSchemaFactory.getManagedSchemaResourceName();
+          ManagedIndexSchema newSchema = new ManagedIndexSchema
+              (managedIndexSchemaFactory.getConfig(), resourceName, inputSource, managedIndexSchemaFactory.isMutable(), 
+                  resourceName, stat.getVersion(), oldSchema.getSchemaUpdateLock());
           managedIndexSchemaFactory.setSchema(newSchema);
           long stop = System.nanoTime();
           log.info("Finished refreshing schema in " + TimeUnit.MILLISECONDS.convert(stop - start, TimeUnit.NANOSECONDS) + " ms");

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/BitDocSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/BitDocSet.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/BitDocSet.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/BitDocSet.java Tue Mar 31 05:22:40 2015
@@ -332,11 +332,6 @@ public class BitDocSet extends DocSetBas
           }
 
           @Override
-          public boolean isCacheable() {
-            return true;
-          }
-
-          @Override
           public long ramBytesUsed() {
             return bs.ramBytesUsed();
           }

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/DocSet.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/DocSet.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/DocSet.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/DocSet.java Tue Mar 31 05:22:40 2015
@@ -17,6 +17,8 @@
 
 package org.apache.solr.search;
 
+import java.io.Closeable;
+
 import org.apache.lucene.search.Filter;
 import org.apache.solr.common.SolrException;
 
@@ -30,7 +32,7 @@ import org.apache.solr.common.SolrExcept
  *
  * @since solr 0.9
  */
-public interface DocSet /* extends Collection<Integer> */ {
+public interface DocSet extends Closeable /* extends Collection<Integer> */ {
   
   /**
    * Adds the specified document if it is not currently in the DocSet

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/DocSetBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/DocSetBase.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/DocSetBase.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/DocSetBase.java Tue Mar 31 05:22:40 2015
@@ -17,6 +17,8 @@
 
 package org.apache.solr.search;
 
+import java.io.IOException;
+
 import org.apache.lucene.index.LeafReader;
 import org.apache.lucene.index.LeafReaderContext;
 import org.apache.lucene.search.BitsFilteredDocIdSet;
@@ -212,11 +214,6 @@ abstract class DocSetBase implements Doc
           }
 
           @Override
-          public boolean isCacheable() {
-            return true;
-          }
-
-          @Override
           public long ramBytesUsed() {
             return bs.ramBytesUsed();
           }
@@ -244,4 +241,9 @@ abstract class DocSetBase implements Doc
     }
   }
 
+
+  /** FUTURE: for off-heap */
+  @Override
+  public void close() throws IOException {
+  }
 }

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/FunctionQParser.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/FunctionQParser.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/FunctionQParser.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/FunctionQParser.java Tue Mar 31 05:22:40 2015
@@ -23,6 +23,7 @@ import org.apache.lucene.search.Query;
 import org.apache.solr.common.params.SolrParams;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.schema.SchemaField;
+import org.apache.solr.search.facet.AggValueSource;
 
 import java.util.ArrayList;
 import java.util.HashMap;
@@ -30,6 +31,10 @@ import java.util.List;
 
 public class FunctionQParser extends QParser {
 
+  public static final int FLAG_CONSUME_DELIMITER = 0x01;  // consume delimiter after parsing arg
+  public static final int FLAG_IS_AGG = 0x02;
+  public static final int FLAG_DEFAULT = FLAG_CONSUME_DELIMITER;
+
   /** @lucene.internal */
   public QueryParsing.StrParser sp;
   boolean parseMultipleSources = true;
@@ -37,6 +42,15 @@ public class FunctionQParser extends QPa
 
   public FunctionQParser(String qstr, SolrParams localParams, SolrParams params, SolrQueryRequest req) {
     super(qstr, localParams, params, req);
+    setString(qstr);
+  }
+
+  @Override
+  public void setString(String s) {
+    super.setString(s);
+    if (s != null) {
+      sp = new QueryParsing.StrParser( s );
+    }
   }
 
   public void setParseMultipleSources(boolean parseMultipleSources) {
@@ -59,13 +73,11 @@ public class FunctionQParser extends QPa
 
   @Override
   public Query parse() throws SyntaxError {
-    sp = new QueryParsing.StrParser(getString());
-
     ValueSource vs = null;
     List<ValueSource> lst = null;
 
     for(;;) {
-      ValueSource valsource = parseValueSource(false);
+      ValueSource valsource = parseValueSource(FLAG_DEFAULT & ~FLAG_CONSUME_DELIMITER);
       sp.eatws();
       if (!parseMultipleSources) {
         vs = valsource; 
@@ -210,7 +222,7 @@ public class FunctionQParser extends QPa
   public List<ValueSource> parseValueSourceList() throws SyntaxError {
     List<ValueSource> sources = new ArrayList<>(3);
     while (hasMoreArguments()) {
-      sources.add(parseValueSource(true));
+      sources.add(parseValueSource(FLAG_DEFAULT | FLAG_CONSUME_DELIMITER));
     }
     return sources;
   }
@@ -220,7 +232,7 @@ public class FunctionQParser extends QPa
    */
   public ValueSource parseValueSource() throws SyntaxError {
     /* consume the delimiter afterward for an external call to parseValueSource */
-    return parseValueSource(true);
+    return parseValueSource(FLAG_DEFAULT | FLAG_CONSUME_DELIMITER);
   }
   
   /*
@@ -274,7 +286,11 @@ public class FunctionQParser extends QPa
    * 
    * @param doConsumeDelimiter whether to consume a delimiter following the ValueSource  
    */
-  protected ValueSource parseValueSource(boolean doConsumeDelimiter) throws SyntaxError {
+   protected ValueSource parseValueSource(boolean doConsumeDelimiter) throws SyntaxError {
+     return parseValueSource( doConsumeDelimiter ? (FLAG_DEFAULT | FLAG_CONSUME_DELIMITER) : (FLAG_DEFAULT & ~FLAG_CONSUME_DELIMITER) );
+   }
+
+   protected ValueSource parseValueSource(int flags) throws SyntaxError {
     ValueSource valueSource;
     
     int ch = sp.peek();
@@ -365,12 +381,57 @@ public class FunctionQParser extends QPa
 
     }
     
-    if (doConsumeDelimiter)
+    if ((flags & FLAG_CONSUME_DELIMITER) != 0) {
       consumeArgumentDelimiter();
+    }
     
     return valueSource;
   }
 
+  /** @lucene.experimental */
+  public AggValueSource parseAgg(int flags) throws SyntaxError {
+    String id = sp.getId();
+    AggValueSource vs = null;
+    boolean hasParen = false;
+
+    if ("agg".equals(id)) {
+      hasParen = sp.opt("(");
+      vs = parseAgg(flags | FLAG_IS_AGG);
+    } else {
+      // parse as an aggregation...
+      if (!id.startsWith("agg_")) {
+        id = "agg_" + id;
+      }
+
+      hasParen = sp.opt("(");
+
+      ValueSourceParser argParser = req.getCore().getValueSourceParser(id);
+      argParser = req.getCore().getValueSourceParser(id);
+      if (argParser == null) {
+        throw new SyntaxError("Unknown aggregation " + id + " in (" + sp + ")");
+      }
+
+      ValueSource vv = argParser.parse(this);
+      if (!(vv instanceof AggValueSource)) {
+        if (argParser == null) {
+          throw new SyntaxError("Expected aggregation from " + id + " but got (" + vv + ") in (" + sp + ")");
+        }
+      }
+      vs = (AggValueSource) vv;
+    }
+
+    if (hasParen) {
+      sp.expect(")");
+    }
+
+    if ((flags & FLAG_CONSUME_DELIMITER) != 0) {
+      consumeArgumentDelimiter();
+    }
+
+    return vs;
+  }
+
+
   /**
    * Consume an argument delimiter (a comma) from the token stream.
    * Only consumes if more arguments should exist (no ending parens or end of string).

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/Insanity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/Insanity.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/Insanity.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/Insanity.java Tue Mar 31 05:22:40 2015
@@ -66,7 +66,7 @@ public class Insanity {
       for (FieldInfo fi : in.getFieldInfos()) {
         if (fi.name.equals(insaneField)) {
           filteredInfos.add(new FieldInfo(fi.name, fi.number, fi.hasVectors(), fi.omitsNorms(),
-                                          fi.hasPayloads(), fi.getIndexOptions(), DocValuesType.NONE, -1, null));
+                                          fi.hasPayloads(), fi.getIndexOptions(), DocValuesType.NONE, -1, Collections.emptyMap()));
         } else {
           filteredInfos.add(fi);
         }

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/LuceneQueryOptimizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/LuceneQueryOptimizer.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/LuceneQueryOptimizer.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/LuceneQueryOptimizer.java Tue Mar 31 05:22:40 2015
@@ -97,7 +97,7 @@ if (c.query instanceof TermQuery) {
         filter = (Filter)cache.get(filterQuery);
       }
       if (filter == null) {                       // miss
-        filter = new CachingWrapperFilter(new QueryWrapperFilter(filterQuery)); // construct new entry
+        filter = new QueryWrapperFilter(new CachingWrapperQuery(filterQuery)); // construct new entry
         synchronized (cache) {
           cache.put(filterQuery, filter);         // cache it
         }

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/QParserPlugin.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/QParserPlugin.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/QParserPlugin.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/QParserPlugin.java Tue Mar 31 05:22:40 2015
@@ -26,6 +26,9 @@ import org.apache.solr.search.mlt.MLTQPa
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;
 
 import java.net.URL;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
 
 public abstract class QParserPlugin implements NamedListInitializedPlugin, SolrInfoMBean {
   /** internal use - name of the default parser */
@@ -38,35 +41,40 @@ public abstract class QParserPlugin impl
    * This result to NPE during initialization.
    * For every plugin, listed here, NAME field has to be final and static.
    */
-  public static final Object[] standardPlugins = {
-    LuceneQParserPlugin.NAME, LuceneQParserPlugin.class,
-    OldLuceneQParserPlugin.NAME, OldLuceneQParserPlugin.class,
-    FunctionQParserPlugin.NAME, FunctionQParserPlugin.class,
-    PrefixQParserPlugin.NAME, PrefixQParserPlugin.class,
-    BoostQParserPlugin.NAME, BoostQParserPlugin.class,
-    DisMaxQParserPlugin.NAME, DisMaxQParserPlugin.class,
-    ExtendedDismaxQParserPlugin.NAME, ExtendedDismaxQParserPlugin.class,
-    FieldQParserPlugin.NAME, FieldQParserPlugin.class,
-    RawQParserPlugin.NAME, RawQParserPlugin.class,
-    TermQParserPlugin.NAME, TermQParserPlugin.class,
-    TermsQParserPlugin.NAME, TermsQParserPlugin.class,
-    NestedQParserPlugin.NAME, NestedQParserPlugin.class,
-    FunctionRangeQParserPlugin.NAME, FunctionRangeQParserPlugin.class,
-    SpatialFilterQParserPlugin.NAME, SpatialFilterQParserPlugin.class,
-    SpatialBoxQParserPlugin.NAME, SpatialBoxQParserPlugin.class,
-    JoinQParserPlugin.NAME, JoinQParserPlugin.class,
-    SurroundQParserPlugin.NAME, SurroundQParserPlugin.class,
-    SwitchQParserPlugin.NAME, SwitchQParserPlugin.class,
-    MaxScoreQParserPlugin.NAME, MaxScoreQParserPlugin.class,
-    BlockJoinParentQParserPlugin.NAME, BlockJoinParentQParserPlugin.class,
-    BlockJoinChildQParserPlugin.NAME, BlockJoinChildQParserPlugin.class,
-    CollapsingQParserPlugin.NAME, CollapsingQParserPlugin.class,
-    SimpleQParserPlugin.NAME, SimpleQParserPlugin.class,
-    ComplexPhraseQParserPlugin.NAME, ComplexPhraseQParserPlugin.class,
-    ReRankQParserPlugin.NAME, ReRankQParserPlugin.class,
-    ExportQParserPlugin.NAME, ExportQParserPlugin.class,
-    MLTQParserPlugin.NAME, MLTQParserPlugin.class
-  };
+  public static final Map<String, Class<? extends QParserPlugin>> standardPlugins;
+
+  static {
+    HashMap<String, Class<? extends QParserPlugin>> map = new HashMap<>();
+    map.put(LuceneQParserPlugin.NAME, LuceneQParserPlugin.class);
+    map.put(OldLuceneQParserPlugin.NAME, OldLuceneQParserPlugin.class);
+    map.put(FunctionQParserPlugin.NAME, FunctionQParserPlugin.class);
+    map.put(PrefixQParserPlugin.NAME, PrefixQParserPlugin.class);
+    map.put(BoostQParserPlugin.NAME, BoostQParserPlugin.class);
+    map.put(DisMaxQParserPlugin.NAME, DisMaxQParserPlugin.class);
+    map.put(ExtendedDismaxQParserPlugin.NAME, ExtendedDismaxQParserPlugin.class);
+    map.put(FieldQParserPlugin.NAME, FieldQParserPlugin.class);
+    map.put(RawQParserPlugin.NAME, RawQParserPlugin.class);
+    map.put(TermQParserPlugin.NAME, TermQParserPlugin.class);
+    map.put(TermsQParserPlugin.NAME, TermsQParserPlugin.class);
+    map.put(NestedQParserPlugin.NAME, NestedQParserPlugin.class);
+    map.put(FunctionRangeQParserPlugin.NAME, FunctionRangeQParserPlugin.class);
+    map.put(SpatialFilterQParserPlugin.NAME, SpatialFilterQParserPlugin.class);
+    map.put(SpatialBoxQParserPlugin.NAME, SpatialBoxQParserPlugin.class);
+    map.put(JoinQParserPlugin.NAME, JoinQParserPlugin.class);
+    map.put(SurroundQParserPlugin.NAME, SurroundQParserPlugin.class);
+    map.put(SwitchQParserPlugin.NAME, SwitchQParserPlugin.class);
+    map.put(MaxScoreQParserPlugin.NAME, MaxScoreQParserPlugin.class);
+    map.put(BlockJoinParentQParserPlugin.NAME, BlockJoinParentQParserPlugin.class);
+    map.put(BlockJoinChildQParserPlugin.NAME, BlockJoinChildQParserPlugin.class);
+    map.put(CollapsingQParserPlugin.NAME, CollapsingQParserPlugin.class);
+    map.put(SimpleQParserPlugin.NAME, SimpleQParserPlugin.class);
+    map.put(ComplexPhraseQParserPlugin.NAME, ComplexPhraseQParserPlugin.class);
+    map.put(ReRankQParserPlugin.NAME, ReRankQParserPlugin.class);
+    map.put(ExportQParserPlugin.NAME, ExportQParserPlugin.class);
+    map.put(MLTQParserPlugin.NAME, MLTQParserPlugin.class);
+    map.put(HashQParserPlugin.NAME, HashQParserPlugin.class);
+    standardPlugins = Collections.unmodifiableMap(map);
+  }
 
   /** return a {@link QParser} */
   public abstract QParser createParser(String qstr, SolrParams localParams, SolrParams params, SolrQueryRequest req);

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/QueryParsing.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/QueryParsing.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/QueryParsing.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/QueryParsing.java Tue Mar 31 05:22:40 2015
@@ -593,9 +593,9 @@ public class QueryParsing {
    * <b>Note: This API is experimental and may change in non backward-compatible ways in the future</b>
    */
   public static class StrParser {
-    String val;
-    int pos;
-    int end;
+    public String val;
+    public int pos;
+    public int end;
 
     public StrParser(String val) {
       this(val, 0, val.length());
@@ -607,19 +607,19 @@ public class QueryParsing {
       this.end = end;
     }
 
-    void eatws() {
+    public void eatws() {
       while (pos < end && Character.isWhitespace(val.charAt(pos))) pos++;
     }
 
-    char ch() {
+    public char ch() {
       return pos < end ? val.charAt(pos) : 0;
     }
 
-    void skip(int nChars) {
+    public void skip(int nChars) {
       pos = Math.max(pos + nChars, end);
     }
 
-    boolean opt(String s) {
+    public boolean opt(String s) {
       eatws();
       int slen = s.length();
       if (val.regionMatches(pos, s, 0, slen)) {
@@ -629,7 +629,7 @@ public class QueryParsing {
       return false;
     }
 
-    boolean opt(char ch) {
+    public boolean opt(char ch) {
       eatws();
       if (pos < end && val.charAt(pos) == ch) {
         pos++;
@@ -639,7 +639,7 @@ public class QueryParsing {
     }
 
 
-    void expect(String s) throws SyntaxError {
+    public void expect(String s) throws SyntaxError {
       eatws();
       int slen = s.length();
       if (val.regionMatches(pos, s, 0, slen)) {
@@ -649,7 +649,7 @@ public class QueryParsing {
       }
     }
 
-    float getFloat() {
+    public float getFloat() {
       eatws();
       char[] arr = new char[end - pos];
       int i;
@@ -669,7 +669,7 @@ public class QueryParsing {
       return Float.parseFloat(new String(arr, 0, i));
     }
 
-    Number getNumber() {
+    public Number getNumber() {
       eatws();
       int start = pos;
       boolean flt = false;
@@ -694,7 +694,7 @@ public class QueryParsing {
       }
     }
 
-    double getDouble() {
+    public double getDouble() {
       eatws();
       char[] arr = new char[end - pos];
       int i;
@@ -714,7 +714,7 @@ public class QueryParsing {
       return Double.parseDouble(new String(arr, 0, i));
     }
 
-    int getInt() {
+    public int getInt() {
       eatws();
       char[] arr = new char[end - pos];
       int i;
@@ -734,11 +734,11 @@ public class QueryParsing {
     }
 
 
-    String getId() throws SyntaxError {
+    public String getId() throws SyntaxError {
       return getId("Expected identifier");
     }
 
-    String getId(String errMessage) throws SyntaxError {
+    public String getId(String errMessage) throws SyntaxError {
       eatws();
       int id_start = pos;
       char ch;
@@ -787,7 +787,7 @@ public class QueryParsing {
      * Skips leading whitespace and returns whatever sequence of non 
      * whitespace it can find (or hte empty string)
      */
-    String getSimpleString() {
+    public String getSimpleString() {
       eatws();
       int startPos = pos;
       char ch;
@@ -804,7 +804,7 @@ public class QueryParsing {
      * sort direction. (True is desc, False is asc).  
      * Position is advanced to after the comma (or end) when result is non null 
      */
-    Boolean getSortDirection() throws SyntaxError {
+    public Boolean getSortDirection() throws SyntaxError {
       final int startPos = pos;
       final String order = getId(null);
 
@@ -835,7 +835,7 @@ public class QueryParsing {
     }
 
     // return null if not a string
-    String getQuotedString() throws SyntaxError {
+    public String getQuotedString() throws SyntaxError {
       eatws();
       char delim = peekChar();
       if (!(delim == '\"' || delim == '\'')) {
@@ -888,13 +888,13 @@ public class QueryParsing {
     }
 
     // next non-whitespace char
-    char peek() {
+    public char peek() {
       eatws();
       return pos < end ? val.charAt(pos) : 0;
     }
 
     // next char
-    char peekChar() {
+    public char peekChar() {
       return pos < end ? val.charAt(pos) : 0;
     }
 

Modified: lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java?rev=1670257&r1=1670256&r2=1670257&view=diff
==============================================================================
--- lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java (original)
+++ lucene/dev/branches/lucene6271/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java Tue Mar 31 05:22:40 2015
@@ -67,20 +67,20 @@ import org.apache.lucene.uninverting.Uni
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
 import org.apache.lucene.util.FixedBitSet;
-import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrException.ErrorCode;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.ModifiableSolrParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.common.util.SimpleOrderedMap;
-import org.apache.solr.core.DirectoryFactory;
 import org.apache.solr.core.DirectoryFactory.DirContext;
+import org.apache.solr.core.DirectoryFactory;
 import org.apache.solr.core.SolrConfig;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrInfoMBean;
 import org.apache.solr.request.LocalSolrQueryRequest;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.request.SolrRequestInfo;
-import org.apache.solr.request.UnInvertedField;
+import org.apache.solr.search.facet.UnInvertedField;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.schema.SchemaField;
@@ -167,7 +167,7 @@ public class SolrIndexSearcher extends I
   
   // TODO: wrap elsewhere and return a "map" from the schema that overrides get() ?
   // this reader supports reopen
-  private static DirectoryReader wrapReader(SolrCore core, DirectoryReader reader) {
+  private static DirectoryReader wrapReader(SolrCore core, DirectoryReader reader) throws IOException {
     assert reader != null;
     return ExitableDirectoryReader.wrap
         (UninvertingReader.wrap(reader, core.getLatestSchema().getUninversionMap(reader)), 
@@ -301,6 +301,9 @@ public class SolrIndexSearcher extends I
       fieldNames.add(fieldInfo.name);
     }
 
+    // We already have our own filter cache
+    setQueryCache(null);
+
     // do this at the end since an exception in the constructor means we won't close    
     numOpens.incrementAndGet();
   }
@@ -772,7 +775,7 @@ public class SolrIndexSearcher extends I
   ////////////////////////////////////////////////////////////////////////////////
 
   /** expert: internal API, subject to change */
-  public SolrCache<String,UnInvertedField> getFieldValueCache() {
+  public SolrCache<String, org.apache.solr.search.facet.UnInvertedField> getFieldValueCache() {
     return fieldValueCache;
   }