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 2013/04/22 16:26:58 UTC

svn commit: r1470539 [5/5] - in /lucene/dev/trunk/solr: ./ contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/ contrib/dataimporthandler-extras/src/test/org/apache/solr/handler/dataimport/ contrib/dataimporthandler/src/java/org/apac...

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestManagedSchema.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestManagedSchema.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestManagedSchema.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/TestManagedSchema.java Mon Apr 22 14:26:55 2013
@@ -17,8 +17,15 @@ package org.apache.solr.schema;
  */
 
 import java.io.File;
+import java.io.FileInputStream;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
 
 import org.apache.commons.io.FileUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.common.params.CoreAdminParams;
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.core.AbstractBadConfigTestBase;
@@ -45,9 +52,14 @@ public class TestManagedSchema extends A
     tmpSolrHome = new File(tmpSolrHomePath).getAbsoluteFile();
     tmpConfDir = new File(tmpSolrHome, confDir);
     File testHomeConfDir = new File(TEST_HOME(), confDir);
+    FileUtils.copyFileToDirectory(new File(testHomeConfDir, "solrconfig-mutable-managed-schema.xml"), tmpConfDir);
     FileUtils.copyFileToDirectory(new File(testHomeConfDir, "solrconfig-managed-schema.xml"), tmpConfDir);
     FileUtils.copyFileToDirectory(new File(testHomeConfDir, "solrconfig-basic.xml"), tmpConfDir);
+    FileUtils.copyFileToDirectory(new File(testHomeConfDir, "schema-one-field-no-dynamic-field.xml"), tmpConfDir);
     FileUtils.copyFileToDirectory(new File(testHomeConfDir, "schema-minimal.xml"), tmpConfDir);
+    FileUtils.copyFileToDirectory(new File(testHomeConfDir, "schema_codec.xml"), tmpConfDir);
+    FileUtils.copyFileToDirectory(new File(testHomeConfDir, "schema-bm25.xml"), tmpConfDir);
+
     // initCore will trigger an upgrade to managed schema, since the solrconfig has
     // <schemaFactory class="ManagedIndexSchemaFactory" ... />
     initCore("solrconfig-managed-schema.xml", "schema-minimal.xml", tmpSolrHome.getPath());
@@ -120,4 +132,255 @@ public class TestManagedSchema extends A
     String collectionSchema = (String)collectionStatus.get(CoreAdminParams.SCHEMA);
     assertEquals("Schema resource name differs from expected name", expectedSchemaResource, collectionSchema);
   }
+
+  public void testAddFieldWhenNotMutable() throws Exception {
+    assertSchemaResource(collection, "managed-schema");
+    String errString = "This ManagedIndexSchema is not mutable.";
+    ignoreException(Pattern.quote(errString));
+    try {
+      IndexSchema oldSchema = h.getCore().getLatestSchema();
+      String fieldName = "new_field";
+      String fieldType = "string";
+      Map<String,?> options = Collections.emptyMap();
+      SchemaField newField = oldSchema.newField(fieldName, fieldType, options);
+      IndexSchema newSchema = oldSchema.addField(newField);
+      h.getCore().setLatestSchema(newSchema);
+      fail();
+    } catch (Exception e) {
+      for (Throwable t = e; t != null; t = t.getCause()) {
+        // short circuit out if we found what we expected
+        if (t.getMessage() != null && -1 != t.getMessage().indexOf(errString)) return;
+      }
+      // otherwise, rethrow it, possibly completely unrelated
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+                              "Unexpected error, expected error matching: " + errString, e);
+    } finally {
+      resetExceptionIgnores();
+    }
+  }
+  
+  public void testAddFieldPersistence() throws Exception {
+    assertSchemaResource(collection, "managed-schema");
+    deleteCore();
+    File managedSchemaFile = new File(tmpConfDir, "managed-schema");
+    assertTrue(managedSchemaFile.delete()); // Delete managed-schema so it won't block parsing a new schema
+    initCore("solrconfig-mutable-managed-schema.xml", "schema-one-field-no-dynamic-field.xml", tmpSolrHome.getPath());
+    
+    assertTrue(managedSchemaFile.exists());
+    String managedSchemaContents = FileUtils.readFileToString(managedSchemaFile, "UTF-8");
+    assertFalse(managedSchemaContents.contains("\"new_field\""));
+    
+    Map<String,Object> options = new HashMap<String,Object>();
+    options.put("stored", "false");
+    IndexSchema oldSchema = h.getCore().getLatestSchema();
+    String fieldName = "new_field";
+    String fieldType = "string";
+    SchemaField newField = oldSchema.newField(fieldName, fieldType, options);
+    IndexSchema newSchema = oldSchema.addField(newField);
+    h.getCore().setLatestSchema(newSchema);
+
+    assertTrue(managedSchemaFile.exists());
+    managedSchemaContents = IOUtils.toString(new FileInputStream(managedSchemaFile), "UTF-8");
+    assertTrue(managedSchemaContents.contains("<field name=\"new_field\" type=\"string\" stored=\"false\"/>"));
+  }
+  
+  public void testAddedFieldIndexableAndQueryable() throws Exception {
+    assertSchemaResource(collection, "managed-schema");
+    deleteCore();
+    File managedSchemaFile = new File(tmpConfDir, "managed-schema");
+    assertTrue(managedSchemaFile.delete()); // Delete managed-schema so it won't block parsing a new schema
+    initCore("solrconfig-mutable-managed-schema.xml", "schema-one-field-no-dynamic-field.xml", tmpSolrHome.getPath());
+
+    assertTrue(managedSchemaFile.exists());
+    String managedSchemaContents = FileUtils.readFileToString(managedSchemaFile, "UTF-8");
+    assertFalse(managedSchemaContents.contains("\"new_field\""));
+
+    clearIndex();
+
+    String errString = "unknown field 'new_field'";
+    ignoreException(Pattern.quote(errString));
+    try {
+      assertU(adoc("new_field", "thing1 thing2", "str", "X"));
+      fail();
+    } catch (Exception e) {
+      for (Throwable t = e; t != null; t = t.getCause()) {
+        // short circuit out if we found what we expected
+        if (t.getMessage() != null && -1 != t.getMessage().indexOf(errString)) return;
+      }
+      // otherwise, rethrow it, possibly completely unrelated
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "Unexpected error, expected error matching: " + errString, e);
+    } finally {
+      resetExceptionIgnores();
+    }
+    assertU(commit());
+    assertQ(req("new_field:thing1"), "//*[@numFound='0']");
+
+    Map<String,Object> options = new HashMap<String,Object>();
+    options.put("stored", "false");
+    IndexSchema oldSchema = h.getCore().getLatestSchema();
+    String fieldName = "new_field";
+    String fieldType = "text";
+    SchemaField newField = oldSchema.newField(fieldName, fieldType, options);
+    IndexSchema newSchema = oldSchema.addField(newField);
+    h.getCore().setLatestSchema(newSchema);
+
+    assertU(adoc("new_field", "thing1 thing2", "str", "X"));
+    assertU(commit());
+
+    assertQ(req("new_field:thing1"), "//*[@numFound='1']");
+  }
+  
+  public void testAddFieldWhenItAlreadyExists() throws Exception{
+    deleteCore();
+    File managedSchemaFile = new File(tmpConfDir, "managed-schema");
+    assertTrue(managedSchemaFile.delete()); // Delete managed-schema so it won't block parsing a new schema
+    initCore("solrconfig-mutable-managed-schema.xml", "schema-one-field-no-dynamic-field.xml", tmpSolrHome.getPath());
+
+    assertNotNull("Field 'str' is not present in the schema", h.getCore().getLatestSchema().getFieldOrNull("str"));
+    
+    String errString = "Field 'str' already exists.";
+    ignoreException(Pattern.quote(errString));
+    try {
+      Map<String,Object> options = new HashMap<String,Object>();
+      IndexSchema oldSchema = h.getCore().getLatestSchema();
+      String fieldName = "str";
+      String fieldType = "string";
+      SchemaField newField = oldSchema.newField(fieldName, fieldType, options);
+      IndexSchema newSchema = oldSchema.addField(newField);
+      h.getCore().setLatestSchema(newSchema);
+      fail("Should fail when adding a field that already exists");
+    } catch (Exception e) {
+      for (Throwable t = e; t != null; t = t.getCause()) {
+        // short circuit out if we found what we expected
+        if (t.getMessage() != null && -1 != t.getMessage().indexOf(errString)) return;
+      }
+      // otherwise, rethrow it, possibly completely unrelated
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "Unexpected error, expected error matching: " + errString, e);
+    } finally {
+      resetExceptionIgnores();
+    }
+  }
+
+  public void testAddSameFieldTwice() throws Exception{
+    deleteCore();
+    File managedSchemaFile = new File(tmpConfDir, "managed-schema");
+    assertTrue(managedSchemaFile.delete()); // Delete managed-schema so it won't block parsing a new schema
+    initCore("solrconfig-mutable-managed-schema.xml", "schema-one-field-no-dynamic-field.xml", tmpSolrHome.getPath());
+
+    Map<String,Object> options = new HashMap<String,Object>();
+    options.put("stored", "false");
+    IndexSchema oldSchema = h.getCore().getLatestSchema();
+    String fieldName = "new_field";
+    String fieldType = "text";
+    SchemaField newField = oldSchema.newField(fieldName, fieldType, options);
+    IndexSchema newSchema = oldSchema.addField(newField);
+    h.getCore().setLatestSchema(newSchema);
+
+    String errString = "Field 'new_field' already exists.";
+    ignoreException(Pattern.quote(errString));
+    try {
+      newSchema = newSchema.addField(newField);
+      h.getCore().setLatestSchema(newSchema);
+      fail("Should fail when adding the same field twice");
+    } catch (Exception e) {
+      for (Throwable t = e; t != null; t = t.getCause()) {
+        // short circuit out if we found what we expected
+        if (t.getMessage() != null && -1 != t.getMessage().indexOf(errString)) return;
+      }
+      // otherwise, rethrow it, possibly completely unrelated
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "Unexpected error, expected error matching: " + errString, e);
+    } finally {
+      resetExceptionIgnores();
+    }
+  }
+
+  public void testAddDynamicField() throws Exception{
+    deleteCore();
+    File managedSchemaFile = new File(tmpConfDir, "managed-schema");
+    assertTrue(managedSchemaFile.delete()); // Delete managed-schema so it won't block parsing a new schema
+    initCore("solrconfig-mutable-managed-schema.xml", "schema-one-field-no-dynamic-field.xml", tmpSolrHome.getPath());
+
+    assertNull("Field '*_s' is present in the schema", h.getCore().getLatestSchema().getFieldOrNull("*_s"));
+
+    String errString = "Can't add dynamic field '*_s'.";
+    ignoreException(Pattern.quote(errString));
+    try {
+      Map<String,Object> options = new HashMap<String,Object>();
+      IndexSchema oldSchema = h.getCore().getLatestSchema();
+      String fieldName = "*_s";
+      String fieldType = "string";
+      SchemaField newField = oldSchema.newField(fieldName, fieldType, options);
+      IndexSchema newSchema = oldSchema.addField(newField);
+      h.getCore().setLatestSchema(newSchema);
+      fail("Should fail when adding a dynamic field");
+    } catch (Exception e) {
+      for (Throwable t = e; t != null; t = t.getCause()) {
+        // short circuit out if we found what we expected
+        if (t.getMessage() != null && -1 != t.getMessage().indexOf(errString)) return;
+      }
+      // otherwise, rethrow it, possibly completely unrelated
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR,
+          "Unexpected error, expected error matching: " + errString, e);
+    } finally {
+      resetExceptionIgnores();
+    }
+  }
+  
+  public void testAddWithSchemaCodecFactory() throws Exception {
+    deleteCore();
+    File managedSchemaFile = new File(tmpConfDir, "managed-schema");
+    assertTrue(managedSchemaFile.delete()); // Delete managed-schema so it won't block parsing a new schema
+    initCore("solrconfig-mutable-managed-schema.xml", "schema_codec.xml", tmpSolrHome.getPath());
+
+    String uniqueKey = "string_f";
+    assertNotNull("Unique key field '" + uniqueKey + "' is not present in the schema", 
+                  h.getCore().getLatestSchema().getFieldOrNull(uniqueKey));
+
+    String fieldName = "string_disk_new_field";
+    assertNull("Field '" + fieldName + "' is present in the schema", 
+               h.getCore().getLatestSchema().getFieldOrNull(fieldName));
+
+    Map<String,Object> options = new HashMap<String,Object>();
+    IndexSchema oldSchema = h.getCore().getLatestSchema();
+    String fieldType = "string_disk";
+    SchemaField newField = oldSchema.newField(fieldName, fieldType, options);
+    IndexSchema newSchema = oldSchema.addField(newField);
+    h.getCore().setLatestSchema(newSchema);
+
+    assertU(adoc(fieldName, "thing", uniqueKey, "aBc"));
+    assertU(commit());
+
+    assertQ(req(fieldName + ":thing"), "//*[@numFound='1']");
+  }
+
+  public void testAddWithSchemaSimilarityFactory() throws Exception {
+    deleteCore();
+    File managedSchemaFile = new File(tmpConfDir, "managed-schema");
+    assertTrue(managedSchemaFile.delete()); // Delete managed-schema so it won't block parsing a new schema
+    initCore("solrconfig-mutable-managed-schema.xml", "schema-bm25.xml", tmpSolrHome.getPath());
+
+    String uniqueKey = "id";
+    assertNotNull("Unique key field '" + uniqueKey + "' is not present in the schema",
+        h.getCore().getLatestSchema().getFieldOrNull(uniqueKey));
+
+    String fieldName = "new_text_field";
+    assertNull("Field '" + fieldName + "' is present in the schema",
+        h.getCore().getLatestSchema().getFieldOrNull(fieldName));
+
+    Map<String,Object> options = new HashMap<String,Object>();
+    IndexSchema oldSchema = h.getCore().getLatestSchema();
+    String fieldType = "text";
+    SchemaField newField = oldSchema.newField(fieldName, fieldType, options);
+    IndexSchema newSchema = oldSchema.addField(newField);
+    h.getCore().setLatestSchema(newSchema);
+
+    assertU(adoc(fieldName, "thing", uniqueKey, "123"));
+    assertU(commit());
+
+    assertQ(req(fieldName + ":thing"), "//*[@numFound='1']");
+  }
+
 }

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/QueryParsingTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/QueryParsingTest.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/QueryParsingTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/QueryParsingTest.java Mon Apr 22 14:26:55 2013
@@ -22,7 +22,6 @@ import org.apache.lucene.search.SortFiel
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.schema.IndexSchema;
 import org.junit.BeforeClass;
 import org.junit.Test;
 
@@ -75,7 +74,6 @@ public class QueryParsingTest extends So
     Sort sort;
     SolrQueryRequest req = req();
 
-    IndexSchema schema = h.getCore().getSchema();
     sort = QueryParsing.parseSort("score desc", req);
     assertNull("sort", sort);//only 1 thing in the list, no Sort specified
 
@@ -188,7 +186,6 @@ public class QueryParsingTest extends So
     Sort sort;
     SolrQueryRequest req = req();
 
-    IndexSchema schema = h.getCore().getSchema();
     //test some bad vals
     try {
       sort = QueryParsing.parseSort("weight, desc", req);

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestAddFieldRealTimeGet.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestAddFieldRealTimeGet.java?rev=1470539&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestAddFieldRealTimeGet.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestAddFieldRealTimeGet.java Mon Apr 22 14:26:55 2013
@@ -0,0 +1,89 @@
+package org.apache.solr.search;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+import org.apache.commons.io.FileUtils;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.schema.TestManagedSchema;
+import org.junit.Before;
+
+import java.io.File;
+import java.util.Collections;
+
+public class TestAddFieldRealTimeGet extends TestRTGBase {
+
+  private static File tmpSolrHome;
+  private static File tmpConfDir;
+
+  private static final String collection = "collection1";
+  private static final String confDir = collection + "/conf";
+
+  @Before
+  private void initManagedSchemaCore() throws Exception {
+    createTempDir();
+    final String tmpSolrHomePath
+        = TEMP_DIR + File.separator + TestManagedSchema.class.getSimpleName() + System.currentTimeMillis();
+    tmpSolrHome = new File(tmpSolrHomePath).getAbsoluteFile();
+    tmpConfDir = new File(tmpSolrHome, confDir);
+    File testHomeConfDir = new File(TEST_HOME(), confDir);
+    final String configFileName = "solrconfig-tlog-mutable-managed-schema.xml";
+    final String schemaFileName = "schema-id-and-version-fields-only.xml";
+    FileUtils.copyFileToDirectory(new File(testHomeConfDir, configFileName), tmpConfDir);
+    FileUtils.copyFileToDirectory(new File(testHomeConfDir, schemaFileName), tmpConfDir);
+
+    // initCore will trigger an upgrade to managed schema, since the solrconfig has
+    // <schemaFactory class="ManagedIndexSchemaFactory" ... />
+    initCore(configFileName, schemaFileName, tmpSolrHome.getPath());
+  }
+
+  public void test() throws Exception {
+    clearIndex();
+    assertU(commit());
+
+    String newFieldName = "newfield";
+    String newFieldType = "string";
+    String newFieldValue = "xyz";
+
+    assertFailedU("Should fail due to unknown field '" + newFieldName + "'", 
+                  adoc("id", "1", newFieldName, newFieldValue));
+
+    IndexSchema schema = h.getCore().getLatestSchema();
+    SchemaField newField = schema.newField(newFieldName, newFieldType, Collections.<String,Object>emptyMap());
+    IndexSchema newSchema = schema.addField(newField);
+    h.getCore().setLatestSchema(newSchema);
+    
+    String newFieldKeyValue = "'" + newFieldName + "':'" + newFieldValue + "'"; 
+    assertU(adoc("id", "1", newFieldName, newFieldValue));
+    assertJQ(req("q","id:1"), 
+             "/response/numFound==0");
+    assertJQ(req("qt","/get", "id","1", "fl","id,"+newFieldName),
+             "=={'doc':{'id':'1'," + newFieldKeyValue + "}}");
+    assertJQ(req("qt","/get","ids","1", "fl","id,"+newFieldName),
+             "=={'response':{'numFound':1,'start':0,'docs':[{'id':'1'," + newFieldKeyValue + "}]}}");
+
+    assertU(commit());
+
+    assertJQ(req("q","id:1"), 
+             "/response/numFound==1");
+    assertJQ(req("qt","/get", "id","1", "fl","id,"+newFieldName),
+        "=={'doc':{'id':'1'," + newFieldKeyValue + "}}");
+    assertJQ(req("qt","/get","ids","1", "fl","id,"+newFieldName),
+        "=={'response':{'numFound':1,'start':0,'docs':[{'id':'1'," + newFieldKeyValue + "}]}}");
+  }
+}

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestPseudoReturnFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestPseudoReturnFields.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestPseudoReturnFields.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/search/TestPseudoReturnFields.java Mon Apr 22 14:26:55 2013
@@ -69,7 +69,7 @@ public class TestPseudoReturnFields exte
 
     // score as psuedo field - precondition checks
     for (String name : new String[] {"score", "val_ss"}) {
-      SchemaField sf = h.getCore().getSchema().getFieldOrNull(name);
+      SchemaField sf = h.getCore().getLatestSchema().getFieldOrNull(name);
       assertNotNull("Test depends on a (dynamic) field mtching '"+name+
                     "', schema was changed out from under us!",sf);
       assertTrue("Test depends on a multivalued dynamic field matching '"+name+

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/DirectUpdateHandlerTest.java Mon Apr 22 14:26:55 2013
@@ -87,7 +87,7 @@ public class DirectUpdateHandlerTest ext
     assertNull("This test requires a schema that has no version field, " +
                "it appears the schema file in use has been edited to violate " +
                "this requirement",
-               h.getCore().getSchema().getFieldOrNull(VersionInfo.VERSION_FIELD));
+               h.getCore().getLatestSchema().getFieldOrNull(VersionInfo.VERSION_FIELD));
 
     assertU(adoc("id","5"));
     assertU(adoc("id","6"));

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/DocumentBuilderTest.java Mon Apr 22 14:26:55 2013
@@ -60,7 +60,7 @@ public class DocumentBuilderTest extends
     try {
       SolrInputDocument doc = new SolrInputDocument();
       doc.setField( "unknown field", 12345, 1.0f );
-      DocumentBuilder.toDocument( doc, core.getSchema() );
+      DocumentBuilder.toDocument( doc, core.getLatestSchema() );
       fail( "should throw an error" );
     }
     catch( SolrException ex ) {
@@ -76,7 +76,7 @@ public class DocumentBuilderTest extends
     // make sure a null value is not indexed
     SolrInputDocument doc = new SolrInputDocument();
     doc.addField( "name", null, 1.0f );
-    Document out = DocumentBuilder.toDocument( doc, core.getSchema() );
+    Document out = DocumentBuilder.toDocument( doc, core.getLatestSchema() );
     assertNull( out.get( "name" ) );
   }
 
@@ -90,7 +90,7 @@ public class DocumentBuilderTest extends
     doc.addField( "id", "123", 1.0f );
     doc.addField( "unknown", "something", 1.0f );
     try {
-      DocumentBuilder.toDocument( doc, core.getSchema() );
+      DocumentBuilder.toDocument( doc, core.getLatestSchema() );
       fail( "added an unknown field" );
     }
     catch( Exception ex ) {
@@ -101,7 +101,7 @@ public class DocumentBuilderTest extends
 
     doc.addField( "weight", "not a number", 1.0f );
     try {
-      DocumentBuilder.toDocument( doc, core.getSchema() );
+      DocumentBuilder.toDocument( doc, core.getLatestSchema() );
       fail( "invalid 'float' field value" );
     }
     catch( Exception ex ) {
@@ -111,7 +111,7 @@ public class DocumentBuilderTest extends
     
     // now make sure it is OK
     doc.setField( "weight", "1.34", 1.0f );
-    DocumentBuilder.toDocument( doc, core.getSchema() );
+    DocumentBuilder.toDocument( doc, core.getLatestSchema() );
   }
 
   @Test
@@ -121,7 +121,7 @@ public class DocumentBuilderTest extends
     // make sure a null value is not indexed
     SolrInputDocument doc = new SolrInputDocument();
     doc.addField( "home", "2.2,3.3", 1.0f );
-    Document out = DocumentBuilder.toDocument( doc, core.getSchema() );
+    Document out = DocumentBuilder.toDocument( doc, core.getLatestSchema() );
     assertNotNull( out.get( "home" ) );//contains the stored value and term vector, if there is one
     assertNotNull( out.getField( "home_0" + FieldType.POLY_FIELD_SEPARATOR + "double" ) );
     assertNotNull( out.getField( "home_1" + FieldType.POLY_FIELD_SEPARATOR + "double" ) );
@@ -130,13 +130,13 @@ public class DocumentBuilderTest extends
   @Test
   public void testCopyFieldWithDocumentBoost() {
     SolrCore core = h.getCore();
-    IndexSchema schema = core.getSchema();
+    IndexSchema schema = core.getLatestSchema();
     assertFalse(schema.getField("title").omitNorms());
     assertTrue(schema.getField("title_stringNoNorms").omitNorms());
     SolrInputDocument doc = new SolrInputDocument();
     doc.setDocumentBoost(3f);
     doc.addField( "title", "mytitle");
-    Document out = DocumentBuilder.toDocument( doc, core.getSchema() );
+    Document out = DocumentBuilder.toDocument( doc, schema );
     assertNotNull( out.get( "title_stringNoNorms" ) );
     assertTrue("title_stringNoNorms has the omitNorms attribute set to true, if the boost is different than 1.0, it will fail",1.0f == out.getField( "title_stringNoNorms" ).boost() );
     assertTrue("It is OK that title has a boost of 3",3.0f == out.getField( "title" ).boost() );
@@ -146,12 +146,12 @@ public class DocumentBuilderTest extends
   @Test
   public void testCopyFieldWithFieldBoost() {
     SolrCore core = h.getCore();
-    IndexSchema schema = core.getSchema();
+    IndexSchema schema = core.getLatestSchema();
     assertFalse(schema.getField("title").omitNorms());
     assertTrue(schema.getField("title_stringNoNorms").omitNorms());
     SolrInputDocument doc = new SolrInputDocument();
     doc.addField( "title", "mytitle", 3.0f );
-    Document out = DocumentBuilder.toDocument( doc, core.getSchema() );
+    Document out = DocumentBuilder.toDocument( doc, schema );
     assertNotNull( out.get( "title_stringNoNorms" ) );
     assertTrue("title_stringNoNorms has the omitNorms attribute set to true, if the boost is different than 1.0, it will fail",1.0f == out.getField( "title_stringNoNorms" ).boost() );
     assertTrue("It is OK that title has a boost of 3",3.0f == out.getField( "title" ).boost() );
@@ -160,7 +160,7 @@ public class DocumentBuilderTest extends
   @Test
   public void testWithPolyFieldsAndFieldBoost() {
     SolrCore core = h.getCore();
-    IndexSchema schema = core.getSchema();
+    IndexSchema schema = core.getLatestSchema();
     assertFalse(schema.getField("store").omitNorms());
     assertTrue(schema.getField("store_0_coordinate").omitNorms());
     assertTrue(schema.getField("store_1_coordinate").omitNorms());
@@ -171,7 +171,7 @@ public class DocumentBuilderTest extends
     SolrInputDocument doc = new SolrInputDocument();
     doc.addField( "store", "40.7143,-74.006", 3.0f );
     doc.addField( "amount", "10.5", 3.0f );
-    Document out = DocumentBuilder.toDocument( doc, core.getSchema() );
+    Document out = DocumentBuilder.toDocument( doc, schema );
     assertNotNull( out.get( "store" ) );
     assertNotNull( out.get( "amount" ) );
     assertNotNull(out.getField("store_0_coordinate"));
@@ -185,7 +185,7 @@ public class DocumentBuilderTest extends
   @Test
   public void testWithPolyFieldsAndDocumentBoost() {
     SolrCore core = h.getCore();
-    IndexSchema schema = core.getSchema();
+    IndexSchema schema = core.getLatestSchema();
     assertFalse(schema.getField("store").omitNorms());
     assertTrue(schema.getField("store_0_coordinate").omitNorms());
     assertTrue(schema.getField("store_1_coordinate").omitNorms());
@@ -197,7 +197,7 @@ public class DocumentBuilderTest extends
     doc.setDocumentBoost(3.0f);
     doc.addField( "store", "40.7143,-74.006");
     doc.addField( "amount", "10.5");
-    Document out = DocumentBuilder.toDocument( doc, core.getSchema() );
+    Document out = DocumentBuilder.toDocument( doc, schema );
     assertNotNull( out.get( "store" ) );
     assertNotNull(out.getField("store_0_coordinate"));
     //NOTE: As the subtypes have omitNorm=true, they must have boost=1F, otherwise this is going to fail when adding the doc to Lucene.
@@ -221,7 +221,7 @@ public class DocumentBuilderTest extends
   
   public void testMultiValuedFieldAndDocBoosts() throws Exception {
     SolrCore core = h.getCore();
-    IndexSchema schema = core.getSchema();
+    IndexSchema schema = core.getLatestSchema();
     SolrInputDocument doc = new SolrInputDocument();
     doc.setDocumentBoost(3.0f);
     SolrInputField field = new SolrInputField( "foo_t" );
@@ -230,7 +230,7 @@ public class DocumentBuilderTest extends
     field.addValue( "living is easy" , 1.0f );
     doc.put( field.getName(), field );
 
-    Document out = DocumentBuilder.toDocument( doc, core.getSchema() );
+    Document out = DocumentBuilder.toDocument( doc, schema );
     IndexableField[] outF = out.getFields( field.getName() );
     assertEquals("wrong number of field values",
                  3, outF.length);
@@ -247,7 +247,7 @@ public class DocumentBuilderTest extends
 
   public void testCopyFieldsAndFieldBoostsAndDocBoosts() throws Exception {
     SolrCore core = h.getCore();
-    IndexSchema schema = core.getSchema();
+    IndexSchema schema = core.getLatestSchema();
     SolrInputDocument doc = new SolrInputDocument();
 
     final float DOC_BOOST = 3.0F;
@@ -269,7 +269,7 @@ public class DocumentBuilderTest extends
     assertEquals(FOO_BOOST, inFoo.getBoost(), 0.0F);
     doc.put( inFoo.getName(), inFoo );
 
-    Document out = DocumentBuilder.toDocument( doc, core.getSchema() );
+    Document out = DocumentBuilder.toDocument( doc, schema );
 
     IndexableField[] outTitle = out.getFields( inTitle.getName() );
     assertEquals("wrong number of title values",

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/TestIndexingPerformance.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/TestIndexingPerformance.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/TestIndexingPerformance.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/TestIndexingPerformance.java Mon Apr 22 14:26:55 2013
@@ -17,10 +17,8 @@
 
 package org.apache.solr.update;
 
-import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.schema.IndexSchema;
 import org.apache.solr.util.AbstractSolrTestCase;
 import org.apache.solr.common.util.StrUtils;
 import org.junit.AfterClass;
@@ -73,7 +71,6 @@ public class TestIndexingPerformance ext
 
 
     SolrQueryRequest req = lrf.makeRequest();
-    IndexSchema schema = req.getSchema();
     UpdateHandler updateHandler = req.getCore().getUpdateHandler();
     String field = "textgap";
 

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/FieldMutatingUpdateProcessorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/FieldMutatingUpdateProcessorTest.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/FieldMutatingUpdateProcessorTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/FieldMutatingUpdateProcessorTest.java Mon Apr 22 14:26:55 2013
@@ -17,36 +17,15 @@
 
 package org.apache.solr.update.processor;
 
-import java.util.ArrayList;
 import java.util.LinkedHashSet;
 import java.util.TreeSet;
-import java.util.HashMap;
-import java.util.Map;
 import java.util.Arrays;
-import java.io.IOException;
-
-import org.apache.solr.SolrTestCaseJ4;
 
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
 import org.apache.solr.common.SolrInputField;
-import org.apache.solr.common.params.ModifiableSolrParams;
-import org.apache.solr.common.params.SolrParams;
-
-import org.apache.solr.core.SolrCore;
 import org.apache.solr.schema.IndexSchema;
-
-import org.apache.solr.request.SolrQueryRequest;
-import org.apache.solr.request.LocalSolrQueryRequest;
-import org.apache.solr.response.SolrQueryResponse;
-
-import org.apache.solr.update.AddUpdateCommand;
-import org.apache.solr.update.processor.UpdateRequestProcessor;
-import org.apache.solr.update.processor.UpdateRequestProcessorChain;
-
-import org.junit.Before;
 import org.junit.BeforeClass;
-import org.junit.Test;
 
 /**
  * Tests the basics of configuring FieldMutatingUpdateProcessors  
@@ -570,7 +549,7 @@ public class FieldMutatingUpdateProcesso
 
   public void testIgnore() throws Exception {
 
-    IndexSchema schema = h.getCore().getSchema();
+    IndexSchema schema = h.getCore().getLatestSchema();
     assertNull("test expects 'foo_giberish' to not be a valid field, looks like schema was changed out from under us",
                schema.getFieldTypeNoEx("foo_giberish"));
     assertNotNull("test expects 't_raw' to be a valid field, looks like schema was changed out from under us",

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/util/SolrPluginUtilsTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/util/SolrPluginUtilsTest.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/util/SolrPluginUtilsTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/util/SolrPluginUtilsTest.java Mon Apr 22 14:26:55 2013
@@ -20,7 +20,6 @@ package org.apache.solr.util;
 import org.apache.solr.SolrTestCaseJ4;
 import org.apache.solr.request.SolrQueryRequest;
 import org.apache.solr.search.QParser;
-import org.apache.solr.util.SolrPluginUtils;
 import org.apache.solr.util.SolrPluginUtils.DisjunctionMaxQueryParser;
 import org.apache.solr.search.SolrIndexSearcher;
 import org.apache.solr.search.DocList;
@@ -200,7 +199,7 @@ public class SolrPluginUtilsTest extends
     assertTrue(t+" sanity test isn't TermQuery: " + out.getClass(),
                out instanceof TermQuery);
     assertEquals(t+" sanity test is wrong field",
-                 h.getCore().getSchema().getDefaultSearchFieldName(),
+                 h.getCore().getLatestSchema().getDefaultSearchFieldName(),
                  ((TermQuery)out).getTerm().field());
 
     t = "subject:XXXXXXXX";

Modified: lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java (original)
+++ lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/BaseDistributedSearchTestCase.java Mon Apr 22 14:26:55 2013
@@ -29,6 +29,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Random;
 import java.util.Set;
+import java.util.SortedMap;
 import java.util.concurrent.atomic.AtomicInteger;
 
 import junit.framework.Assert;
@@ -52,6 +53,7 @@ import org.apache.solr.common.params.Sol
 import org.apache.solr.common.util.NamedList;
 import org.apache.solr.schema.TrieDateField;
 import org.apache.solr.util.AbstractSolrTestCase;
+import org.eclipse.jetty.servlet.ServletHolder;
 import org.junit.BeforeClass;
 import org.junit.AfterClass;
 import org.junit.Test;
@@ -363,7 +365,9 @@ public abstract class BaseDistributedSea
   
   public JettySolrRunner createJetty(File solrHome, String dataDir, String shardList, String solrConfigOverride, String schemaOverride, boolean explicitCoreNodeName) throws Exception {
 
-    JettySolrRunner jetty = new JettySolrRunner(solrHome.getAbsolutePath(), context, 0, solrConfigOverride, schemaOverride);
+    boolean stopAtShutdown = true;
+    JettySolrRunner jetty = new JettySolrRunner
+        (solrHome.getAbsolutePath(), context, 0, solrConfigOverride, schemaOverride, stopAtShutdown, getExtraServlets());
     jetty.setShards(shardList);
     jetty.setDataDir(dataDir);
     if (explicitCoreNodeName) {
@@ -374,12 +378,17 @@ public abstract class BaseDistributedSea
     return jetty;
   }
   
+  /** Override this method to insert extra servlets into the JettySolrRunners that are created using createJetty() */
+  public SortedMap<ServletHolder,String> getExtraServlets() {
+    return null;
+  }
+  
   protected SolrServer createNewSolrServer(int port) {
     try {
       // setup the server...
       String url = "http://127.0.0.1:" + port + context;
       HttpSolrServer s = new HttpSolrServer(url);
-      s.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);;
+      s.setConnectionTimeout(DEFAULT_CONNECTION_TIMEOUT);
       s.setSoTimeout(60000);
       s.setDefaultMaxConnectionsPerHost(100);
       s.setMaxTotalConnections(100);

Modified: lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/util/RestTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/util/RestTestBase.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/util/RestTestBase.java (original)
+++ lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/util/RestTestBase.java Mon Apr 22 14:26:55 2013
@@ -229,7 +229,8 @@ abstract public class RestTestBase exten
 
     for (String test : tests) {
       if (null == test || 0 == test.length()) continue;
-      String testJSON = test.replace('\'', '"');
+      String testJSON = test.replaceAll("(?<!\\\\)\'", "\"");
+      testJSON = testJSON.replaceAll("\\\\\'", "'");
 
       try {
         failed = true;
@@ -255,6 +256,170 @@ abstract public class RestTestBase exten
     }
   }
 
+  
+  
+  /**
+   * Validates the response from a PUT request matches some JSON test expressions
+   * 
+   * @see org.apache.solr.JSONTestUtil#DEFAULT_DELTA
+   * @see #assertJQ(String,double,String...)
+   */
+  public static void assertJPut(String request, String content, String... tests) throws Exception {
+    assertJPut(request, content, JSONTestUtil.DEFAULT_DELTA, tests);
+  }
+
+
+  /**
+   * Validates the response from a PUT request matches some JSON test expressions
+   * and closes the query. The text expression is of the form path==JSON.
+   * To facilitate easy embedding in Java strings, the JSON can have double
+   * quotes replaced with single quotes.
+   * <p>
+   * Please use this with care: this makes it easy to match complete
+   * structures, but doing so can result in fragile tests if you are
+   * matching more than what you want to test.
+   * </p>
+   * @param request a URL path with optional query params, e.g. "/schema/fields?fl=id,_version_"
+   * @param content The content to include with the PUT request
+   * @param delta tolerance allowed in comparing float/double values
+   * @param tests JSON path expression + '==' + expected value
+   */
+  public static void assertJPut(String request, String content, double delta, String... tests) throws Exception {
+    int queryStartPos = request.indexOf('?');
+    String query;
+    String path;
+    if (-1 == queryStartPos) {
+      query = "";
+      path = request;
+    } else {
+      query = request.substring(queryStartPos + 1);
+      path = request.substring(0, queryStartPos);
+    }
+    query = setParam(query, "wt", "json");
+    request = path + '?' + setParam(query, "indent", "on");
+
+    String response;
+    boolean failed = true;
+    try {
+      response = restTestHarness.put(request, content);
+      failed = false;
+    } finally {
+      if (failed) {
+        log.error("REQUEST FAILED: " + request);
+      }
+    }
+
+    for (String test : tests) {
+      if (null == test || 0 == test.length()) continue;
+      String testJSON = test.replaceAll("(?<!\\\\)\'", "\"");
+      testJSON = testJSON.replaceAll("\\\\\'", "'");
+
+      try {
+        failed = true;
+        String err = JSONTestUtil.match(response, testJSON, delta);
+        failed = false;
+        if (err != null) {
+          log.error("query failed JSON validation. error=" + err +
+              "\n expected =" + testJSON +
+              "\n response = " + response +
+              "\n request = " + request + "\n"
+          );
+          throw new RuntimeException(err);
+        }
+      } finally {
+        if (failed) {
+          log.error("JSON query validation threw an exception." +
+              "\n expected =" + testJSON +
+              "\n response = " + response +
+              "\n request = " + request + "\n"
+          );
+        }
+      }
+    }
+  }
+
+  /**
+   * Validates the response from a POST request matches some JSON test expressions
+   *
+   * @see org.apache.solr.JSONTestUtil#DEFAULT_DELTA
+   * @see #assertJQ(String,double,String...)
+   */
+  public static void assertJPost(String request, String content, String... tests) throws Exception {
+    assertJPost(request, content, JSONTestUtil.DEFAULT_DELTA, tests);
+  }
+
+
+  /**
+   * Validates the response from a PUT request matches some JSON test expressions
+   * and closes the query. The text expression is of the form path==JSON.
+   * To facilitate easy embedding in Java strings, the JSON can have double
+   * quotes replaced with single quotes.
+   * <p>
+   * Please use this with care: this makes it easy to match complete
+   * structures, but doing so can result in fragile tests if you are
+   * matching more than what you want to test.
+   * </p>
+   * @param request a URL path with optional query params, e.g. "/schema/fields?fl=id,_version_"
+   * @param content The content to include with the PUT request
+   * @param delta tolerance allowed in comparing float/double values
+   * @param tests JSON path expression + '==' + expected value
+   */
+  public static void assertJPost(String request, String content, double delta, String... tests) throws Exception {
+    int queryStartPos = request.indexOf('?');
+    String query;
+    String path;
+    if (-1 == queryStartPos) {
+      query = "";
+      path = request;
+    } else {
+      query = request.substring(queryStartPos + 1);
+      path = request.substring(0, queryStartPos);
+    }
+    query = setParam(query, "wt", "json");
+    request = path + '?' + setParam(query, "indent", "on");
+
+    String response;
+    boolean failed = true;
+    try {
+      response = restTestHarness.post(request, content);
+      failed = false;
+    } finally {
+      if (failed) {
+        log.error("REQUEST FAILED: " + request);
+      }
+    }
+
+    for (String test : tests) {
+      if (null == test || 0 == test.length()) continue;
+      String testJSON = test.replaceAll("(?<!\\\\)\'", "\"");
+      testJSON = testJSON.replaceAll("\\\\\'", "'");
+
+      try {
+        failed = true;
+        String err = JSONTestUtil.match(response, testJSON, delta);
+        failed = false;
+        if (err != null) {
+          log.error("query failed JSON validation. error=" + err +
+              "\n expected =" + testJSON +
+              "\n response = " + response +
+              "\n request = " + request + "\n"
+          );
+          throw new RuntimeException(err);
+        }
+      } finally {
+        if (failed) {
+          log.error("JSON query validation threw an exception." +
+              "\n expected =" + testJSON +
+              "\n response = " + response +
+              "\n request = " + request + "\n"
+          );
+        }
+      }
+    }
+  }
+
+
+
   /**
    * Insures that the given param is included in the query with the given value.
    *

Modified: lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/util/RestTestHarness.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/util/RestTestHarness.java?rev=1470539&r1=1470538&r2=1470539&view=diff
==============================================================================
--- lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/util/RestTestHarness.java (original)
+++ lucene/dev/trunk/solr/test-framework/src/java/org/apache/solr/util/RestTestHarness.java Mon Apr 22 14:26:55 2013
@@ -17,12 +17,12 @@ package org.apache.solr.util;
  */
 
 import org.apache.commons.io.IOUtils;
-import org.eclipse.jetty.util.IO;
 
 import javax.xml.xpath.XPathExpressionException;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
 import java.io.StringWriter;
 import java.net.HttpURLConnection;
 import java.net.URL;
@@ -43,20 +43,37 @@ public class RestTestHarness extends Bas
   }
   
   /**
-   * Validates a "query" response against an array of XPath test strings
+   * Validates an XML "query" response against an array of XPath test strings
    *
    * @param request the Query to process
    * @return null if all good, otherwise the first test that fails.
    * @exception Exception any exception in the response.
    * @exception java.io.IOException if there is a problem writing the XML
    */
-  public String validateQuery(String request, String... tests)
-      throws Exception {
+  public String validateQuery(String request, String... tests) throws Exception {
 
     String res = query(request);
     return validateXPath(res, tests);
   }
 
+
+  /**
+   * Validates an XML PUT response against an array of XPath test strings
+   *
+   * @param request the PUT request to process
+   * @param content the content to send with the PUT request
+   * @param tests the validating XPath tests
+   * @return null if all good, otherwise the first test that fails.
+   * @exception Exception any exception in the response.
+   * @exception java.io.IOException if there is a problem writing the XML
+   */
+  public String validatePut(String request, String content, String... tests) throws Exception {
+
+    String res = put(request, content);
+    return validateXPath(res, tests);
+  }
+
+
   /**
    * Processes a "query" using a URL path (with no context path) + optional query params,
    * e.g. "/schema/fields?indent=on"
@@ -84,7 +101,74 @@ public class RestTestHarness extends Bas
     return strWriter.toString();
   }
 
-  public String checkQueryStatus(String xml, String code) throws Exception {
+  /**
+   * Processes a PUT request using a URL path (with no context path) + optional query params,
+   * e.g. "/schema/fields/newfield", PUTs the given content, and returns the response content.
+   * 
+   * @param request The URL path and optional query params
+   * @param content The content to include with the PUT request
+   * @return The response to the PUT request
+   */
+  public String put(String request, String content) throws IOException {
+    URL url = new URL(getBaseURL() + request);
+    HttpURLConnection connection = (HttpURLConnection)url.openConnection();
+    connection.setDoOutput(true);
+    connection.setRequestMethod("PUT");
+    OutputStreamWriter out = new OutputStreamWriter(connection.getOutputStream(), "UTF-8");
+    out.write(content);
+    out.close();
+    InputStream inputStream = null;
+    StringWriter stringWriter;
+    try {
+      try {
+        inputStream = connection.getInputStream();
+      } catch (IOException e) {
+        inputStream = connection.getErrorStream();
+      }
+      stringWriter = new StringWriter();
+      IOUtils.copy(new InputStreamReader(inputStream, "UTF-8"), stringWriter);
+    } finally {
+      IOUtils.closeQuietly(inputStream);
+    }
+    return stringWriter.toString();
+  }
+
+  /**
+   * Processes a POST request using a URL path (with no context path) + optional query params,
+   * e.g. "/schema/fields/newfield", PUTs the given content, and returns the response content.
+   *
+   * @param request The URL path and optional query params
+   * @param content The content to include with the POST request
+   * @return The response to the PUT request
+   */
+  public String post(String request, String content) throws IOException {
+    URL url = new URL(getBaseURL() + request);
+    HttpURLConnection connection = (HttpURLConnection)url.openConnection();
+    connection.setDoOutput(true);
+    connection.setRequestMethod("POST");
+    connection.setRequestProperty("Content-Type", "application/json; charset=utf-8");
+
+    OutputStreamWriter out = new OutputStreamWriter(connection.getOutputStream(), "UTF-8");
+    out.write(content);
+    out.close();
+    InputStream inputStream = null;
+    StringWriter stringWriter;
+    try {
+      try {
+        inputStream = connection.getInputStream();
+      } catch (IOException e) {
+        inputStream = connection.getErrorStream();
+      }
+      stringWriter = new StringWriter();
+      IOUtils.copy(new InputStreamReader(inputStream, "UTF-8"), stringWriter);
+    } finally {
+      IOUtils.closeQuietly(inputStream);
+    }
+    return stringWriter.toString();
+  }
+
+
+  public String checkResponseStatus(String xml, String code) throws Exception {
     try {
       String response = query(xml);
       String valid = validateXPath(response, "//int[@name='status']="+code );
@@ -94,9 +178,10 @@ public class RestTestHarness extends Bas
     }
   }
 
+  
   @Override
   public void reload() throws Exception {
-    String xml = checkQueryStatus("/admin/cores?action=RELOAD", "0");
+    String xml = checkResponseStatus("/admin/cores?action=RELOAD", "0");
     if (null != xml) {
       throw new RuntimeException("RELOAD failed:\n" + xml);
     }
@@ -112,7 +197,7 @@ public class RestTestHarness extends Bas
   @Override
   public String update(String xml) {
     try {
-      return query("/update?stream.base=" + URLEncoder.encode(xml, "UTF-8"));
+      return query("/update?stream.body=" + URLEncoder.encode(xml, "UTF-8"));
     } catch (Exception e) {
       throw new RuntimeException(e);
     }