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/06/10 23:27:46 UTC

svn commit: r1601776 - in /lucene/dev/branches/branch_4x: ./ solr/ solr/core/ solr/core/src/java/org/apache/solr/rest/schema/ solr/core/src/java/org/apache/solr/schema/ solr/core/src/java/org/apache/solr/update/processor/ solr/core/src/test/org/apache/...

Author: sarowe
Date: Tue Jun 10 21:27:45 2014
New Revision: 1601776

URL: http://svn.apache.org/r1601776
Log:
SOLR-6145: Fix Schema API optimistic concurrency by moving it out of ManagedIndexSchema.add(Copy)Fields() into the consumers of those methods: CopyFieldCollectionResource, FieldCollectionResource, FieldResource, and AddSchemaFieldsUpdateProcessorFactory. (merged trunk r1601770 and r1601775)

Added:
    lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaAddFields.java
      - copied unchanged from r1601770, lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaAddFields.java
    lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaCopyFields.java
      - copied unchanged from r1601770, lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaCopyFields.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/rest/schema/CopyFieldCollectionResource.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/rest/schema/FieldCollectionResource.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/rest/schema/FieldResource.java
    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/test/org/apache/solr/schema/TestCloudManagedSchemaAddField.java

Modified: lucene/dev/branches/branch_4x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/CHANGES.txt?rev=1601776&r1=1601775&r2=1601776&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/solr/CHANGES.txt Tue Jun 10 21:27:45 2014
@@ -95,6 +95,12 @@ Bug Fixes
 
 * SOLR-6149: Specifying the query value without any index value does not work in
   Analysis browser. (Aman Tandon, shalin)
+  
+* SOLR-6145: Fix Schema API optimistic concurrency by moving it out of 
+  ManagedIndexSchema.add(Copy)Fields() into the consumers of those methods: 
+  CopyFieldCollectionResource, FieldCollectionResource, FieldResource,
+  and AddSchemaFieldsUpdateProcessorFactory.
+  (Gregory Chanan, Alexey Serba, Steve Rowe)
 
 * SOLR-6146: Incorrect configuration such as wrong chroot in zk server address can
   cause CloudSolrServer to leak resources. (Jessica Cheng, Varun Thacker, shalin)

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/rest/schema/CopyFieldCollectionResource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/rest/schema/CopyFieldCollectionResource.java?rev=1601776&r1=1601775&r2=1601776&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/rest/schema/CopyFieldCollectionResource.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/rest/schema/CopyFieldCollectionResource.java Tue Jun 10 21:27:45 2014
@@ -30,7 +30,6 @@ import org.restlet.resource.ResourceExce
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.io.IOException;
 import java.util.Arrays;
 import java.util.Collection;
 import java.util.Collections;
@@ -40,6 +39,8 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import static org.apache.solr.common.SolrException.ErrorCode;
+
 /**
  * This class responds to requests at /solr/(corename)/schema/copyfields
  * <p/>
@@ -110,13 +111,13 @@ public class CopyFieldCollectionResource
     try {
       if (!getSchema().isMutable()) {
         final String message = "This IndexSchema is not mutable.";
-        throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, message);
+        throw new SolrException(ErrorCode.BAD_REQUEST, message);
       } else {
         if (!entity.getMediaType().equals(MediaType.APPLICATION_JSON, true)) {
           String message = "Only media type " + MediaType.APPLICATION_JSON.toString() + " is accepted."
               + "  Request has media type " + entity.getMediaType().toString() + ".";
           log.error(message);
-          throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, message);
+          throw new SolrException(ErrorCode.BAD_REQUEST, message);
         } else {
           Object object = ObjectBuilder.fromJSON(entity.getText());
 
@@ -124,7 +125,7 @@ public class CopyFieldCollectionResource
             String message = "Invalid JSON type " + object.getClass().getName() + ", expected List of the form"
                 + " (ignore the backslashes): [{\"source\":\"foo\",\"dest\":\"comma-separated list of targets\"}, {...}, ...]";
             log.error(message);
-            throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, message);
+            throw new SolrException(ErrorCode.BAD_REQUEST, message);
           } else {
             List<Map<String, Object>> list = (List<Map<String, Object>>) object;
             Map<String, Collection<String>> fieldsToCopy = new HashMap<>();
@@ -135,7 +136,7 @@ public class CopyFieldCollectionResource
               if (null == fieldName) {
                 String message = "Missing '" + IndexSchema.SOURCE + "' mapping.";
                 log.error(message);
-                throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, message);
+                throw new SolrException(ErrorCode.BAD_REQUEST, message);
               }
               Object dest = map.get(IndexSchema.DESTINATION);
               List<String> destinations = null;
@@ -147,7 +148,7 @@ public class CopyFieldCollectionResource
                 } else {
                   String message = "Invalid '" + IndexSchema.DESTINATION + "' type.";
                   log.error(message);
-                  throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, message);
+                  throw new SolrException(ErrorCode.BAD_REQUEST, message);
                 }
               }
               if (destinations == null) {
@@ -165,11 +166,22 @@ public class CopyFieldCollectionResource
                 message.setLength(message.length() - 2);//drop the last ,
               }
               log.error(message.toString().trim());
-              throw new SolrException(SolrException.ErrorCode.BAD_REQUEST, message.toString().trim());
+              throw new SolrException(ErrorCode.BAD_REQUEST, message.toString().trim());
             }
-            IndexSchema newSchema = oldSchema.addCopyFields(fieldsToCopy);
-            if (newSchema != null) {
-              getSolrCore().setLatestSchema(newSchema);
+            boolean success = false;
+            while (!success) {
+              try {
+                IndexSchema newSchema = oldSchema.addCopyFields(fieldsToCopy);
+                if (null != newSchema) {
+                  getSolrCore().setLatestSchema(newSchema);
+                  success = true;
+                } else {
+                  throw new SolrException(ErrorCode.SERVER_ERROR, "Failed to add fields.");
+                }
+              } catch (ManagedIndexSchema.SchemaChangedInZkException e) {
+                  log.debug("Schema changed while processing request, retrying");
+                  oldSchema = (ManagedIndexSchema)getSolrCore().getLatestSchema();
+              }
             }
           }
         }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/rest/schema/FieldCollectionResource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/rest/schema/FieldCollectionResource.java?rev=1601776&r1=1601775&r2=1601776&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/rest/schema/FieldCollectionResource.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/rest/schema/FieldCollectionResource.java Tue Jun 10 21:27:45 2014
@@ -24,6 +24,7 @@ import org.apache.solr.common.util.Simpl
 import org.apache.solr.rest.GETable;
 import org.apache.solr.rest.POSTable;
 import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.ManagedIndexSchema;
 import org.apache.solr.schema.SchemaField;
 import org.noggit.ObjectBuilder;
 import org.restlet.data.MediaType;
@@ -139,9 +140,9 @@ public class FieldCollectionResource ext
           } else {
             List<Map<String, Object>> list = (List<Map<String, Object>>) object;
             List<SchemaField> newFields = new ArrayList<>();
+            List<NewFieldArguments> newFieldArguments = new ArrayList<>();
             IndexSchema oldSchema = getSchema();
             Map<String, Collection<String>> copyFields = new HashMap<>();
-            Set<String> malformed = new HashSet<>();
             for (Map<String, Object> map : list) {
               String fieldName = (String) map.remove(IndexSchema.NAME);
               if (null == fieldName) {
@@ -174,10 +175,38 @@ public class FieldCollectionResource ext
                 copyFields.put(fieldName, copyTo);
               }
               newFields.add(oldSchema.newField(fieldName, fieldType, map));
+              newFieldArguments.add(new NewFieldArguments(fieldName, fieldType, map));
+            }
+            boolean firstAttempt = true;
+            boolean success = false;
+            while (!success) {
+              try {
+                if (!firstAttempt) {
+                  // If this isn't the first attempt, we must have failed due to
+                  // the schema changing in Zk during optimistic concurrency control.
+                  // In that case, rerun creating the new fields, because they may
+                  // fail now due to changes in the schema.  This behavior is consistent
+                  // with what would happen if we locked the schema and the other schema
+                  // change went first.
+                  newFields.clear();
+                  for (NewFieldArguments args : newFieldArguments) {
+                    newFields.add(oldSchema.newField(
+                      args.getName(), args.getType(), args.getMap()));
+                  }
+                }
+                firstAttempt = false;
+                IndexSchema newSchema = oldSchema.addFields(newFields, copyFields);
+                if (null != newSchema) {
+                  getSolrCore().setLatestSchema(newSchema);
+                  success = true;
+                } else {
+                  throw new SolrException(ErrorCode.SERVER_ERROR, "Failed to add fields.");
+                }
+              } catch (ManagedIndexSchema.SchemaChangedInZkException e) {
+                log.debug("Schema changed while processing request, retrying");
+                oldSchema = getSolrCore().getLatestSchema();
+              }
             }
-            IndexSchema newSchema = oldSchema.addFields(newFields, copyFields);
-
-            getSolrCore().setLatestSchema(newSchema);
           }
         }
       }
@@ -188,4 +217,19 @@ public class FieldCollectionResource ext
 
     return new SolrOutputRepresentation();
   }
+
+  private static class NewFieldArguments {
+    private String name;
+    private String type;
+    Map<String, Object> map;
+    NewFieldArguments(String name, String type, Map<String, Object> map){
+      this.name = name;
+      this.type = type;
+      this.map = map;
+    }
+
+    public String getName() { return name; }
+    public String getType() { return type; }
+    public Map<String, Object> getMap() { return map; }
+  }
 }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/rest/schema/FieldResource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/rest/schema/FieldResource.java?rev=1601776&r1=1601775&r2=1601776&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/rest/schema/FieldResource.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/rest/schema/FieldResource.java Tue Jun 10 21:27:45 2014
@@ -162,9 +162,22 @@ public class FieldResource extends BaseF
                 if (copyFieldNames != null) {
                   map.remove(IndexSchema.COPY_FIELDS);
                 }
-                SchemaField newField = oldSchema.newField(fieldName, fieldType, map);
-                IndexSchema newSchema = oldSchema.addField(newField, copyFieldNames);
-                getSolrCore().setLatestSchema(newSchema);
+                boolean success = false;
+                while (!success) {
+                  try {
+                    SchemaField newField = oldSchema.newField(fieldName, fieldType, map);
+                    IndexSchema newSchema = oldSchema.addField(newField, copyFieldNames);
+                    if (null != newSchema) {
+                      getSolrCore().setLatestSchema(newSchema);
+                      success = true;
+                    } else {
+                      throw new SolrException(ErrorCode.SERVER_ERROR, "Failed to add field.");
+                    }
+                  } catch (ManagedIndexSchema.SchemaChangedInZkException e) {
+                    log.debug("Schema changed while processing request, retrying");
+                    oldSchema = (ManagedIndexSchema)getSolrCore().getLatestSchema();
+                  }
+                }
               }
             }
           }

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=1601776&r1=1601775&r2=1601776&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 Jun 10 21:27:45 2014
@@ -129,6 +129,7 @@ public final class ManagedIndexSchema ex
     final SolrZkClient zkClient = zkController.getZkClient();
     final String managedSchemaPath = zkLoader.getCollectionZkPath() + "/" + managedSchemaResourceName;
     boolean success = true;
+    boolean schemaChangedInZk = false;
     try {
       // Persist the managed schema
       StringWriter writer = new StringWriter();
@@ -151,9 +152,8 @@ public final class ManagedIndexSchema ex
           schemaZkVersion = stat.getVersion();
           log.info("Persisted managed schema at " + managedSchemaPath);
         } catch (KeeperException.BadVersionException e) {
-          log.info("Failed to persist managed schema at " + managedSchemaPath 
-                  + " - version mismatch");
           success = false;
+          schemaChangedInZk = true;
         }
       }
     } catch (Exception e) {
@@ -164,6 +164,12 @@ public final class ManagedIndexSchema ex
       log.error(msg, e);
       throw new SolrException(ErrorCode.SERVER_ERROR, msg, e);
     }
+    if (schemaChangedInZk) {
+      String msg = "Failed to persist managed schema at " + managedSchemaPath
+        + " - version mismatch";
+      log.info(msg);
+      throw new SchemaChangedInZkException(ErrorCode.CONFLICT, msg + ", retry.");
+    }
     return success; 
   }
 
@@ -183,6 +189,12 @@ public final class ManagedIndexSchema ex
     }
   }
 
+  public class SchemaChangedInZkException extends SolrException {
+    public SchemaChangedInZkException(ErrorCode code, String msg) {
+      super(code, msg);
+    }
+  }
+
   @Override
   public ManagedIndexSchema addFields(Collection<SchemaField> newFields) {
     return addFields(newFields, Collections.<String, Collection<String>>emptyMap());
@@ -196,45 +208,44 @@ public final class ManagedIndexSchema ex
       if (copyFieldNames == null){
         copyFieldNames = Collections.emptyMap();
       }
-      while ( ! success) { // optimistic concurrency
-        // even though fields is volatile, we need to synchronize to avoid two addFields
-        // happening concurrently (and ending up missing one of them)
-        synchronized (getSchemaUpdateLock()) {
-          newSchema = shallowCopy(true);
-
-          for (SchemaField newField : newFields) {
-            if (null != newSchema.getFieldOrNull(newField.getName())) {
-              String msg = "Field '" + newField.getName() + "' already exists.";
-              throw new FieldExistsException(ErrorCode.BAD_REQUEST, msg);
-            }
-            newSchema.fields.put(newField.getName(), newField);
+      // even though fields is volatile, we need to synchronize to avoid two addFields
+      // happening concurrently (and ending up missing one of them)
+      synchronized (getSchemaUpdateLock()) {
+        newSchema = shallowCopy(true);
+
+        for (SchemaField newField : newFields) {
+          if (null != newSchema.getFieldOrNull(newField.getName())) {
+            String msg = "Field '" + newField.getName() + "' already exists.";
+            throw new FieldExistsException(ErrorCode.BAD_REQUEST, msg);
+          }
+          newSchema.fields.put(newField.getName(), newField);
 
-            if (null != newField.getDefaultValue()) {
-              log.debug(newField.getName() + " contains default value: " + newField.getDefaultValue());
-              newSchema.fieldsWithDefaultValue.add(newField);
-            }
-            if (newField.isRequired()) {
-              log.debug("{} is required in this schema", newField.getName());
-              newSchema.requiredFields.add(newField);
-            }
-            Collection<String> copyFields = copyFieldNames.get(newField.getName());
-            if (copyFields != null) {
-              for (String copyField : copyFields) {
-                newSchema.registerCopyField(newField.getName(), copyField);
-              }
-            }
+          if (null != newField.getDefaultValue()) {
+            log.debug(newField.getName() + " contains default value: " + newField.getDefaultValue());
+            newSchema.fieldsWithDefaultValue.add(newField);
           }
-          // Run the callbacks on SchemaAware now that everything else is done
-          for (SchemaAware aware : newSchema.schemaAware) {
-            aware.inform(newSchema);
+          if (newField.isRequired()) {
+            log.debug("{} is required in this schema", newField.getName());
+            newSchema.requiredFields.add(newField);
           }
-          newSchema.refreshAnalyzers();
-          success = newSchema.persistManagedSchema(false); // don't just create - update it if it already exists
-          if (success) {
-            log.debug("Added field(s): {}", newFields);
+          Collection<String> copyFields = copyFieldNames.get(newField.getName());
+          if (copyFields != null) {
+            for (String copyField : copyFields) {
+              newSchema.registerCopyField(newField.getName(), copyField);
+            }
           }
         }
-        // release the lock between tries to allow the schema reader to update the schema & schemaZkVersion
+        // Run the callbacks on SchemaAware now that everything else is done
+        for (SchemaAware aware : newSchema.schemaAware) {
+          aware.inform(newSchema);
+        }
+        newSchema.refreshAnalyzers();
+        success = newSchema.persistManagedSchema(false); // don't just create - update it if it already exists
+        if (success) {
+          log.debug("Added field(s): {}", newFields);
+        } else {
+          log.error("Failed to add field(s): {}", newFields);
+        }
       }
     } else {
       String msg = "This ManagedIndexSchema is not mutable.";
@@ -249,29 +260,29 @@ public final class ManagedIndexSchema ex
     ManagedIndexSchema newSchema = null;
     if (isMutable) {
       boolean success = false;
-      while (!success) { // optimistic concurrency
-        // even though fields is volatile, we need to synchronize to avoid two addCopyFields
-        // happening concurrently (and ending up missing one of them)
-        synchronized (getSchemaUpdateLock()) {
-          newSchema = shallowCopy(true);
-          for (Map.Entry<String, Collection<String>> entry : copyFields.entrySet()) {
-            //Key is the name of the field, values are the destinations
+      // even though fields is volatile, we need to synchronize to avoid two addCopyFields
+      // happening concurrently (and ending up missing one of them)
+      synchronized (getSchemaUpdateLock()) {
+        newSchema = shallowCopy(true);
+        for (Map.Entry<String, Collection<String>> entry : copyFields.entrySet()) {
+          //Key is the name of the field, values are the destinations
 
-            for (String destination : entry.getValue()) {
-              newSchema.registerCopyField(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();
-          success = newSchema.persistManagedSchema(false); // don't just create - update it if it already exists
-          if (success) {
-            log.debug("Added copy fields for {} sources", copyFields.size());
+          for (String destination : entry.getValue()) {
+            newSchema.registerCopyField(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();
+        success = newSchema.persistManagedSchema(false); // don't just create - update it if it already exists
+        if (success) {
+          log.debug("Added copy fields for {} sources", copyFields.size());
+        } else {
+          log.error("Failed to add copy fields for {} sources", copyFields.size());
+        }
       }
     }
     return newSchema;

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=1601776&r1=1601775&r2=1601776&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 Jun 10 21:27:45 2014
@@ -291,7 +291,7 @@ public class AddSchemaFieldsUpdateProces
         final IndexSchema oldSchema = core.getLatestSchema();
         List<SchemaField> newFields = new ArrayList<>();
         for (final String fieldName : doc.getFieldNames()) {
-          if (selector.shouldMutate(fieldName)) {
+          if (selector.shouldMutate(fieldName)) { // returns false if the field already exists in the latest schema
             String fieldTypeName = mapValueClassesToFieldType(doc.getField(fieldName));
             newFields.add(oldSchema.newField(fieldName, fieldTypeName, Collections.<String,Object>emptyMap()));
           }
@@ -316,13 +316,20 @@ public class AddSchemaFieldsUpdateProces
         }
         try {
           IndexSchema newSchema = oldSchema.addFields(newFields);
-          cmd.getReq().getCore().setLatestSchema(newSchema);
-          cmd.getReq().updateSchemaToLatest();
-          log.debug("Successfully added field(s) to the schema.");
-          break; // success - exit from the retry loop
+          if (null != newSchema) {
+            cmd.getReq().getCore().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.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);

Modified: lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaAddField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaAddField.java?rev=1601776&r1=1601775&r2=1601776&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaAddField.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/schema/TestCloudManagedSchemaAddField.java Tue Jun 10 21:27:45 2014
@@ -79,11 +79,12 @@ public class TestCloudManagedSchemaAddFi
     }
   }
   
-  @Override                                                                                                                 
+  @Override
   public void doTest() throws Exception {
     setupHarnesses();
     
-    // First. add a bunch of fields, and verify each is present in all shards' schemas
+    // First. add a bunch of fields, but do it fast enough
+    // and verify shards' schemas after all of them are added
     int numFields = 25;
     for (int i = 1 ; i <= numFields ; ++i) {
       RestTestHarness publisher = restTestHarnesses.get(r.nextInt(restTestHarnesses.size()));
@@ -91,69 +92,32 @@ public class TestCloudManagedSchemaAddFi
       final String content = "{\"type\":\"text\",\"stored\":\"false\"}";
       String request = "/schema/fields/" + newFieldName + "?wt=xml";             
       String response = publisher.put(request, content);
-      final long addFieldTime = System.currentTimeMillis(); 
       String result = publisher.validateXPath
           (response, "/response/lst[@name='responseHeader']/int[@name='status'][.='0']");
       if (null != result) {
         fail("PUT REQUEST FAILED: xpath=" + result + "  request=" + request 
             + "  content=" + content + "  response=" + response);
       }
-        
-      int maxAttempts = 40;
-      long retryPauseMillis = 20;
+    }
+    
+    Thread.sleep(100000);
 
+    for (int i = 1 ; i <= numFields ; ++i) {
+      String newFieldName = "newfield" + i;
       for (RestTestHarness client : restTestHarnesses) {
-        boolean stillTrying = true;
-        for (int attemptNum = 1; stillTrying && attemptNum <= maxAttempts ; ++attemptNum) {
-          request = "/schema/fields/" + newFieldName + "?wt=xml";
-          response = client.query(request);
-          long elapsedTimeMillis = System.currentTimeMillis() - addFieldTime;
-          result = client.validateXPath(response,
-                                        "/response/lst[@name='responseHeader']/int[@name='status'][.='0']",
-                                        "/response/lst[@name='field']/str[@name='name'][.='" + newFieldName + "']");
-          if (null == result) {
-            stillTrying = false;
-            if (attemptNum > 1) {
-              log.info("On attempt #" + attemptNum + ", successful request " + request + " against server "
-                      + client.getBaseURL() + " after " + elapsedTimeMillis + " ms");
-            }
-          } else {
-            if (attemptNum == maxAttempts || ! response.contains("Field '" + newFieldName + "' not found.")) {
-              String msg = "QUERY FAILED: xpath=" + result + "  request=" + request + "  response=" + response;
-              if (attemptNum == maxAttempts) {
-                msg = "Max retry count " + maxAttempts + " exceeded after " + elapsedTimeMillis +" ms.  " + msg;
-              }
-              log.error(msg);
-              fail(msg);
-            }
-            Thread.sleep(retryPauseMillis);
+        String request = "/schema/fields/" + newFieldName + "?wt=xml";
+        String response = client.query(request);
+        String result = client.validateXPath(response,
+                                      "/response/lst[@name='responseHeader']/int[@name='status'][.='0']",
+                                      "/response/lst[@name='field']/str[@name='name'][.='" + newFieldName + "']");
+        if (null != result) {
+          if (response.contains("Field '" + newFieldName + "' not found.")) {
+            String msg = "QUERY FAILED: xpath=" + result + "  request=" + request + "  response=" + response;
+            log.error(msg);
+            fail(msg);
           }
         }
       }
     }
-    
-    // Add a doc with one of the newly created fields
-    String fieldName = "newfield" + (r.nextInt(numFields) + 1);
-    
-    int addDocClientNum = r.nextInt(restTestHarnesses.size());
-    RestTestHarness client = restTestHarnesses.get(addDocClientNum);
-    String updateResult = client.validateUpdate(adoc(fieldName, "word1 word2", "id", "88"));
-    assertNull("Error adding a document with field " + fieldName + ": " + updateResult, updateResult);
-    updateResult = client.validateUpdate(BaseTestHarness.commit());
-    assertNull("Error committing: " + updateResult, updateResult);
-    
-    // Query for the newly added doc against a different client
-    int queryDocClientNum = r.nextInt(restTestHarnesses.size());
-    while (queryDocClientNum == addDocClientNum) {
-      queryDocClientNum = r.nextInt(restTestHarnesses.size()); 
-    }
-    client = restTestHarnesses.get(queryDocClientNum);
-    String response = client.query("/select?q=" + fieldName + ":word2");
-    String queryResult = client.validateXPath(response,
-                                              "/response/result[@name='response'][@numFound='1']",
-                                              "count(/response/result[@name='response']/doc/int[@name='id']) = 1",
-                                              "/response/result[@name='response']/doc/int[@name='id'] = '88'");
-    assertNull("Error querying for a document with field " + fieldName + ": " + queryResult
-              + "  response=" + response, queryResult);
   }
 }