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);
}
}