You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by sa...@apache.org on 2014/07/15 18:27:42 UTC

svn commit: r1610743 - in /lucene/dev/branches/branch_4x: ./ solr/ solr/core/ solr/core/src/java/org/apache/solr/schema/ solr/core/src/java/org/apache/solr/update/processor/ solr/core/src/test-files/solr/collection1/conf/ solr/core/src/test/org/apache/...

Author: sarowe
Date: Tue Jul 15 16:27:42 2014
New Revision: 1610743

URL: http://svn.apache.org/r1610743
Log:
SOLR-6137: Schemaless concurrency improvements:
- Fixed an NPE when reloading a managed schema with no dynamic copy fields
- Moved parsing and schema fields addition to after the distributed phase
- AddSchemaFieldsUpdateProcessor now uses a fixed schema rather than always retrieving the latest, and holds the schema update lock through the entire schema swap-out process
(merged trunk r1610725 and r1610726)

Added:
    lucene/dev/branches/branch_4x/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml   (props changed)
      - copied unchanged from r1610725, lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-schemaless.xml
    lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/schema/TestCloudSchemaless.java   (props changed)
      - copied unchanged from r1610725, lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestCloudSchemaless.java
Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/solr/   (props changed)
    lucene/dev/branches/branch_4x/solr/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_4x/solr/core/   (props changed)
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java
    lucene/dev/branches/branch_4x/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml
    lucene/dev/branches/branch_4x/solr/example/   (props changed)
    lucene/dev/branches/branch_4x/solr/example/example-schemaless/solr/collection1/   (props changed)
    lucene/dev/branches/branch_4x/solr/example/example-schemaless/solr/collection1/conf/   (props changed)
    lucene/dev/branches/branch_4x/solr/example/example-schemaless/solr/collection1/conf/solrconfig.xml

Modified: lucene/dev/branches/branch_4x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/CHANGES.txt?rev=1610743&r1=1610742&r2=1610743&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/solr/CHANGES.txt Tue Jul 15 16:27:42 2014
@@ -104,6 +104,14 @@ Bug Fixes
 
 * SOLR-6245: Socket and Connection configuration are ignored in HttpSolrServer when passing in HttpClient.
   (Patanachai Tangchaisin, shalin)
+  
+* SOLR-6137: Schemaless concurrency improvements:
+  - Fixed an NPE when reloading a managed schema with no dynamic copy fields
+  - Moved parsing and schema fields addition to after the distributed phase
+  - AddSchemaFieldsUpdateProcessor now uses a fixed schema rather than always
+    retrieving the latest, and holds the schema update lock through the entire
+    schema swap-out process
+  (Gregory Chanan via Steve Rowe)
 
 Optimizations
 ---------------------

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java?rev=1610743&r1=1610742&r2=1610743&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/schema/ManagedIndexSchema.java Tue Jul 15 16:27:42 2014
@@ -336,7 +336,7 @@ public final class ManagedIndexSchema ex
       // create new copyField-related objects so we don't affect the
       // old schema
       newSchema.copyFieldsMap = new HashMap<>();
-      newSchema.dynamicCopyFields = null;
+      newSchema.dynamicCopyFields = new DynamicCopy[] {};
       newSchema.copyFieldTargetCounts = new HashMap<>();
       newSchema.loadCopyFields(document, xpath);
       if (null != uniqueKeyField) {

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java?rev=1610743&r1=1610742&r2=1610743&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/processor/AddSchemaFieldsUpdateProcessorFactory.java Tue Jul 15 16:27:42 2014
@@ -22,6 +22,7 @@ import org.apache.solr.common.SolrInputD
 import org.apache.solr.common.SolrInputField;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.response.SolrQueryResponse;
 import org.apache.solr.schema.IndexSchema;
@@ -127,14 +128,9 @@ public class AddSchemaFieldsUpdateProces
   private List<TypeMapping> typeMappings = Collections.emptyList();
   private SelectorParams inclusions = new SelectorParams();
   private Collection<SelectorParams> exclusions = new ArrayList<>();
-  private FieldNameSelector selector = null;
+  private SolrResourceLoader solrResourceLoader = null;
   private String defaultFieldType;
 
-  protected final FieldMutatingUpdateProcessor.FieldNameSelector getSelector() {
-    if (null != selector) return selector;
-    throw new SolrException(SERVER_ERROR, "selector was never initialized, inform(SolrCore) never called???");
-  }
-
   @Override
   public UpdateRequestProcessor getInstance(SolrQueryRequest req, 
                                             SolrQueryResponse rsp, 
@@ -168,28 +164,13 @@ public class AddSchemaFieldsUpdateProces
 
   @Override
   public void inform(SolrCore core) {
-    selector = FieldMutatingUpdateProcessor.createFieldNameSelector
-        (core.getResourceLoader(), core, inclusions, getDefaultSelector(core));
-
-    for (SelectorParams exc : exclusions) {
-      selector = FieldMutatingUpdateProcessor.wrap(selector, FieldMutatingUpdateProcessor.createFieldNameSelector
-          (core.getResourceLoader(), core, exc, FieldMutatingUpdateProcessor.SELECT_NO_FIELDS));
-    }
+    solrResourceLoader = core.getResourceLoader();
 
     for (TypeMapping typeMapping : typeMappings) {
       typeMapping.populateValueClasses(core);
     }
   }
 
-  private FieldNameSelector getDefaultSelector(final SolrCore core) {
-    return new FieldNameSelector() {
-      @Override
-      public boolean shouldMutate(final String fieldName) {
-        return null == core.getLatestSchema().getFieldTypeNoEx(fieldName);
-      }
-    };
-  }
-
   private static List<TypeMapping> parseTypeMappings(NamedList args) {
     List<TypeMapping> typeMappings = new ArrayList<>();
     List<Object> typeMappingsParams = args.getAll(TYPE_MAPPING_PARAM);
@@ -281,17 +262,23 @@ public class AddSchemaFieldsUpdateProces
     
     @Override
     public void processAdd(AddUpdateCommand cmd) throws IOException {
-      if ( ! cmd.getReq().getCore().getLatestSchema().isMutable()) {
+      if ( ! cmd.getReq().getSchema().isMutable()) {
         final String message = "This IndexSchema is not mutable.";
         throw new SolrException(BAD_REQUEST, message);
       }
       final SolrInputDocument doc = cmd.getSolrInputDocument();
       final SolrCore core = cmd.getReq().getCore();
+      // use the cmd's schema rather than the latest, because the schema
+      // can be updated during processing.  Using the cmd's schema guarantees
+      // this will be detected and the cmd's schema updated.
+      IndexSchema oldSchema = cmd.getReq().getSchema();
       for (;;) {
-        final IndexSchema oldSchema = core.getLatestSchema();
         List<SchemaField> newFields = new ArrayList<>();
+        // build a selector each time through the loop b/c the schema we are
+        // processing may have changed
+        FieldNameSelector selector = buildSelector(oldSchema);
         for (final String fieldName : doc.getFieldNames()) {
-          if (selector.shouldMutate(fieldName)) { // returns false if the field already exists in the latest schema
+          if (selector.shouldMutate(fieldName)) { // returns false if the field already exists in the current schema
             String fieldTypeName = mapValueClassesToFieldType(doc.getField(fieldName));
             newFields.add(oldSchema.newField(fieldName, fieldTypeName, Collections.<String,Object>emptyMap()));
           }
@@ -314,28 +301,32 @@ public class AddSchemaFieldsUpdateProces
           builder.append("]");
           log.debug(builder.toString());
         }
-        try {
-          synchronized (oldSchema.getSchemaUpdateLock()) {
+        // Need to hold the lock during the entire attempt to ensure that
+        // the schema on the request is the latest
+        synchronized (oldSchema.getSchemaUpdateLock()) {
+          try {
             IndexSchema newSchema = oldSchema.addFields(newFields);
             if (null != newSchema) {
-              cmd.getReq().getCore().setLatestSchema(newSchema);
+              core.setLatestSchema(newSchema);
               cmd.getReq().updateSchemaToLatest();
               log.debug("Successfully added field(s) to the schema.");
               break; // success - exit from the retry loop
             } else {
               throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, "Failed to add fields.");
             }
+          } catch (ManagedIndexSchema.FieldExistsException e) {
+            log.error("At least one field to be added already exists in the schema - retrying.");
+            oldSchema = core.getLatestSchema();
+            cmd.getReq().updateSchemaToLatest();
+          } catch (ManagedIndexSchema.SchemaChangedInZkException e) {
+            log.debug("Schema changed while processing request - retrying.");
+            oldSchema = core.getLatestSchema();
+            cmd.getReq().updateSchemaToLatest();
           }
-        } catch(ManagedIndexSchema.FieldExistsException e) {
-          log.debug("At least one field to be added already exists in the schema - retrying.");
-          // No action: at least one field to be added already exists in the schema, so retry 
-          // We should never get here, since selector.shouldMutate(field) will exclude already existing fields
-        } catch(ManagedIndexSchema.SchemaChangedInZkException e) {
-          log.debug("Schema changed while processing request - retrying.");
         }
       }
       super.processAdd(cmd);
-    }                          
+    }
 
     private String mapValueClassesToFieldType(SolrInputField field) {
       NEXT_TYPE_MAPPING: for (TypeMapping typeMapping : typeMappings) {
@@ -354,5 +345,25 @@ public class AddSchemaFieldsUpdateProces
       // At least one of this field's values is not an instance of any configured fieldType's valueClass-s
       return defaultFieldType;
     }
+
+    private FieldNameSelector getDefaultSelector(final IndexSchema schema) {
+      return new FieldNameSelector() {
+        @Override
+        public boolean shouldMutate(final String fieldName) {
+          return null == schema.getFieldTypeNoEx(fieldName);
+        }
+      };
+    }
+
+    private FieldNameSelector buildSelector(IndexSchema schema) {
+      FieldNameSelector selector = FieldMutatingUpdateProcessor.createFieldNameSelector
+        (solrResourceLoader, schema, inclusions, getDefaultSelector(schema));
+
+      for (SelectorParams exc : exclusions) {
+        selector = FieldMutatingUpdateProcessor.wrap(selector, FieldMutatingUpdateProcessor.createFieldNameSelector
+          (solrResourceLoader, schema, exc, FieldMutatingUpdateProcessor.SELECT_NO_FIELDS));
+      }
+      return selector;
+    }
   }
 }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java?rev=1610743&r1=1610742&r2=1610743&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java Tue Jul 15 16:27:42 2014
@@ -32,7 +32,7 @@ import org.apache.solr.common.SolrExcept
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.SolrResourceLoader;
 import org.apache.solr.schema.FieldType;
-
+import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.update.AddUpdateCommand;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -184,37 +184,66 @@ public abstract class FieldMutatingUpdat
 
   /**
    * Utility method that can be used to define a FieldNameSelector
+   * using the same types of rules as the FieldMutatingUpdateProcessor init
+   * code.  This may be useful for Factories that wish to define default
+   * selectors in similar terms to what the configuration would look like.
+   * @lucene.internal
+   */
+  public static FieldNameSelector createFieldNameSelector
+    (final SolrResourceLoader loader,
+     final SolrCore core,
+     final SelectorParams params,
+     final FieldNameSelector defSelector) {
+
+    if (params.noSelectorsSpecified()) {
+      return defSelector;
+    }
+
+    final ConfigurableFieldNameSelectorHelper helper =
+      new ConfigurableFieldNameSelectorHelper(loader, params);
+    return new FieldNameSelector() {
+      @Override
+      public boolean shouldMutate(String fieldName) {
+        return helper.shouldMutateBasedOnSchema(fieldName, core.getLatestSchema());
+      }
+    };
+  }
+
+  /**
+   * Utility method that can be used to define a FieldNameSelector
    * using the same types of rules as the FieldMutatingUpdateProcessor init 
    * code.  This may be useful for Factories that wish to define default 
    * selectors in similar terms to what the configuration would look like.
+   * Uses {@code schema} for checking field existence.
    * @lucene.internal
    */
   public static FieldNameSelector createFieldNameSelector
     (final SolrResourceLoader loader,
-     final SolrCore core,
+     final IndexSchema schema,
      final SelectorParams params,
      final FieldNameSelector defSelector) {
 
     if (params.noSelectorsSpecified()) {
       return defSelector;
     }
-    
-    return new ConfigurableFieldNameSelector(loader, core, params); 
+
+    final ConfigurableFieldNameSelectorHelper helper =
+      new ConfigurableFieldNameSelectorHelper(loader, params);
+    return new FieldNameSelector() {
+      @Override
+      public boolean shouldMutate(String fieldName) {
+        return helper.shouldMutateBasedOnSchema(fieldName, schema);
+      }
+    };
   }
   
-  
-  
-  private static final class ConfigurableFieldNameSelector 
-    implements FieldNameSelector {
+  private static final class ConfigurableFieldNameSelectorHelper {
 
-    final SolrCore core;
     final SelectorParams params;
     final Collection<Class> classes;
 
-    private ConfigurableFieldNameSelector(final SolrResourceLoader loader,
-                                          final SolrCore core,
+    private ConfigurableFieldNameSelectorHelper(final SolrResourceLoader loader,
                                           final SelectorParams params) {
-      this.core = core;
       this.params = params;
 
       final Collection<Class> classes = new ArrayList<>(params.typeClass.size());
@@ -229,9 +258,7 @@ public abstract class FieldMutatingUpdat
       this.classes = classes;
     }
 
-    @Override
-    public boolean shouldMutate(final String fieldName) {
-      
+    public boolean shouldMutateBasedOnSchema(final String fieldName, IndexSchema schema) {
       // order of checks is based on what should be quicker
       // (ie: set lookups faster the looping over instanceOf / matches tests
       
@@ -241,7 +268,7 @@ public abstract class FieldMutatingUpdat
       
       // do not consider it an error if the fieldName has no type
       // there might be another processor dealing with it later
-      FieldType t =  core.getLatestSchema().getFieldTypeNoEx(fieldName);
+      FieldType t =  schema.getFieldTypeNoEx(fieldName);
       final boolean fieldExists = (null != t);
 
       if ( (null != params.fieldNameMatchesSchemaField) &&

Modified: lucene/dev/branches/branch_4x/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml?rev=1610743&r1=1610742&r2=1610743&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/test-files/solr/collection1/conf/solrconfig-add-schema-fields-update-processor-chains.xml Tue Jul 15 16:27:42 2014
@@ -66,6 +66,8 @@
   </updateRequestProcessorChain>
 
   <updateRequestProcessorChain name="add-fields">
+    <processor class="solr.LogUpdateProcessorFactory" />
+    <processor class="solr.DistributedUpdateProcessorFactory" />
     <processor class="solr.AddSchemaFieldsUpdateProcessorFactory">
       <str name="defaultFieldType">text</str>
       <lst name="typeMapping">
@@ -98,6 +100,8 @@
   </updateRequestProcessorChain>
 
   <updateRequestProcessorChain name="parse-and-add-fields">
+    <processor class="solr.LogUpdateProcessorFactory" />
+    <processor class="solr.DistributedUpdateProcessorFactory" />
     <processor class="solr.ParseBooleanFieldUpdateProcessorFactory"/>
     <processor class="solr.ParseLongFieldUpdateProcessorFactory"/>
     <processor class="solr.ParseDoubleFieldUpdateProcessorFactory"/>

Modified: lucene/dev/branches/branch_4x/solr/example/example-schemaless/solr/collection1/conf/solrconfig.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/example/example-schemaless/solr/collection1/conf/solrconfig.xml?rev=1610743&r1=1610742&r2=1610743&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/example/example-schemaless/solr/collection1/conf/solrconfig.xml (original)
+++ lucene/dev/branches/branch_4x/solr/example/example-schemaless/solr/collection1/conf/solrconfig.xml Tue Jul 15 16:27:42 2014
@@ -1563,6 +1563,8 @@
        See http://wiki.apache.org/solr/GuessingFieldTypes
     -->
   <updateRequestProcessorChain name="add-unknown-fields-to-the-schema">
+    <processor class="solr.LogUpdateProcessorFactory"/>
+    <processor class="solr.DistributedUpdateProcessorFactory"/>
     <processor class="solr.RemoveBlankFieldUpdateProcessorFactory"/>
     <processor class="solr.ParseBooleanFieldUpdateProcessorFactory"/>
     <processor class="solr.ParseLongFieldUpdateProcessorFactory"/>
@@ -1608,7 +1610,6 @@
         <str name="fieldType">tdoubles</str>
       </lst>
     </processor>
-    <processor class="solr.LogUpdateProcessorFactory"/>
     <processor class="solr.RunUpdateProcessorFactory"/>
   </updateRequestProcessorChain>