You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ho...@apache.org on 2012/07/12 22:49:42 UTC

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

Author: hossman
Date: Thu Jul 12 20:49:41 2012
New Revision: 1360931

URL: http://svn.apache.org/viewvc?rev=1360931&view=rev
Log:
SOLR-1725: StatelessScriptUpdateProcessorFactory

Added:
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/ScriptEngineCustomizer.java   (with props)
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/addfields.updateprocessor.js   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-bogus-scriptengine-name.xml   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-invalid-scriptfile.xml   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-missing-scriptfile.xml   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/conditional.updateprocessor.js   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/missing.functions.updateprocessor.js   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/missleading.extension.updateprocessor.js.txt   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-script-updateprocessor.xml   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/throw.error.on.add.updateprocessor.js   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/trivial.updateprocessor0.js   (with props)
    lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/trivial.updateprocessor1.js   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/AbstractBadConfigTestBase.java   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/ScriptEngineTest.java   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactoryTest.java   (with props)
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/UpdateProcessorTestBase.java   (with props)
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestBadConfig.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/BadIndexSchemaTest.java
    lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/FieldMutatingUpdateProcessorTest.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1360931&r1=1360930&r2=1360931&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Thu Jul 12 20:49:41 2012
@@ -60,6 +60,10 @@ New Features
 * SOLR-3460: Add cloud-scripts directory and a zkcli.sh|bat tool for easy scripting
   and interaction with ZooKeeper. (Mark Miller)
 
+* SOLR-1725: StatelessScriptUpdateProcessorFactory allows users to implement
+  the full ScriptUpdateProcessor API using any scripting language with a 
+  javax.script.ScriptEngineFactory
+  (Uri Boness, ehatcher, Simon Rosenthal, hossman)
 
 Bug Fixes
 

Modified: lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java?rev=1360931&r1=1360930&r2=1360931&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java (original)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/core/SolrCore.java Thu Jul 12 20:49:41 2012
@@ -664,7 +664,8 @@ public final class SolrCore implements S
       latch.countDown();//release the latch, otherwise we block trying to do the close.  This should be fine, since counting down on a latch of 0 is still fine
       //close down the searcher and any other resources, if it exists, as this is not recoverable
       close();
-      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, null, e);
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, 
+                              e.getMessage(), e);
     } finally {
       // allow firstSearcher events to fire and make sure it is released
       latch.countDown();

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/ScriptEngineCustomizer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/ScriptEngineCustomizer.java?rev=1360931&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/ScriptEngineCustomizer.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/ScriptEngineCustomizer.java Thu Jul 12 20:49:41 2012
@@ -0,0 +1,29 @@
+/**
+ * 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.
+ */
+
+package org.apache.solr.update.processor;
+
+import javax.script.ScriptEngine;
+
+/**
+ * Enables customization of a script engine. Will mostly be used to register engine scoped variables.
+ */
+public interface ScriptEngineCustomizer {
+
+  void customize(ScriptEngine engine);
+
+}

Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java?rev=1360931&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java (added)
+++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactory.java Thu Jul 12 20:49:41 2012
@@ -0,0 +1,494 @@
+package org.apache.solr.update.processor;
+
+/*
+ * 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.solr.common.SolrException;
+import org.apache.solr.common.util.NamedList;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrResourceLoader;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.request.LocalSolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.update.*;
+import org.apache.solr.util.plugin.SolrCoreAware;
+
+import org.apache.commons.lang.StringUtils;
+import org.apache.commons.io.IOUtils;
+import org.apache.commons.io.FilenameUtils;
+
+import javax.script.Invocable;
+import javax.script.ScriptEngine;
+import javax.script.ScriptEngineManager;
+import javax.script.ScriptEngineFactory;
+import javax.script.ScriptException;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.Reader;
+import java.util.Set;
+import java.util.LinkedHashSet;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Collection;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * <p>
+ * An update request processor factory that enables the use of update 
+ * processors implemented as scripts which can be loaded by the 
+ * {@link SolrResourceLoader} (usually via the <code>conf</code> dir for 
+ * the SolrCore).
+ * </p>
+ * <p>
+ * This factory requires at least one configuration parameter named
+ * <code>script</code> which may be the name of a script file as a string, 
+ * or an array of multiple script files.  If multiple script files are 
+ * specified, they are executed sequentially in the order specified in the 
+ * configuration -- as if multiple factories were configured sequentially
+ * </p>
+ * <p>
+ * Each script file is expected to declare functions with the same name 
+ * as each method in {@link UpdateRequestProcessor}, using the same 
+ * arguments.  One slight deviation is in the optional return value from 
+ * these functions: If a script function has a <code>boolean</code> return 
+ * value, and that value is <code>false</code> then the processor will 
+ * cleanly terminate processing of the command and return, without forwarding 
+ * the command on to the next script or processor in the chain.
+ * Due to limitations in the {@link ScriptEngine} API used by 
+ * this factory, it can not enforce that all functions exist on initialization,
+ * so errors from missing functions will only be generated at runtime when
+ * the chain attempts to use them.
+ * </p>
+ * <p>
+ * The factory may also be configured with an optional "params" argument, 
+ * which can be an {@link NamedList} (or array, or any other simple Java 
+ * object) which will be put into the global scope for each script.
+ * </p>
+ * <p>
+ * The following variables are define as global variables for each script:
+ * <ul>
+ *  <li>req - The {@link SolrQueryRequest}</li>
+ *  <li>rsp - The {@link SolrQueryResponse}</li>
+ *  <li>logger - A {@link Logger} that can be used for logging purposes in the script</li>
+ *  <li>params - The "params" init argument in the factory configuration (if any)</li>
+ * </ul>
+ * </p>
+ * <p>
+ * Internally this update processor uses JDK 6 scripting engine support, 
+ * and any {@link Invocable} implementations of <code>ScriptEngine</code> 
+ * that can be loaded using the Solr Plugin ClassLoader may be used.  
+ * By default, the engine used for each script is determined by the filed 
+ * extension (ie: a *.js file will be treated as a JavaScript script) but 
+ * this can be overridden by specifying an explicit "engine" name init 
+ * param for the factory, which identifies a registered name of a 
+ * {@link ScriptEngineFactory}. 
+ * (This may be particularly useful if multiple engines are available for 
+ * the same scripting language, and you wish to force the usage of a 
+ * particular engine because of known quirks)
+ * </p>
+ * <p>
+ * A new {@link ScriptEngineManager} is created for each 
+ * <code>SolrQueryRequest</code> defining a "global" scope for the script(s) 
+ * which is request specific.  Separate <code>ScriptEngine</code> instances 
+ * are then used to evaluate the script files, resulting in an "engine" scope 
+ * that is specific to each script.
+ * </p>
+ * <p>
+ * A simple example...
+ * </p>
+ * <pre class="prettyprint">
+ * &lt;processor class="solr.StatelessScriptUpdateProcessorFactory"&gt;
+ *   &lt;str name="script"&gt;updateProcessor.js&lt;/str&gt;
+ * &lt;/processor&gt;
+ * </pre>
+ * <p>
+ * A more complex example involving multiple scripts in different languages, 
+ * and a "params" <code>NamedList</code> that will be put into the global 
+ * scope of each script...
+ * </p>
+ * <pre class="prettyprint">
+ * &lt;processor class="solr.StatelessScriptUpdateProcessorFactory"&gt;
+ *   &lt;arr name="script"&gt;
+ *     &lt;str name="script"&gt;first-processor.js&lt;/str&gt;
+ *     &lt;str name="script"&gt;second-processor.py&lt;/str&gt;
+ *   &lt;/arr&gt;
+ *   &lt;lst name="params"&gt;
+ *     &lt;bool name="a_bool_value"&gt;true&lt;/bool&gt;
+ *     &lt;int name="and_int_value"&gt;3&lt;/int&gt;
+ *   &lt;/lst&gt;
+ * &lt;/processor&gt;
+ * </pre>
+ * <p>
+ * An example where the script file extensions are ignored, and an 
+ * explicit script engine is used....
+ * </p>
+ * <pre class="prettyprint">
+ * &lt;processor class="solr.StatelessScriptUpdateProcessorFactory"&gt;
+ *   &lt;arr name="script"&gt;
+ *     &lt;str name="script"&gt;first-processor.txt&lt;/str&gt;
+ *     &lt;str name="script"&gt;second-processor.txt&lt;/str&gt;
+ *   &lt;/arr&gt;
+ *   &lt;str name="engine"&gt;rhino&lt;/str&gt;
+ * &lt;/processor&gt;
+ * </pre>
+ * 
+ */
+public class StatelessScriptUpdateProcessorFactory extends UpdateRequestProcessorFactory implements SolrCoreAware {
+
+  public static Logger log = LoggerFactory.getLogger(StatelessScriptUpdateProcessorFactory.class);
+
+  private final static String SCRIPT_ARG = "script";
+  private final static String PARAMS_ARG = "params";
+  private final static String ENGINE_NAME_ARG = "engine";
+
+  private List<ScriptFile> scriptFiles;
+
+  /** if non null, this is an override for the engine for all scripts */
+  private String engineName = null;
+
+  private Object params = null;
+
+  private SolrResourceLoader resourceLoader;
+
+  private ScriptEngineCustomizer scriptEngineCustomizer;
+
+  @Override
+  public void init(NamedList args) {
+    Collection<String> scripts = 
+      FieldMutatingUpdateProcessorFactory.oneOrMany(args, SCRIPT_ARG);
+    if (scripts.isEmpty()) {
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, 
+                              "StatelessScriptUpdateProcessorFactory must be " +
+                              "initialized with at least one " + SCRIPT_ARG);
+    }
+    scriptFiles = new ArrayList<ScriptFile>();
+    for (String script : scripts) {
+      scriptFiles.add(new ScriptFile(script));
+    }
+
+    params = args.remove(PARAMS_ARG);
+
+    Object engine = args.remove(ENGINE_NAME_ARG);
+    if (engine != null) {
+      if (engine instanceof String) {
+        engineName = (String)engine;
+      } else {
+        throw new SolrException
+          (SolrException.ErrorCode.SERVER_ERROR, 
+           "'" + ENGINE_NAME_ARG + "' init param must be a String (found: " + 
+           engine.getClass() + ")");
+      }
+    }
+
+    super.init(args);
+
+  }
+
+  @Override
+  public UpdateRequestProcessor getInstance(SolrQueryRequest req, SolrQueryResponse rsp, UpdateRequestProcessor next) {
+    List<EngineInfo> scriptEngines = null;
+
+    scriptEngines = initEngines(req, rsp);
+
+    return new ScriptUpdateProcessor(req, rsp, scriptEngines, next);
+  }
+
+  // TODO: Make this useful outside of tests, such that a ScriptEngineCustomizer could be looked up through the resource loader
+  void setScriptEngineCustomizer(ScriptEngineCustomizer scriptEngineCustomizer) {
+    this.scriptEngineCustomizer = scriptEngineCustomizer;
+  }
+
+  @Override
+  public void inform(SolrCore core) {
+    resourceLoader = core.getResourceLoader();
+
+    // test that our engines & scripts are valid
+
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, new ModifiableSolrParams());
+    try {
+      initEngines(req, rsp);
+    } catch (Exception e) {
+      String msg = "Unable to initialize scripts: " + e.getMessage();
+      log.error(msg, e);
+      throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg, e);
+    } finally {
+      req.close();
+    }
+
+    
+  }
+
+
+  //================================================ Helper Methods ==================================================
+
+  /**
+   * Initializes a list of script engines - an engine per script file.
+   *
+   * @param req The solr request.
+   * @param rsp The solr response
+   * @return The list of initialized script engines.
+   */
+  private List<EngineInfo> initEngines(SolrQueryRequest req, 
+                                       SolrQueryResponse rsp) 
+    throws SolrException {
+    
+    List<EngineInfo> scriptEngines = new ArrayList<EngineInfo>();
+
+    ScriptEngineManager scriptEngineManager 
+      = new ScriptEngineManager(resourceLoader.getClassLoader());
+
+    scriptEngineManager.put("logger", log);
+    scriptEngineManager.put("req", req);
+    scriptEngineManager.put("rsp", rsp);
+    if (params != null) {
+      scriptEngineManager.put("params", params);
+    }
+
+    for (ScriptFile scriptFile : scriptFiles) {
+      ScriptEngine engine = null;
+      if (null != engineName) {
+        engine = scriptEngineManager.getEngineByName(engineName);
+        if (engine == null) {
+          String details = getSupportedEngines(scriptEngineManager, false);
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, 
+                                  "No ScriptEngine found by name: "
+                                  + engineName + 
+                                  (null != details ? 
+                                   " -- supported names: " + details : ""));
+        }
+      } else {
+        engine = scriptEngineManager.getEngineByExtension
+          (scriptFile.getExtension());
+        if (engine == null) {
+          String details = getSupportedEngines(scriptEngineManager, true);
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, 
+                                  "No ScriptEngine found by file extension: "
+                                  + scriptFile.getFileName() + 
+                                  (null != details ? 
+                                   " -- supported extensions: " + details : ""));
+                                  
+        }
+      }
+
+      if (! (engine instanceof Invocable)) {
+        String msg = 
+          "Engine " + ((null != engineName) ? engineName : 
+                       ("for script " + scriptFile.getFileName())) +
+          " does not support function invocation (via Invocable): " +
+          engine.getClass().toString() + " (" +
+          engine.getFactory().getEngineName() + ")";
+        log.error(msg);
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, msg);
+      }
+
+      if (scriptEngineCustomizer != null) {
+        scriptEngineCustomizer.customize(engine);
+      }
+
+      scriptEngines.add(new EngineInfo((Invocable)engine, scriptFile));
+      Reader scriptSrc = scriptFile.openReader(resourceLoader);
+
+      try {
+        engine.eval(scriptSrc);
+      } catch (ScriptException e) {
+        throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, 
+                                "Unable to evaluate script: " + 
+                                scriptFile.getFileName(), e);
+      } finally {
+        IOUtils.closeQuietly(scriptSrc);
+      }
+    }
+    return scriptEngines;
+  }
+
+  /**
+   * For error messages - returns null if there are any exceptions of any 
+   * kind building the string (or of the list is empty for some unknown reason).
+   * @param ext - if true, list of extensions, otherwise a list of engine names
+   */
+  private static String getSupportedEngines(ScriptEngineManager mgr,
+                                            boolean ext) {
+    String result = null;
+    try {
+      List<ScriptEngineFactory> factories = mgr.getEngineFactories();
+      if (null == factories) return result;
+
+      Set<String> engines = new LinkedHashSet<String>(factories.size());
+      for (ScriptEngineFactory f : factories) {
+        if (ext) {
+          engines.addAll(f.getExtensions());
+        } else {
+          engines.addAll(f.getNames());
+        }
+      }
+      result = StringUtils.join(engines, ", ");
+    } catch (RuntimeException e) {
+      /* :NOOP: */
+    }
+    return result;
+  }
+
+
+
+  //================================================= Inner Classes ==================================================
+
+  /**
+   * The actual update processor. All methods delegate to scripts.
+   */
+  private static class ScriptUpdateProcessor extends UpdateRequestProcessor {
+
+    private List<EngineInfo> engines;
+
+    private ScriptUpdateProcessor(SolrQueryRequest req, SolrQueryResponse res, List<EngineInfo> engines, UpdateRequestProcessor next) {
+      super(next);
+      this.engines = engines;
+    }
+
+    @Override
+    public void processAdd(AddUpdateCommand cmd) throws IOException {
+      if (invokeFunction("processAdd", cmd)) {
+        super.processAdd(cmd);
+      }
+    }
+
+    @Override
+    public void processDelete(DeleteUpdateCommand cmd) throws IOException {
+      if (invokeFunction("processDelete", cmd)) {
+        super.processDelete(cmd);
+      }
+        
+    }
+
+    @Override
+    public void processMergeIndexes(MergeIndexesCommand cmd) throws IOException {
+      if (invokeFunction("processMergeIndexes", cmd)) {
+        super.processMergeIndexes(cmd);
+      }
+    }
+
+    @Override
+    public void processCommit(CommitUpdateCommand cmd) throws IOException {
+      if (invokeFunction("processCommit", cmd)) {
+        super.processCommit(cmd);
+      }
+    }
+
+    @Override
+    public void processRollback(RollbackUpdateCommand cmd) throws IOException {
+      if (invokeFunction("processRollback", cmd)) {
+        super.processRollback(cmd);
+      }
+    }
+
+    @Override
+    public void finish() throws IOException {
+      if (invokeFunction("finish")) {
+        super.finish();
+      }
+    }
+
+    /**
+     * returns true if processing should continue, or false if the 
+     * request should be ended now.  Result value is computed from the return 
+     * value of the script function if: it exists, is non-null, and can be 
+     * cast to a java Boolean.
+     */
+    private boolean invokeFunction(String name, Object... cmd) {
+
+      for (EngineInfo engine : engines) {
+        try {
+          Object result = engine.getEngine().invokeFunction(name, cmd);
+          if (null != result && result instanceof Boolean) {
+            if (! ((Boolean)result).booleanValue() ) {
+              return false;
+            }
+          }
+
+        } catch (ScriptException e) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, 
+                                  "Unable to invoke function " + name + 
+                                  " in script: " + 
+                                  engine.getScriptFile().getFileName() + 
+                                  ": " + e.getMessage(), e);
+        } catch (NoSuchMethodException e) {
+          throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, 
+                                  "Unable to invoke function " + name + 
+                                  " in script: " + 
+                                  engine.getScriptFile().getFileName() + 
+                                  ": " + e.getMessage(), e);
+        }
+      }
+
+      return true;
+    }
+  }
+
+  /**
+   * Holds the script engine and its associated script file.
+   */
+  private static class EngineInfo {
+
+    private final Invocable engine;
+    private final ScriptFile scriptFile;
+
+    private EngineInfo(Invocable engine, ScriptFile scriptFile) {
+      this.engine = engine;
+      this.scriptFile = scriptFile;
+    }
+
+    public Invocable getEngine() {
+      return engine;
+    }
+
+    public ScriptFile getScriptFile() {
+      return scriptFile;
+    }
+  }
+
+  /**
+   * Represents a script file.
+   */
+  private static class ScriptFile {
+
+    private final String fileName;
+    private final String extension;
+
+    private ScriptFile(String fileName) {
+      this.fileName = fileName;
+      this.extension = FilenameUtils.getExtension(fileName);
+    }
+
+    public String getFileName() {
+      return fileName;
+    }
+
+    public String getExtension() {
+      return extension;
+    }
+
+    public Reader openReader(SolrResourceLoader resourceLoader) {
+      InputStream input = resourceLoader.openResource(fileName);
+      return org.apache.lucene.util.IOUtils.getDecodingReader
+        (input, org.apache.lucene.util.IOUtils.CHARSET_UTF_8);
+    }
+  }
+}

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/addfields.updateprocessor.js
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/addfields.updateprocessor.js?rev=1360931&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/addfields.updateprocessor.js (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/addfields.updateprocessor.js Thu Jul 12 20:49:41 2012
@@ -0,0 +1,26 @@
+function processAdd(cmd) {
+    // Integer.valueOf is needed here to get a tru java object, because 
+    // all javascript numbers are floating point (ie: java.lang.Double)
+    cmd.getSolrInputDocument().addField("script_added_i", 
+                                        java.lang.Integer.valueOf(42));
+    cmd.getSolrInputDocument().addField("script_added_d", 42.3);
+    
+}
+
+// // // 
+
+function processDelete() {
+    // NOOP
+}
+function processCommit() { 
+    // NOOP
+}
+function processRollback() {
+    // NOOP
+}
+function processMergeIndexes() {
+    // NOOP
+}
+function finish() { 
+    // NOOP
+}

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-bogus-scriptengine-name.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-bogus-scriptengine-name.xml?rev=1360931&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-bogus-scriptengine-name.xml (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-bogus-scriptengine-name.xml Thu Jul 12 20:49:41 2012
@@ -0,0 +1,32 @@
+<?xml version="1.0" ?>
+
+<!--
+ 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.
+-->
+
+<config>
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+
+  <updateRequestProcessorChain name="force-script-engine" default="true">
+    <processor class="solr.StatelessScriptUpdateProcessorFactory">
+      <str name="engine">giberish</str>
+      <str name="script">missleading.extension.updateprocessor.js.txt</str>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+</config>

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-invalid-scriptfile.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-invalid-scriptfile.xml?rev=1360931&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-invalid-scriptfile.xml (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-invalid-scriptfile.xml Thu Jul 12 20:49:41 2012
@@ -0,0 +1,33 @@
+<?xml version="1.0" ?>
+
+<!--
+ 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.
+-->
+
+<config>
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+
+  <updateRequestProcessorChain name="force-script-engine" default="true">
+    <processor class="solr.StatelessScriptUpdateProcessorFactory">
+      <str name="engine">javascript</str>
+      <!-- not parsable as javascript -->
+      <str name="script">currency.xml</str>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+</config>

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-missing-scriptfile.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-missing-scriptfile.xml?rev=1360931&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-missing-scriptfile.xml (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/bad-solrconfig-missing-scriptfile.xml Thu Jul 12 20:49:41 2012
@@ -0,0 +1,31 @@
+<?xml version="1.0" ?>
+
+<!--
+ 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.
+-->
+
+<config>
+
+  <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+
+  <updateRequestProcessorChain name="force-script-engine" default="true">
+    <processor class="solr.StatelessScriptUpdateProcessorFactory">
+      <str name="script">a-file-name-that-does-not-exist.js</str>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+</config>

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/conditional.updateprocessor.js
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/conditional.updateprocessor.js?rev=1360931&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/conditional.updateprocessor.js (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/conditional.updateprocessor.js Thu Jul 12 20:49:41 2012
@@ -0,0 +1,25 @@
+function processAdd(cmd) {
+  if (req.getParams().getBool("go-for-it",false)) {
+    cmd.getSolrInputDocument().addField("script_added_s", "i went for it");
+    return true;
+  }
+  return false;
+}
+
+// // // 
+
+function processDelete() {
+    // NOOP
+}
+function processCommit() { 
+    // NOOP
+}
+function processRollback() {
+    // NOOP
+}
+function processMergeIndexes() {
+    // NOOP
+}
+function finish() { 
+    // NOOP
+}

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/missing.functions.updateprocessor.js
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/missing.functions.updateprocessor.js?rev=1360931&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/missing.functions.updateprocessor.js (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/missing.functions.updateprocessor.js Thu Jul 12 20:49:41 2012
@@ -0,0 +1,3 @@
+function doSomeStuff() {
+  return "This script doesn't contain any update processor functions";
+}

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/missleading.extension.updateprocessor.js.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/missleading.extension.updateprocessor.js.txt?rev=1360931&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/missleading.extension.updateprocessor.js.txt (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/missleading.extension.updateprocessor.js.txt Thu Jul 12 20:49:41 2012
@@ -0,0 +1,23 @@
+function processAdd(cmd) {
+    // Integer.valueOf is needed here to get a tru java object, because 
+    // all javascript numbers are floating point (ie: java.lang.Double)
+    cmd.getSolrInputDocument().addField("script_added_i", 
+                                        java.lang.Integer.valueOf(42));
+    cmd.getSolrInputDocument().addField("script_added_d", 42.3);
+    
+}
+function processDelete() {
+    // NOOP
+}
+function processCommit() { 
+    // NOOP
+}
+function processRollback() {
+    // NOOP
+}
+function processMergeIndexes() {
+    // NOOP
+}
+function finish() { 
+    // NOOP
+}

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-script-updateprocessor.xml
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-script-updateprocessor.xml?rev=1360931&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-script-updateprocessor.xml (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/solrconfig-script-updateprocessor.xml Thu Jul 12 20:49:41 2012
@@ -0,0 +1,112 @@
+<?xml version="1.0" ?>
+
+<!--
+ 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.
+-->
+
+<!--
+   Test Config that for ScriptUpdateProcessor
+
+  -->
+<config>
+  <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
+  <requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>
+  <directoryFactory name="DirectoryFactory" class="${solr.directoryFactory:solr.RAMDirectoryFactory}"/>
+
+  <updateRequestProcessorChain name="force-script-engine" default="true">
+    <processor class="solr.StatelessScriptUpdateProcessorFactory">
+      <str name="engine">javascript</str>
+      <str name="script">missleading.extension.updateprocessor.js.txt</str>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+
+  <updateRequestProcessorChain name="run-no-scripts">
+    <!-- for bypassing all scripts -->
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="single-script">
+    <processor class="solr.StatelessScriptUpdateProcessorFactory">
+      <str name="script">trivial.updateprocessor0.js</str>
+      <lst name="params">
+        <bool name="boolValue">true</bool>
+        <int name="intValue">1</int>
+      </lst>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="dual-scripts-arr">
+    <processor class="solr.StatelessScriptUpdateProcessorFactory">
+      <arr name="script">
+        <str>trivial.updateprocessor0.js</str>
+        <str>trivial.updateprocessor1.js</str>
+      </arr>
+      <lst name="params">
+        <bool name="boolValue">true</bool>
+        <int name="intValue">1</int>
+      </lst>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="dual-scripts-strs">
+    <processor class="solr.StatelessScriptUpdateProcessorFactory">
+      <str name="script">trivial.updateprocessor0.js</str>
+      <str name="script">trivial.updateprocessor1.js</str>
+      <lst name="params">
+        <bool name="boolValue">true</bool>
+        <int name="intValue">1</int>
+      </lst>
+    </processor>
+    <processor class="solr.RunUpdateProcessorFactory" />
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="conditional-scripts">
+    <!-- multiple scripts, 
+         test that the first one can conditionally stop execution -->
+    <processor class="solr.StatelessScriptUpdateProcessorFactory">
+      <str name="script">conditional.updateprocessor.js</str>
+      <str name="script">addfields.updateprocessor.js</str>
+    </processor>
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="conditional-script">
+    <!-- single script, followed by another processor 
+         (that happens to be a script). 
+         test that the first one can conditionally stop execution -->
+    <processor class="solr.StatelessScriptUpdateProcessorFactory">
+      <str name="script">conditional.updateprocessor.js</str>
+    </processor>
+    <processor class="solr.StatelessScriptUpdateProcessorFactory">
+      <str name="script">addfields.updateprocessor.js</str>
+    </processor>
+  </updateRequestProcessorChain>
+
+  <updateRequestProcessorChain name="error-on-add">
+    <processor class="solr.StatelessScriptUpdateProcessorFactory">
+      <str name="script">throw.error.on.add.updateprocessor.js</str>
+    </processor>
+  </updateRequestProcessorChain>
+  <updateRequestProcessorChain name="missing-functions">
+    <processor class="solr.StatelessScriptUpdateProcessorFactory">
+      <str name="script">missing.functions.updateprocessor.js</str>
+    </processor>
+  </updateRequestProcessorChain>
+
+</config>

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/throw.error.on.add.updateprocessor.js
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/throw.error.on.add.updateprocessor.js?rev=1360931&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/throw.error.on.add.updateprocessor.js (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/throw.error.on.add.updateprocessor.js Thu Jul 12 20:49:41 2012
@@ -0,0 +1,21 @@
+function processAdd() {
+  throw "guess what? no-soup-fo-you !!!";
+}
+
+// // // 
+
+function processDelete() {
+    // NOOP
+}
+function processCommit() { 
+    // NOOP
+}
+function processRollback() {
+    // NOOP
+}
+function processMergeIndexes() {
+    // NOOP
+}
+function finish() { 
+    // NOOP
+}

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/trivial.updateprocessor0.js
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/trivial.updateprocessor0.js?rev=1360931&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/trivial.updateprocessor0.js (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/trivial.updateprocessor0.js Thu Jul 12 20:49:41 2012
@@ -0,0 +1,57 @@
+function processAdd(cmd) {
+    functionMessages.add("processAdd0");
+    testCase.assertNotNull(req);
+    testCase.assertNotNull(rsp);
+    testCase.assertNotNull(logger);
+    testCase.assertNotNull(cmd);
+    testCase.assertNotNull(params);
+    testCase.assertTrue(1 == params.get('intValue').intValue());  // had issues with assertTrue(1, params.get('intValue').intValue()) casting to wrong variant
+    testCase.assertTrue(params.get('boolValue').booleanValue());
+
+    // Integer.valueOf is needed here to get a tru java object, because 
+    // all javascript numbers are floating point (ie: java.lang.Double)
+    cmd.getSolrInputDocument().addField("script_added_i", 
+                                        java.lang.Integer.valueOf(42));
+    cmd.getSolrInputDocument().addField("script_added_d", 42.3);
+    
+}
+
+function processDelete(cmd) {
+    functionMessages.add("processDelete0");
+    testCase.assertNotNull(req);
+    testCase.assertNotNull(rsp);
+    testCase.assertNotNull(logger);
+    testCase.assertNotNull(cmd);
+}
+
+function processMergeIndexes(cmd) {
+    functionMessages.add("processMergeIndexes0");
+    testCase.assertNotNull(req);
+    testCase.assertNotNull(rsp);
+    testCase.assertNotNull(logger);
+    testCase.assertNotNull(cmd);
+}
+
+function processCommit(cmd) {
+    functionMessages.add("processCommit0");
+    testCase.assertNotNull(req);
+    testCase.assertNotNull(rsp);
+    testCase.assertNotNull(logger);
+    testCase.assertNotNull(cmd);
+}
+
+function processRollback(cmd) {
+    functionMessages.add("processRollback0");
+    testCase.assertNotNull(req);
+    testCase.assertNotNull(rsp);
+    testCase.assertNotNull(logger);
+    testCase.assertNotNull(cmd);
+}
+
+function finish() {
+    functionMessages.add("finish0");
+    testCase.assertNotNull(req);
+    testCase.assertNotNull(rsp);
+    testCase.assertNotNull(logger);
+}
+

Added: lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/trivial.updateprocessor1.js
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/trivial.updateprocessor1.js?rev=1360931&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/trivial.updateprocessor1.js (added)
+++ lucene/dev/trunk/solr/core/src/test-files/solr/collection1/conf/trivial.updateprocessor1.js Thu Jul 12 20:49:41 2012
@@ -0,0 +1,25 @@
+function processAdd(cmd) {
+    functionMessages.add("processAdd1");
+    
+}
+
+function processDelete(cmd) {
+    functionMessages.add("processDelete1");
+}
+
+function processMergeIndexes(cmd) {
+    functionMessages.add("processMergeIndexes1");
+}
+
+function processCommit(cmd) {
+    functionMessages.add("processCommit1");
+}
+
+function processRollback(cmd) {
+    functionMessages.add("processRollback1");
+}
+
+function finish() {
+    functionMessages.add("finish1");
+}
+

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/AbstractBadConfigTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/AbstractBadConfigTestBase.java?rev=1360931&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/AbstractBadConfigTestBase.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/AbstractBadConfigTestBase.java Thu Jul 12 20:49:41 2012
@@ -0,0 +1,66 @@
+/*
+ * 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.
+ */
+
+package org.apache.solr.core;
+
+import org.apache.solr.SolrTestCaseJ4;
+import org.apache.solr.common.SolrException;
+import org.apache.solr.common.SolrException.ErrorCode;
+
+import java.util.regex.Pattern;
+
+import javax.script.ScriptEngineManager;
+
+import org.junit.Assume;
+
+public abstract class AbstractBadConfigTestBase extends SolrTestCaseJ4 {
+
+  /**
+   * Given a solrconfig.xml file name, a schema file name, and an 
+   * expected errString, asserts that initializing a core with these 
+   * files causes an error matching the specified errString ot be thrown.
+   */
+  protected final void assertConfigs(final String solrconfigFile,
+                                     final String schemaFile, 
+                                     final String errString) 
+    throws Exception {
+
+    ignoreException(Pattern.quote(errString));
+    try {
+      initCore( solrconfigFile, schemaFile );
+    } catch (Exception e) {
+      // short circuit out if we found what we expected
+      if (-1 != e.getMessage().indexOf(errString)) return;
+      // Test the cause too in case the expected error is wrapped by the TestHarness
+      // (NOTE: we don't go all the way down. Either errString should be changed,
+      // or some error wrapping should use a better message or both)
+      if (null != e.getCause() &&
+          null != e.getCause().getMessage() &&
+          -1 != e.getCause().getMessage().indexOf(errString)) return;
+
+      // otherwise, rethrow it, possibly completley unrelated
+      throw new SolrException
+        (ErrorCode.SERVER_ERROR, 
+         "Unexpected error, expected error matching: " + errString, e);
+    } finally {
+      deleteCore();
+      resetExceptionIgnores();
+    }
+    fail("Did not encounter any exception from: " + solrconfigFile + " using " + schemaFile);
+  }
+
+}

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestBadConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestBadConfig.java?rev=1360931&r1=1360930&r2=1360931&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestBadConfig.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/core/TestBadConfig.java Thu Jul 12 20:49:41 2012
@@ -17,31 +17,36 @@
 
 package org.apache.solr.core;
 
-import org.apache.solr.util.AbstractSolrTestCase;
+import javax.script.ScriptEngineManager;
 
-public class TestBadConfig extends AbstractSolrTestCase {
+import org.junit.Assume;
 
-  @Override
-  public String getSchemaFile() { return "schema.xml"; }
-  @Override
-  public String getSolrConfigFile() { return "bad_solrconfig.xml"; }
-
-  @Override
-  public void setUp() throws Exception {
-    ignoreException("unset.sys.property");
-    try {
-      super.setUp();
-      fail("Exception should have been thrown");
-    } catch (Exception e) {
-      assertTrue(e.getMessage().contains("unset.sys.property"));
-    } finally {
-      resetExceptionIgnores();
-    }
+public class TestBadConfig extends AbstractBadConfigTestBase {
+
+  public void testUnsetSysProperty() throws Exception {
+    assertConfigs("bad_solrconfig.xml","schema.xml","unset.sys.property");
+  }
+
+  public void testBogusScriptEngine() throws Exception {
+    // sanity check
+    Assume.assumeTrue(null == (new ScriptEngineManager()).getEngineByName("giberish"));
+                      
+    assertConfigs("bad-solrconfig-bogus-scriptengine-name.xml",
+                  "schema.xml","giberish");
   }
-    
 
-  public void testNothing() {
-    // Empty test case as the real test is that the initialization of the TestHarness fails
-    assertTrue(true);
+  public void testMissingScriptFile() throws Exception {
+    // sanity check
+    Assume.assumeNotNull((new ScriptEngineManager()).getEngineByExtension("js"));
+    assertConfigs("bad-solrconfig-missing-scriptfile.xml",
+                  "schema.xml","a-file-name-that-does-not-exist.js");
   }
-}
\ No newline at end of file
+
+  public void testInvalidScriptFile() throws Exception {
+    // sanity check
+    Assume.assumeNotNull((new ScriptEngineManager()).getEngineByName("javascript"));
+    assertConfigs("bad-solrconfig-invalid-scriptfile.xml",
+                  "schema.xml","currency.xml");
+  }
+
+}

Modified: lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/BadIndexSchemaTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/BadIndexSchemaTest.java?rev=1360931&r1=1360930&r2=1360931&view=diff
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/BadIndexSchemaTest.java (original)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/schema/BadIndexSchemaTest.java Thu Jul 12 20:49:41 2012
@@ -17,38 +17,15 @@
 
 package org.apache.solr.schema;
 
-import org.apache.solr.SolrTestCaseJ4;
-import org.apache.solr.common.SolrException;
-import org.apache.solr.common.SolrException.ErrorCode;
-import org.apache.solr.core.SolrConfig;
+import org.apache.solr.core.AbstractBadConfigTestBase;
 
 import java.util.regex.Pattern;
 
-import org.junit.Test;
-
-public class BadIndexSchemaTest extends SolrTestCaseJ4 {
+public class BadIndexSchemaTest extends AbstractBadConfigTestBase {
 
   private void doTest(final String schema, final String errString) 
     throws Exception {
-
-    ignoreException(Pattern.quote(errString));
-    try {
-      initCore( "solrconfig.xml", schema );
-    } catch (Exception e) {
-      // short circuit out if we found what we expected
-      if (-1 != e.getMessage().indexOf(errString)) return;
-      // Test the cause too in case the expected error is wrapped
-      if (null != e.getCause() && 
-          -1 != e.getCause().getMessage().indexOf(errString)) return;
-
-      // otherwise, rethrow it, possibly completley unrelated
-      throw new SolrException
-        (ErrorCode.SERVER_ERROR, 
-         "Unexpected error, expected error matching: " + errString, e);
-    } finally {
-      deleteCore();
-    }
-    fail("Did not encounter any exception from: " + schema);
+    assertConfigs("solrconfig.xml", schema, errString);
   }
 
   public void testSevereErrorsForInvalidFieldOptions() throws Exception {

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=1360931&r1=1360930&r2=1360931&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 Thu Jul 12 20:49:41 2012
@@ -53,7 +53,7 @@ import org.junit.Test;
  * (mainly via TrimFieldUpdateProcessor) and the logic of other various 
  * subclasses.
  */
-public class FieldMutatingUpdateProcessorTest extends SolrTestCaseJ4 {
+public class FieldMutatingUpdateProcessorTest extends UpdateProcessorTestBase {
 
   @BeforeClass
   public static void beforeClass() throws Exception {
@@ -816,64 +816,4 @@ public class FieldMutatingUpdateProcesso
                  3.0F, d.getField("foo_s").getBoost(), 0.0F);
   }
 
-  /** 
-   * Convenience method for building up SolrInputDocuments
-   */
-  SolrInputDocument doc(SolrInputField... fields) {
-    SolrInputDocument d = new SolrInputDocument();
-    for (SolrInputField f : fields) {
-      d.put(f.getName(), f);
-    }
-    return d;
-  }
-
-  /** 
-   * Convenience method for building up SolrInputFields
-   */
-  SolrInputField field(String name, float boost, Object... values) {
-    SolrInputField f = new SolrInputField(name);
-    for (Object v : values) {
-      f.addValue(v, 1.0F);
-    }
-    f.setBoost(boost);
-    return f;
-  }
-
-  /** 
-   * Convenience method for building up SolrInputFields with default boost
-   */
-  SolrInputField f(String name, Object... values) {
-    return field(name, 1.0F, values);
-  }
-
-
-  /**
-   * Runs a document through the specified chain, and returns the final 
-   * document used when the chain is completed (NOTE: some chains may 
-   * modify the document in place
-   */
-  SolrInputDocument processAdd(final String chain, 
-                               final SolrInputDocument docIn) 
-    throws IOException {
-
-    SolrCore core = h.getCore();
-    UpdateRequestProcessorChain pc = core.getUpdateProcessingChain(chain);
-    assertNotNull("No Chain named: " + chain, pc);
-
-    SolrQueryResponse rsp = new SolrQueryResponse();
-
-    SolrQueryRequest req = new LocalSolrQueryRequest
-      (core, new ModifiableSolrParams());
-    try {
-      AddUpdateCommand cmd = new AddUpdateCommand(req);
-      cmd.solrDoc = docIn;
-
-      UpdateRequestProcessor processor = pc.createProcessor(req, rsp);
-      processor.processAdd(cmd);
-
-      return cmd.solrDoc;
-    } finally {
-      req.close();
-    }
-  }
 }

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/ScriptEngineTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/ScriptEngineTest.java?rev=1360931&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/ScriptEngineTest.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/ScriptEngineTest.java Thu Jul 12 20:49:41 2012
@@ -0,0 +1,95 @@
+/*
+ * 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.
+ */
+
+package org.apache.solr.update.processor;
+
+import org.apache.lucene.util.LuceneTestCase;
+
+import javax.script.Invocable;
+import javax.script.ScriptEngine;
+import javax.script.ScriptEngineManager;
+import javax.script.ScriptException;
+import java.io.StringReader;
+
+/**
+ * Sanity tests basic functionality of {@link ScriptEngineManager} and 
+ * {@link ScriptEngine} w/o excercising any Lucene specific code.
+ */
+public class ScriptEngineTest extends LuceneTestCase {
+
+  private ScriptEngineManager manager;
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+    manager = new ScriptEngineManager();
+  }
+
+  public void testGetEngineByName() {
+    ScriptEngine engine = manager.getEngineByName("JavaScript");
+    assertNotNull(engine);
+    engine = manager.getEngineByName("DummyScript");
+    assertNull(engine);
+  }
+
+  public void testGetEngineByExtension() {
+    ScriptEngine engine = manager.getEngineByExtension("js");
+    assertNotNull(engine);
+    engine = manager.getEngineByExtension("foobar");
+    assertNull(engine);
+  }
+
+  public void testEvalText() throws ScriptException, NoSuchMethodException {
+    ScriptEngine engine = manager.getEngineByName("JavaScript");
+    assertNotNull(engine);
+    engine.eval("function add(a,b) { return a + b }");
+    Double result = (Double) ((Invocable)engine).invokeFunction("add", 1, 2);
+    assertNotNull(result);
+    assertEquals(3, result.intValue());
+  }
+
+  public void testEvalReader() throws ScriptException, NoSuchMethodException {
+    ScriptEngine engine = manager.getEngineByName("JavaScript");
+    assertNotNull(engine);
+    StringReader reader = new StringReader("function add(a,b) { return a + b }");
+    engine.eval(reader);
+    Double result = (Double) ((Invocable)engine).invokeFunction("add", 1, 2);
+    assertNotNull(result);
+    assertEquals(3, result.intValue());
+  }
+
+  public void testPut() throws ScriptException, NoSuchMethodException {
+    manager.put("a", 1);
+    ScriptEngine engine = manager.getEngineByName("JavaScript");
+    engine.put("b", 2);
+    assertNotNull(engine);
+    engine.eval("function add() { return a + b }");
+    Double result = (Double) ((Invocable)engine).invokeFunction("add", 1, 2);
+    assertNotNull(result);
+    assertEquals(3, result.intValue());
+  }
+
+//  public void testJRuby() throws ScriptException, NoSuchMethodException {  // Simply adding jruby.jar to Solr's lib/ directory gets this test passing
+//    ScriptEngine engine = manager.getEngineByName("jruby");
+//    assertNotNull(engine);
+//    engine.eval("def add(a,b); a + b; end");
+//    Long result = (Long) ((Invocable)engine).invokeFunction("add", 1, 2);
+//    assertNotNull(result);
+//    assertEquals(3, result.intValue());
+//  }
+
+}

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactoryTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactoryTest.java?rev=1360931&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactoryTest.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/StatelessScriptUpdateProcessorFactoryTest.java Thu Jul 12 20:49:41 2012
@@ -0,0 +1,272 @@
+package org.apache.solr.update.processor;
+
+/*
+ * 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.solr.common.params.SolrParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.params.UpdateParams;
+import org.apache.solr.common.SolrException;
+
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.util.RefCounted;
+
+import org.junit.Assume;
+import org.junit.BeforeClass;
+
+import javax.script.ScriptEngineManager;
+import javax.script.ScriptEngine;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Tests {@link StatelessScriptUpdateProcessorFactory}.
+ *
+ * TODO: This test, to run from an IDE, requires a working directory of <path-to>/solr/core/src/test-files.  Fix!
+ */
+public class StatelessScriptUpdateProcessorFactoryTest extends UpdateProcessorTestBase {
+
+  @BeforeClass
+  public static void beforeClass() throws Exception {
+    initCore("solrconfig-script-updateprocessor.xml", "schema12.xml");
+
+    Assume.assumeNotNull((new ScriptEngineManager()).getEngineByExtension("js"));
+  }
+
+  /**
+   * simple test of a basic script processor chain using the full 
+   * RequestHandler + UpdateProcessorChain flow
+   */
+  public void testFullRequestHandlerFlow() throws Exception {
+
+    assertU("Simple assertion that adding a document works",
+            adoc("id",  "4055",
+                 "subject", "Hoss"));
+    assertU(commit());
+
+    assertQ("couldn't find hoss using script added field",
+            req("q","script_added_i:[40 TO 45]",
+                "fq","id:4055")
+            ,"//result[@numFound=1]"
+            ,"//str[@name='id'][.='4055']"
+            );
+
+    // clean up
+    processDeleteById("run-no-scripts","4055");
+    processCommit("run-no-scripts");
+    
+  }
+
+  public void testSingleScript() throws Exception {
+    SolrCore core = h.getCore();
+    UpdateRequestProcessorChain chained = core.getUpdateProcessingChain("single-script");
+    final StatelessScriptUpdateProcessorFactory factory = ((StatelessScriptUpdateProcessorFactory) chained.getFactories()[0]);
+    final List<String> functionMessages = new ArrayList<String>();
+    factory.setScriptEngineCustomizer(new ScriptEngineCustomizer() {
+      public void customize(ScriptEngine engine) {
+        engine.put("testCase", StatelessScriptUpdateProcessorFactoryTest.this);
+        engine.put("functionMessages", functionMessages);
+      }
+    });
+    assertNotNull(chained);
+
+    SolrInputDocument d = processAdd("single-script",
+        doc(f("id", "1"),
+            f("name", " foo "),
+            f("subject", "bar")));
+
+    processCommit("run-no-scripts");
+
+    assertQ("couldn't find doc by id",
+            req("q","id:1")
+            , "//result[@numFound=1]");
+
+    processDeleteById("single-script","1");
+    processCommit("single-script");
+    
+    assertQ("found deleted doc",
+            req("q","id:1")
+            , "//result[@numFound=0]");
+
+
+    assertEquals(3, functionMessages.size());
+
+    assertTrue(functionMessages.contains("processAdd0"));
+    assertTrue(functionMessages.contains("processDelete0"));
+    assertTrue(functionMessages.contains("processCommit0"));
+
+  }
+
+  public void testMultipleScripts() throws Exception {
+    SolrCore core = h.getCore();
+
+    for (final String chain : new String[] {"dual-scripts-arr", 
+                                            "dual-scripts-strs"}) {
+    
+      UpdateRequestProcessorChain chained = core.getUpdateProcessingChain(chain);
+      final StatelessScriptUpdateProcessorFactory factory = 
+        ((StatelessScriptUpdateProcessorFactory) chained.getFactories()[0]);
+      final List<String> functionMessages = new ArrayList<String>();
+      ScriptEngineCustomizer customizer = new ScriptEngineCustomizer() {
+          public void customize(ScriptEngine engine) {
+            engine.put("testCase", StatelessScriptUpdateProcessorFactoryTest.this);
+            engine.put("functionMessages", functionMessages);
+          }
+        };
+      factory.setScriptEngineCustomizer(customizer);
+      assertNotNull(chained);
+
+      SolrInputDocument d = processAdd(chain,
+                                       doc(f("id", "2"),
+                                           f("name", " foo "),
+                                           f("subject", "bar")));
+      
+      assertEquals(chain + " didn't add Double field", 
+                   42.3d, d.getFieldValue("script_added_d"));
+      assertEquals(chain + " didn't add integer field", 
+                   new Integer(42), d.getFieldValue("script_added_i"));
+      
+      processCommit("run-no-scripts");
+
+      assertQ(chain + ": couldn't find doc by id",
+              req("q","id:2")
+              , "//result[@numFound=1]");
+
+      processDeleteById(chain, "2");
+      processCommit(chain);
+      
+      assertEquals(chain, 6, functionMessages.size());
+      assertTrue(chain, functionMessages.contains("processAdd0"));
+      assertTrue(chain, functionMessages.contains("processAdd1"));
+      assertTrue(chain + ": script order doesn't match conf order",
+                 functionMessages.indexOf("processAdd0") 
+                 < functionMessages.indexOf("processAdd1"));
+
+      assertTrue(chain, functionMessages.contains("processDelete0"));
+      assertTrue(chain, functionMessages.contains("processDelete1"));
+      assertTrue(chain + ": script order doesn't match conf order",
+                 functionMessages.indexOf("processDelete0") 
+                 < functionMessages.indexOf("processDelete1"));
+
+      assertTrue(chain, functionMessages.contains("processCommit0"));
+      assertTrue(chain, functionMessages.contains("processCommit1"));
+      assertTrue(chain + ": script order doesn't match conf order",
+                 functionMessages.indexOf("processCommit0") 
+                 < functionMessages.indexOf("processCommit1"));
+
+      finish(chain);
+    
+      assertEquals(chain, 8, functionMessages.size());
+
+      assertTrue(chain, functionMessages.contains("finish0"));
+      assertTrue(chain, functionMessages.contains("finish1"));
+      assertTrue(chain + ": script order doesn't match conf order",
+                 functionMessages.indexOf("finish0") 
+                 < functionMessages.indexOf("finish1"));
+
+      assertQ(chain + ": found deleted doc",
+              req("q","id:2")
+              , "//result[@numFound=0]");
+      
+    }
+  }
+
+
+  public void testConditionalExecution() throws Exception {
+    for (String chain : new String[] {"conditional-script", 
+                                      "conditional-scripts"}) {
+
+      ModifiableSolrParams reqParams = new ModifiableSolrParams();
+      
+      SolrInputDocument d = processAdd(chain,
+                                       reqParams,
+                                       doc(f("id", "3"),
+                                           f("name", " foo "),
+                                           f("subject", "bar")));
+      
+      assertFalse(chain + " added String field despite condition", 
+                  d.containsKey("script_added_s"));
+      assertFalse(chain + " added Double field despite condition", 
+                  d.containsKey("script_added_d"));
+      
+      reqParams.add("go-for-it", "true");
+      
+      d = processAdd(chain,
+                     reqParams,
+                     doc(f("id", "4"),
+                         f("name", " foo "),
+                         f("subject", "bar")));
+      
+      assertEquals(chain + " didn't add String field", 
+                   "i went for it", d.getFieldValue("script_added_s"));
+      assertEquals(chain +" didn't add Double field", 
+                   42.3d, d.getFieldValue("script_added_d"));
+      assertEquals(chain + " didn't add integer field", 
+                   new Integer(42), d.getFieldValue("script_added_i"));
+    }
+  }
+
+  public void testForceEngine() throws Exception {
+    Assume.assumeNotNull((new ScriptEngineManager()).getEngineByName("javascript"));
+
+    final String chain = "force-script-engine";
+    SolrInputDocument d = processAdd(chain,
+                                     doc(f("id", "5"),
+                                         f("name", " foo "),
+                                         f("subject", "bar")));
+      
+    assertEquals(chain +" didn't add Double field", 
+                 42.3d, d.getFieldValue("script_added_d"));
+    assertEquals(chain + " didn't add integer field", 
+                 new Integer(42), d.getFieldValue("script_added_i"));
+  }
+
+  public void testPropogatedException() throws Exception  {
+    final String chain = "error-on-add";
+    try {
+      SolrInputDocument d = processAdd(chain,
+                                       doc(f("id", "5"),
+                                           f("name", " foo "),
+                                           f("subject", "bar")));
+    } catch (SolrException e) {
+      assertTrue("Exception doesn't contain script error string: " + e.getMessage(),
+                 0 < e.getMessage().indexOf("no-soup-fo-you"));
+      return;
+    }
+    fail("Did not get exception from script");
+
+  }
+
+  public void testMissingFunctions() throws Exception  {
+    final String chain = "missing-functions";
+    try {
+      SolrInputDocument d = processAdd(chain,
+                                       doc(f("id", "5"),
+                                           f("name", " foo "),
+                                           f("subject", "bar")));
+    } catch (SolrException e) {
+      assertTrue("Exception doesn't contain expected error: " + e.getMessage(),
+                 0 < e.getMessage().indexOf("processAdd"));
+      return;
+    }
+    fail("Did not get exception from script");
+  }
+
+}

Added: lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/UpdateProcessorTestBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/UpdateProcessorTestBase.java?rev=1360931&view=auto
==============================================================================
--- lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/UpdateProcessorTestBase.java (added)
+++ lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/UpdateProcessorTestBase.java Thu Jul 12 20:49:41 2012
@@ -0,0 +1,163 @@
+package org.apache.solr.update.processor;
+
+/*
+ * 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.solr.SolrTestCaseJ4;
+import org.apache.solr.common.params.SolrParams;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.common.SolrInputField;
+import org.apache.solr.common.params.ModifiableSolrParams;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.request.LocalSolrQueryRequest;
+import org.apache.solr.request.SolrQueryRequest;
+import org.apache.solr.response.SolrQueryResponse;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.CommitUpdateCommand;
+import org.apache.solr.update.DeleteUpdateCommand;
+
+import java.io.IOException;
+
+public class UpdateProcessorTestBase extends SolrTestCaseJ4 {
+
+  /**
+   * Runs a document through the specified chain, and returns the final
+   * document used when the chain is completed (NOTE: some chains may
+   * modify the document in place
+   */
+  protected SolrInputDocument processAdd(final String chain,
+                                         final SolrInputDocument docIn)
+    throws IOException {
+
+    return processAdd(chain, new ModifiableSolrParams(), docIn);
+  }
+
+  /**
+   * Runs a document through the specified chain, and returns the final
+   * document used when the chain is completed (NOTE: some chains may
+   * modify the document in place
+   */
+  protected SolrInputDocument processAdd(final String chain,
+                                         final SolrParams requestParams,
+                                         final SolrInputDocument docIn)
+    throws IOException {
+
+    SolrCore core = h.getCore();
+    UpdateRequestProcessorChain pc = core.getUpdateProcessingChain(chain);
+    assertNotNull("No Chain named: " + chain, pc);
+
+    SolrQueryResponse rsp = new SolrQueryResponse();
+
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, requestParams);
+    try {
+      AddUpdateCommand cmd = new AddUpdateCommand(req);
+      cmd.solrDoc = docIn;
+
+      UpdateRequestProcessor processor = pc.createProcessor(req, rsp);
+      processor.processAdd(cmd);
+
+      return cmd.solrDoc;
+    } finally {
+      req.close();
+    }
+  }
+
+  protected void processCommit(final String chain) throws IOException {
+    SolrCore core = h.getCore();
+    UpdateRequestProcessorChain pc = core.getUpdateProcessingChain(chain);
+    assertNotNull("No Chain named: " + chain, pc);
+
+    SolrQueryResponse rsp = new SolrQueryResponse();
+
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, new ModifiableSolrParams());
+
+    CommitUpdateCommand cmd = new CommitUpdateCommand(req,false);
+    UpdateRequestProcessor processor = pc.createProcessor(req, rsp);
+    try {
+      processor.processCommit(cmd);
+    } finally {
+      req.close();
+    }
+  }
+
+  protected void processDeleteById(final String chain, String id) throws IOException {
+    SolrCore core = h.getCore();
+    UpdateRequestProcessorChain pc = core.getUpdateProcessingChain(chain);
+    assertNotNull("No Chain named: " + chain, pc);
+
+    SolrQueryResponse rsp = new SolrQueryResponse();
+
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, new ModifiableSolrParams());
+
+    DeleteUpdateCommand cmd = new DeleteUpdateCommand(req);
+    cmd.setId(id);
+    UpdateRequestProcessor processor = pc.createProcessor(req, rsp);
+    try {
+      processor.processDelete(cmd);
+    } finally {
+      req.close();
+    }
+  }
+
+  protected void finish(final String chain) throws IOException {
+    SolrCore core = h.getCore();
+    UpdateRequestProcessorChain pc = core.getUpdateProcessingChain(chain);
+    assertNotNull("No Chain named: " + chain, pc);
+
+    SolrQueryResponse rsp = new SolrQueryResponse();
+    SolrQueryRequest req = new LocalSolrQueryRequest(core, new ModifiableSolrParams());
+
+    UpdateRequestProcessor processor = pc.createProcessor(req, rsp);
+    try {
+      processor.finish();
+    } finally {
+      req.close();
+    }
+  }
+
+
+  /**
+   * Convenience method for building up SolrInputDocuments
+   */
+  final SolrInputDocument doc(SolrInputField... fields) {
+    SolrInputDocument d = new SolrInputDocument();
+    for (SolrInputField f : fields) {
+      d.put(f.getName(), f);
+    }
+    return d;
+  }
+
+  /**
+   * Convenience method for building up SolrInputFields
+   */
+  final SolrInputField field(String name, float boost, Object... values) {
+    SolrInputField f = new SolrInputField(name);
+    for (Object v : values) {
+      f.addValue(v, 1.0F);
+    }
+    f.setBoost(boost);
+    return f;
+  }
+
+  /**
+   * Convenience method for building up SolrInputFields with default boost
+   */
+  final SolrInputField f(String name, Object... values) {
+    return field(name, 1.0F, values);
+  }
+}