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 2015/11/03 04:10:44 UTC

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

Author: hossman
Date: Tue Nov  3 03:10:44 2015
New Revision: 1712204

URL: http://svn.apache.org/viewvc?rev=1712204&view=rev
Log:
SOLR-8113: CloneFieldUpdateProcessorFactory now supports choosing a dest field name based on a regex pattern and replacement init options. (merge r1712195)

Added:
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/update/processor/CloneFieldUpdateProcessorFactoryTest.java
      - copied unchanged from r1712195, lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/CloneFieldUpdateProcessorFactoryTest.java
Modified:
    lucene/dev/branches/branch_5x/   (props changed)
    lucene/dev/branches/branch_5x/solr/   (props changed)
    lucene/dev/branches/branch_5x/solr/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_5x/solr/core/   (props changed)
    lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/processor/CloneFieldUpdateProcessorFactory.java
    lucene/dev/branches/branch_5x/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml
    lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/update/processor/FieldMutatingUpdateProcessorTest.java

Modified: lucene/dev/branches/branch_5x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/CHANGES.txt?rev=1712204&r1=1712203&r2=1712204&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_5x/solr/CHANGES.txt Tue Nov  3 03:10:44 2015
@@ -78,6 +78,8 @@ New Features
 * SOLR-8217: JSON Facet API: add "method" param to terms/field facets to give an execution
   hint for what method should be used to facet.  (yonik)
 
+* SOLR-8113: CloneFieldUpdateProcessorFactory now supports choosing a "dest" field name based on a regex
+  pattern and replacement init options. (Gus Heck, hossman)
 
 Bug Fixes
 ----------------------

Modified: lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/processor/CloneFieldUpdateProcessorFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/processor/CloneFieldUpdateProcessorFactory.java?rev=1712204&r1=1712203&r2=1712204&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/processor/CloneFieldUpdateProcessorFactory.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/java/org/apache/solr/update/processor/CloneFieldUpdateProcessorFactory.java Tue Nov  3 03:10:44 2015
@@ -21,8 +21,14 @@ import static org.apache.solr.common.Sol
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
 import java.util.HashSet;
 import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import java.util.regex.PatternSyntaxException;
 
 import org.apache.solr.common.SolrException;
 import org.apache.solr.common.SolrInputDocument;
@@ -40,18 +46,25 @@ import org.slf4j.LoggerFactory;
 
 /**
  * Clones the values found in any matching <code>source</code> field into 
- * the configured <code>dest</code> field.
+ * a configured <code>dest</code> field.
  * <p>
- * While the <code>dest</code> field must be a single <code>&lt;str&gt;</code>, 
- * the <code>source</code> fields can be configured as either:
+ * The <code>source</code> field(s) can be configured as either:
  * </p>
  * <ul>
  *  <li>One or more <code>&lt;str&gt;</code></li>
  *  <li>An <code>&lt;arr&gt;</code> of <code>&lt;str&gt;</code></li>
  *  <li>A <code>&lt;lst&gt;</code> containing {@link FieldMutatingUpdateProcessorFactory FieldMutatingUpdateProcessorFactory style selector arguments}</li>
  * </ul>
- * <p>
- * If the <code>dest</code> field already exists in the document, then the 
+ *
+ * <p> The <code>dest</code> field can be a single <code>&lt;str&gt;</code> 
+ * containing the literal name of a destination field, or it may be a <code>&lt;lst&gt;</code> specifying a 
+ * regex <code>pattern</code> and a <code>replacement</code> string. If the pattern + replacement option 
+ * is used the pattern will be matched against all fields matched by the source selector, and the replacement 
+ * string (including any capture groups specified from the pattern) will be evaluated a using 
+ * {@link Matcher#replaceAll(String)} to generate the literal name of the destination field.
+ * </p>
+ *
+ * <p>If the resolved <code>dest</code> field already exists in the document, then the 
  * values from the <code>source</code> fields will be added to it.  The 
  * "boost" value associated with the <code>dest</code> will not be changed, 
  * and any boost specified on the <code>source</code> fields will be ignored.  
@@ -59,14 +72,23 @@ import org.slf4j.LoggerFactory;
  * newly created <code>dest</code> field will have the default boost of 1.0)
  * </p>
  * <p>
- * In the example below, the <code>category</code> field will be cloned 
- * into the <code>category_s</code> field, both the <code>authors</code> and 
- * <code>editors</code> fields will be cloned into the <code>contributors</code>
- * field, and any field with a name ending in <code>_price</code> -- except for 
- * <code>list_price</code> -- will be cloned into the <code>all_prices</code> 
- * field. 
+ * In the example below:
  * </p>
- * <!-- see solrconfig-update-processors-chains.xml for where this is tested -->
+ * <ul>
+ *   <li>The <code>category</code> field will be cloned into the <code>category_s</code> field</li>
+ *   <li>Both the <code>authors</code> and <code>editors</code> fields will be cloned into the 
+ *       <code>contributors</code> field
+ *   </li>
+ *   <li>Any field with a name ending in <code>_price</code> -- except for 
+ *       <code>list_price</code> -- will be cloned into the <code>all_prices</code>
+ *   </li>
+ *   <li>Any field name beginning with feat and ending in s (i.e. feats or features) 
+ *       will be cloned into a field prefixed with key_ and not ending in s. (i.e. key_feat or key_feature)
+ *   </li>
+ * </ul>
+ *
+ * <!-- see solrconfig-update-processors-chains.xml and 
+ *      CloneFieldUpdateProcessorFactoryTest.testCloneFieldExample for where this is tested -->
  * <pre class="prettyprint">
  *   &lt;updateRequestProcessorChain name="multiple-clones"&gt;
  *     &lt;processor class="solr.CloneFieldUpdateProcessorFactory"&gt;
@@ -82,30 +104,87 @@ import org.slf4j.LoggerFactory;
  *     &lt;/processor&gt;
  *     &lt;processor class="solr.CloneFieldUpdateProcessorFactory"&gt;
  *       &lt;lst name="source"&gt;
- *         &lt;str name="fieldRegex"&gt;.*_price&lt;/str&gt;
+ *         &lt;str name="fieldRegex"&gt;.*_price$&lt;/str&gt;
  *         &lt;lst name="exclude"&gt;
  *           &lt;str name="fieldName"&gt;list_price&lt;/str&gt;
  *         &lt;/lst&gt;
  *       &lt;/lst&gt;
  *       &lt;str name="dest"&gt;all_prices&lt;/str&gt;
  *     &lt;/processor&gt;
+ *     &lt;processor class="solr.processor.CloneFieldUpdateProcessorFactory"&gt;
+ *       &lt;lst name="source"&gt;
+ *         &lt;str name="fieldRegex"&gt;^feat(.*)s$&lt;/str&gt;
+ *       &lt;/lst&gt;
+ *       &lt;lst name="dest"&gt;
+ *         &lt;str name="pattern"&gt;^feat(.*)s$&lt;/str&gt;
+ *         &lt;str name="replacement"&gt;key_feat$1&lt;/str&gt;
+ *       &lt;/str&gt;
+ *     &lt;/processor&gt;
  *   &lt;/updateRequestProcessorChain&gt;
  * </pre>
+ *
+ * <p>
+ * In common case situations where you wish to use a single regular expression as both a 
+ * <code>fieldRegex</code> selector and a destination <code>pattern</code>, a "short hand" syntax 
+ * is support for convinience: The <code>pattern</code> and <code>replacement</code> may be specified 
+ * at the top level, omitting <code>source</code> and <code>dest</code> declarations completely, and 
+ * the <code>pattern</code> will be used to construct an equivilent <code>source</code> selector internally.
+ * </p>
+ * <p>
+ * For example, both of the following configurations are equivilent:
+ * </p>
+ * <pre class="prettyprint">
+ * &lt;!-- full syntax --&gt;
+ * &lt;processor class="solr.processor.CloneFieldUpdateProcessorFactory"&gt;
+ *   &lt;lst name="source"&gt;
+ *     &lt;str name="fieldRegex"^gt;$feat(.*)s$&lt;/str&gt;
+ *   &lt;/lst&gt;
+ *   &lt;lst name="dest"&gt;
+ *     &lt;str name="pattern"&gt;^feat(.*)s$&lt;/str&gt;
+ *     &lt;str name="replacement"&gt;key_feat$1&lt;/str&gt;
+ *   &lt;/str&gt;
+ * &lt;/processor&gt;
+ * 
+ * &lt;!-- syntactic sugar syntax --&gt;
+ * &lt;processor class="solr.processor.CloneFieldUpdateProcessorFactory"&gt;
+ *   &lt;str name="pattern"&gt;^feat(.*)s$&lt;/str&gt;
+ *   &lt;str name="replacement"&gt;key_feat$1&lt;/str&gt;
+ * &lt;/processor&gt;
+ * </pre>
+ *
+ * <p>
+ * When cloning multiple fields (or a single multivalued field) into a single valued field, one of the 
+ * {@link FieldValueSubsetUpdateProcessorFactory} implementations configured after the 
+ * <code>CloneFieldUpdateProcessorFactory</code> can be useful to reduce the list of values down to a 
+ * single value.
+ * </p>
+ * 
+ * @see FieldValueSubsetUpdateProcessorFactory
  */
 public class CloneFieldUpdateProcessorFactory 
   extends UpdateRequestProcessorFactory implements SolrCoreAware {
-  
+
   private final static Logger log = LoggerFactory.getLogger(CloneFieldUpdateProcessorFactory.class);
   
   public static final String SOURCE_PARAM = "source";
   public static final String DEST_PARAM = "dest";
-  
+  public static final String PATTERN_PARAM = "pattern";
+  public static final String REPLACEMENT_PARAM = "replacement";
+
   private SelectorParams srcInclusions = new SelectorParams();
   private Collection<SelectorParams> srcExclusions 
     = new ArrayList<>();
 
   private FieldNameSelector srcSelector = null;
+
+  /** 
+   * If pattern is null, this this is a literal field name.  If pattern is non-null then this
+   * is a replacement string that may contain meta-characters (ie: capture group identifiers)
+   * @see #pattern
+   */
   private String dest = null;
+  /** @see #dest */
+  private Pattern pattern = null;
 
   protected final FieldNameSelector getSourceSelector() {
     if (null != srcSelector) return srcSelector;
@@ -117,70 +196,196 @@ public class CloneFieldUpdateProcessorFa
   @SuppressWarnings("unchecked")
   @Override
   public void init(NamedList args) {
-    Object d = args.remove(DEST_PARAM);
-    if (null == d) {
-      throw new SolrException
-        (SERVER_ERROR, "Init param '" + DEST_PARAM + "' must be specified"); 
-    } else if (! (d instanceof CharSequence) ) {
-      throw new SolrException
-        (SERVER_ERROR, "Init param '" + DEST_PARAM + "' must be a string (ie: 'str')");
+
+    // high level (loose) check for which type of config we have.
+    // 
+    // individual init methods do more strict syntax checking
+    if (0 <= args.indexOf(SOURCE_PARAM, 0) && 0 <= args.indexOf(DEST_PARAM, 0) ) {
+      initSourceSelectorSyntax(args);
+    } else if (0 <= args.indexOf(PATTERN_PARAM, 0) && 0 <= args.indexOf(REPLACEMENT_PARAM, 0)) {
+      initSimpleRegexReplacement(args);
+    } else {
+      throw new SolrException(SERVER_ERROR, "A combination of either '" + SOURCE_PARAM + "' + '"+
+                              DEST_PARAM + "', or '" + REPLACEMENT_PARAM + "' + '" +
+                              PATTERN_PARAM + "' init params are mandatory");
+    }
+    
+    if (0 < args.size()) {
+      throw new SolrException(SERVER_ERROR,
+          "Unexpected init param(s): '" +
+              args.getName(0) + "'");
+    }
+
+    super.init(args);
+  }
+
+  /**
+   * init helper method that should only be called when we know for certain that both the 
+   * "source" and "dest" init params do <em>not</em> exist.
+   */
+  @SuppressWarnings("unchecked")
+  private void initSimpleRegexReplacement(NamedList args) {
+    // The syntactic sugar for the case where there is only one regex pattern for source and the same pattern
+    // is used for the destination pattern...
+    //
+    //  pattern != null && replacement != null
+    //    
+    // ...as top level elements, with no other config options specified
+    
+    // if we got here we know we had pattern and replacement, now check for the other two  so that we can give a better
+    // message than "unexpected"
+    if (0 <= args.indexOf(SOURCE_PARAM, 0) || 0 <= args.indexOf(DEST_PARAM, 0) ) {
+      throw new SolrException(SERVER_ERROR,"Short hand syntax must not be mixed with full syntax. Found " + 
+          PATTERN_PARAM + " and " + REPLACEMENT_PARAM + " but also found " + SOURCE_PARAM + " or " + DEST_PARAM);
+    }
+    
+    assert args.indexOf(SOURCE_PARAM, 0) < 0;
+    
+    Object patt = args.remove(PATTERN_PARAM);
+    Object replacement = args.remove(REPLACEMENT_PARAM);
+
+    if (null == patt || null == replacement) {
+      throw new SolrException(SERVER_ERROR, "Init params '" + PATTERN_PARAM + "' and '" +
+                              REPLACEMENT_PARAM + "' are both mandatory if '" + SOURCE_PARAM + "' and '"+
+                              DEST_PARAM + "' are not both specified");
+    }
+
+    if (0 != args.size()) {
+      throw new SolrException(SERVER_ERROR, "Init params '" + REPLACEMENT_PARAM + "' and '" +
+                              PATTERN_PARAM + "' must be children of '" + DEST_PARAM +
+                              "' to be combined with other options.");
+    }
+    
+    if (!(replacement instanceof String)) {
+      throw new SolrException(SERVER_ERROR, "Init param '" + REPLACEMENT_PARAM + "' must be a string (i.e. <str>)");
+    }
+    if (!(patt instanceof String)) {
+      throw new SolrException(SERVER_ERROR, "Init param '" + PATTERN_PARAM + "' must be a string (i.e. <str>)");
+    }
+    
+    dest = replacement.toString();
+    try {
+      this.pattern = Pattern.compile(patt.toString());
+    } catch (PatternSyntaxException pe) {
+      throw new SolrException(SERVER_ERROR, "Init param " + PATTERN_PARAM +
+                              " is not a valid regex pattern: " + patt, pe);
+      
+    }
+    srcInclusions = new SelectorParams();
+    srcInclusions.fieldRegex = Collections.singletonList(this.pattern);
+  }
+ 
+  /**
+   * init helper method that should only be called when we know for certain that both the 
+   * "source" and "dest" init params <em>do</em> exist.
+   */
+  @SuppressWarnings("unchecked")
+  private void initSourceSelectorSyntax(NamedList args) {
+    // Full and complete syntax where source and dest are mandatory.
+    //
+    // source may be a single string or a selector.
+    // dest may be a single string or list containing pattern and replacement
+    //
+    //   source != null && dest != null
+
+    // if we got here we know we had source and dest, now check for the other two so that we can give a better
+    // message than "unexpected"
+    if (0 <= args.indexOf(PATTERN_PARAM, 0) || 0 <= args.indexOf(REPLACEMENT_PARAM, 0) ) {
+      throw new SolrException(SERVER_ERROR,"Short hand syntax must not be mixed with full syntax. Found " +
+          SOURCE_PARAM + " and " + DEST_PARAM + " but also found " + PATTERN_PARAM + " or " + REPLACEMENT_PARAM);
     }
-    dest = d.toString();
 
+    Object d = args.remove(DEST_PARAM);
+    assert null != d;
+    
     List<Object> sources = args.getAll(SOURCE_PARAM);
-    if (0 == sources.size()) {
-      throw new SolrException
-        (SERVER_ERROR, "Init param '" + SOURCE_PARAM + "' must be specified"); 
-    } 
-    if (1 == sources.size() && sources.get(0) instanceof NamedList) {
-      // nested set of selector options
-      NamedList selectorConfig = (NamedList) args.remove(SOURCE_PARAM);
-
-      srcInclusions = parseSelectorParams(selectorConfig);
-
-      List<Object> excList = selectorConfig.getAll("exclude");
-
-      for (Object excObj : excList) {
-        if (null == excObj) {
-          throw new SolrException
-            (SERVER_ERROR, "Init param '" + SOURCE_PARAM + 
-             "' child 'exclude' can not be null"); 
-        }
-        if (! (excObj instanceof NamedList) ) {
-          throw new SolrException
-            (SERVER_ERROR, "Init param '" + SOURCE_PARAM + 
-             "' child 'exclude' must be <lst/>"); 
-        }
-        NamedList exc = (NamedList) excObj;
-        srcExclusions.add(parseSelectorParams(exc));
-        if (0 < exc.size()) {
-          throw new SolrException(SERVER_ERROR, "Init param '" + SOURCE_PARAM + 
-                                  "' has unexpected 'exclude' sub-param(s): '" 
-                                  + selectorConfig.getName(0) + "'");
+    assert null != sources;
+
+    if (1 == sources.size()) {
+      if (sources.get(0) instanceof NamedList) {
+        // nested set of selector options
+        NamedList selectorConfig = (NamedList) args.remove(SOURCE_PARAM);
+
+        srcInclusions = parseSelectorParams(selectorConfig);
+
+        List<Object> excList = selectorConfig.getAll("exclude");
+
+        for (Object excObj : excList) {
+          if (null == excObj) {
+            throw new SolrException(SERVER_ERROR, "Init param '" + SOURCE_PARAM +
+                                    "' child 'exclude' can not be null");
+          }
+          if (!(excObj instanceof NamedList)) {
+            throw new SolrException(SERVER_ERROR, "Init param '" + SOURCE_PARAM +
+                                    "' child 'exclude' must be <lst/>");
+          }
+          NamedList exc = (NamedList) excObj;
+          srcExclusions.add(parseSelectorParams(exc));
+          if (0 < exc.size()) {
+            throw new SolrException(SERVER_ERROR, "Init param '" + SOURCE_PARAM +
+                                    "' has unexpected 'exclude' sub-param(s): '"
+                                    + selectorConfig.getName(0) + "'");
+          }
+          // call once per instance
+          selectorConfig.remove("exclude");
         }
-        // call once per instance
-        selectorConfig.remove("exclude");
-      }
 
-      if (0 < selectorConfig.size()) {
-        throw new SolrException(SERVER_ERROR, "Init param '" + SOURCE_PARAM + 
-                                "' contains unexpected child param(s): '" + 
-                                selectorConfig.getName(0) + "'");
+        if (0 < selectorConfig.size()) {
+          throw new SolrException(SERVER_ERROR, "Init param '" + SOURCE_PARAM +
+                                  "' contains unexpected child param(s): '" +
+                                  selectorConfig.getName(0) + "'");
+        }
+        // consume from the named list so it doesn't interfere with subsequent processing
+        sources.remove(0);
       }
-    } else {
+    }
+    if (1 <= sources.size()) {
       // source better be one or more strings
       srcInclusions.fieldName = new HashSet<>(args.removeConfigArgs("source"));
     }
-
+    if (srcInclusions == null) {
+      throw new SolrException(SERVER_ERROR, "Init params do not specify anything to clone, please supply either "
+      + SOURCE_PARAM + " and " + DEST_PARAM + " or " + PATTERN_PARAM + " and " + REPLACEMENT_PARAM + ". See javadocs" +
+          "for CloneFieldUpdateProcessorFactory for further details.");
+    }
     
+    if (d instanceof NamedList) {
+      NamedList destList = (NamedList) d;
 
-    if (0 < args.size()) {
-      throw new SolrException(SERVER_ERROR, 
-                              "Unexpected init param(s): '" + 
-                              args.getName(0) + "'");
+      Object patt = destList.remove(PATTERN_PARAM);
+      Object replacement = destList.remove(REPLACEMENT_PARAM);
+      
+      if (null == patt || null == replacement) {
+        throw new SolrException(SERVER_ERROR, "Init param '" + DEST_PARAM + "' children '" +
+                                PATTERN_PARAM + "' and '" + REPLACEMENT_PARAM +
+                                "' are both mandatoryand can not be null");
+      }
+      if (! (patt instanceof String && replacement instanceof String)) {
+        throw new SolrException(SERVER_ERROR, "Init param '" + DEST_PARAM + "' children '" +
+                                PATTERN_PARAM + "' and '" + REPLACEMENT_PARAM +
+                                "' must both be strings (i.e. <str>)");
+      }
+      if (0 != destList.size()) {
+        throw new SolrException(SERVER_ERROR, "Init param '" + DEST_PARAM + "' has unexpected children: '"
+                                + destList.getName(0) + "'");
+      }
+      
+      try {
+        this.pattern = Pattern.compile(patt.toString());
+      } catch (PatternSyntaxException pe) {
+        throw new SolrException(SERVER_ERROR, "Init param '" + DEST_PARAM + "' child '" + PATTERN_PARAM +
+                                " is not a valid regex pattern: " + patt, pe);
+      }
+      dest = replacement.toString();
+        
+    } else if (d instanceof String) {
+      dest = d.toString();
+    } else {
+      throw new SolrException(SERVER_ERROR, "Init param '" + DEST_PARAM + "' must either be a string " +
+                              "(i.e. <str>) or a list (i.e. <lst>) containing '" +
+                              PATTERN_PARAM + "' and '" + REPLACEMENT_PARAM);
     }
 
-    super.init(args);
   }
 
   @Override
@@ -202,32 +407,58 @@ public class CloneFieldUpdateProcessorFa
   public final UpdateRequestProcessor getInstance(SolrQueryRequest req,
                                                   SolrQueryResponse rsp,
                                                   UpdateRequestProcessor next) {
+    final FieldNameSelector srcSelector = getSourceSelector();
     return new UpdateRequestProcessor(next) {
       @Override
       public void processAdd(AddUpdateCommand cmd) throws IOException {
 
         final SolrInputDocument doc = cmd.getSolrInputDocument();
 
+        // destination may be regex replace string, which can cause multiple output fields.
+        Map<String,SolrInputField> destMap = new HashMap<>();
+
         // preserve initial values and boost (if any)
-        SolrInputField destField = doc.containsKey(dest) ? 
-          doc.getField(dest) : new SolrInputField(dest); 
-        
-        boolean modified = false;
         for (final String fname : doc.getFieldNames()) {
           if (! srcSelector.shouldMutate(fname)) continue;
           
           Collection<Object> srcFieldValues = doc.getFieldValues(fname);
           if(srcFieldValues == null || srcFieldValues.isEmpty()) continue;
+          
+          String resolvedDest = dest;
+
+          if (pattern != null) {
+            Matcher matcher = pattern.matcher(fname);
+            if (matcher.find()) {
+              resolvedDest = matcher.replaceAll(dest);
+            } else {
+              log.debug("CloneFieldUpdateProcessor.srcSelector.shouldMutate(\"{}\") returned true, " +
+                  "but replacement pattern did not match, field skipped.", fname);
+              continue;
+            }
+          }
+          SolrInputField destField;
+          if (doc.containsKey(resolvedDest)) {
+            destField = doc.getField(resolvedDest);
+          } else {
+            SolrInputField targetField = destMap.get(resolvedDest);
+            if (targetField == null) {
+              destField = new SolrInputField(resolvedDest);
+            } else {
+              destField = targetField;
+            }
+          }
 
           for (Object val : srcFieldValues) {
             // preserve existing dest boost (multiplicitive), ignore src boost
             destField.addValue(val, 1.0f);
           }
-          modified=true;
+          // put it in map to avoid concurrent modification...
+          destMap.put(resolvedDest, destField);
         }
 
-        if (modified) doc.put(dest, destField);
-
+        for (String dest : destMap.keySet()) {
+          doc.put(dest, destMap.get(dest));
+        }
         super.processAdd(cmd);
       }
     };

Modified: lucene/dev/branches/branch_5x/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml?rev=1712204&r1=1712203&r2=1712204&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test-files/solr/collection1/conf/solrconfig-update-processor-chains.xml Tue Nov  3 03:10:44 2015
@@ -292,6 +292,17 @@
       <str name="dest">dest_s</str>
     </processor>
   </updateRequestProcessorChain>
+  
+  <updateRequestProcessorChain name="clone-single-regex">
+    <processor class="solr.CloneFieldUpdateProcessorFactory">
+      <str name="source">source1_s</str>
+      <lst name="dest">
+        <str name="pattern">source\d(_s)</str>
+        <str name="replacement">dest$1</str>
+      </lst>
+    </processor>
+  </updateRequestProcessorChain>
+  
   <updateRequestProcessorChain name="clone-multi">
     <processor class="solr.CloneFieldUpdateProcessorFactory">
       <str name="source">source1_s</str>
@@ -300,6 +311,17 @@
     </processor>
   </updateRequestProcessorChain>
 
+  <updateRequestProcessorChain name="clone-multi-regex">
+    <processor class="solr.CloneFieldUpdateProcessorFactory">
+      <str name="source">source1_s</str>
+      <str name="source">source2_s</str>
+      <lst name="dest">
+        <str name="pattern">source\d(_s)</str>
+        <str name="replacement">dest$1</str>
+      </lst>
+    </processor>
+  </updateRequestProcessorChain>
+
   <updateRequestProcessorChain name="clone-array">
     <processor class="solr.CloneFieldUpdateProcessorFactory">
       <arr name="source">
@@ -310,6 +332,19 @@
     </processor>
   </updateRequestProcessorChain>
 
+  <updateRequestProcessorChain name="clone-array-regex">
+    <processor class="solr.CloneFieldUpdateProcessorFactory">
+      <arr name="source">
+        <str>source1_s</str>
+        <str>source2_s</str>
+      </arr>
+      <lst name="dest">
+        <str name="pattern">source\d(_s)</str>
+        <str name="replacement">dest$1</str>
+      </lst>
+    </processor>
+  </updateRequestProcessorChain>
+
   <updateRequestProcessorChain name="clone-selector">
     <processor class="solr.CloneFieldUpdateProcessorFactory">
       <lst name="source">
@@ -321,6 +356,41 @@
       <str name="dest">dest_s</str>
     </processor>
   </updateRequestProcessorChain>
+  
+  <updateRequestProcessorChain name="clone-selector-regex">
+    <processor class="solr.CloneFieldUpdateProcessorFactory">
+      <lst name="source">
+        <str name="fieldRegex">source\d_.*</str>
+        <lst name="exclude">
+          <str name="fieldRegex">source0_.*</str>
+        </lst>
+      </lst>
+      <lst name="dest">
+        <str name="pattern">source\d(_s)</str>
+        <str name="replacement">dest$1</str>
+      </lst>
+    </processor>
+  </updateRequestProcessorChain>
+  
+  <updateRequestProcessorChain name="clone-simple-regex-syntax">
+    <processor class="solr.CloneFieldUpdateProcessorFactory">
+      <str name="pattern">source\d_.*</str>
+      <str name="replacement">dest_s</str>
+    </processor>
+  </updateRequestProcessorChain>
+  
+  <updateRequestProcessorChain name="clone-regex-replaceall">
+    <processor class="solr.CloneFieldUpdateProcessorFactory">
+      <lst name="source">
+        <str name="fieldRegex">foo.*</str>
+      </lst>
+      <lst name="dest">
+        <!-- unbounded pattern that can be replaced multiple times in field name -->
+        <str name="pattern">x(\d)</str>
+        <str name="replacement">y$1</str>
+      </lst>
+    </processor>
+  </updateRequestProcessorChain>
 
   <updateRequestProcessorChain name="clone-max-chars">
     <processor class="solr.CloneFieldUpdateProcessorFactory">
@@ -364,6 +434,16 @@
       <str name="fieldName">toField</str>
     </processor>
   </updateRequestProcessorChain>
+  
+  <updateRequestProcessorChain name="clone-first">
+    <processor class="solr.CloneFieldUpdateProcessorFactory">
+      <str name="pattern">field(.*)</str>
+      <str name="replacement">toField</str>
+    </processor>
+    <processor class="solr.FirstFieldValueUpdateProcessorFactory">
+      <str name="fieldName">toField</str>
+    </processor>
+  </updateRequestProcessorChain>
 
   <!-- example used in CloneFieldUpdateProcessorFactory javadocs -->
   <updateRequestProcessorChain name="multiple-clones">
@@ -380,13 +460,27 @@
     </processor>
     <processor class="solr.CloneFieldUpdateProcessorFactory">
       <lst name="source">
-        <str name="fieldRegex">.*_price</str>
+        <str name="fieldRegex">.*_price$</str>
         <lst name="exclude">
           <str name="fieldName">list_price</str>
         </lst>
       </lst>
       <str name="dest">all_prices</str>
     </processor>
+    <processor class="solr.processor.CloneFieldUpdateProcessorFactory">
+      <lst name="source">
+        <str name="fieldRegex">^feat(.*)s$</str>
+      </lst>
+      <lst name="dest">
+        <str name="pattern">^feat(.*)s$</str>
+        <str name="replacement">key_feat$1</str>
+      </lst>
+    </processor>
+    <!-- equivilent sugar syntax to above, with slightly diff destination name -->
+    <processor class="solr.processor.CloneFieldUpdateProcessorFactory">
+      <str name="pattern">^feat(.*)s$</str>
+      <str name="replacement">best_feat$1</str>
+    </processor>
   </updateRequestProcessorChain>
 
   <updateRequestProcessorChain name="clone-then-count">

Modified: lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/update/processor/FieldMutatingUpdateProcessorTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/update/processor/FieldMutatingUpdateProcessorTest.java?rev=1712204&r1=1712203&r2=1712204&view=diff
==============================================================================
--- lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/update/processor/FieldMutatingUpdateProcessorTest.java (original)
+++ lucene/dev/branches/branch_5x/solr/core/src/test/org/apache/solr/update/processor/FieldMutatingUpdateProcessorTest.java Tue Nov  3 03:10:44 2015
@@ -728,120 +728,6 @@ public class FieldMutatingUpdateProcesso
 
   }
 
-  public void testCloneField() throws Exception {
-
-    SolrInputDocument d = null;
-
-    // regardless of chain, all of these should be equivilent
-    for (String chain : Arrays.asList("clone-single", "clone-multi", 
-                                      "clone-array","clone-selector" )) {
-
-      // simple clone
-      d = processAdd(chain,       
-                     doc(f("id", "1111"),
-                         f("source0_s", "NOT COPIED"),
-                         f("source1_s", "123456789", "", 42, "abcd")));
-      assertNotNull(chain, d);
-      assertEquals(chain,
-                   Arrays.asList("123456789", "", 42, "abcd"), 
-                   d.getFieldValues("source1_s"));
-      assertEquals(chain,
-                   Arrays.asList("123456789", "", 42, "abcd"), 
-                   d.getFieldValues("dest_s"));
-
-      // append to existing values, preserve boost
-      d = processAdd(chain,       
-                     doc(f("id", "1111"),
-                         field("dest_s", 2.3f, "orig1", "orig2"),
-                         f("source0_s", "NOT COPIED"),
-                         f("source1_s", "123456789", "", 42, "abcd")));
-      assertNotNull(chain, d);
-      assertEquals(chain,
-                   Arrays.asList("123456789", "", 42, "abcd"), 
-                   d.getFieldValues("source1_s"));
-      assertEquals(chain,
-                   Arrays.asList("orig1", "orig2", "123456789", "", 42, "abcd"),
-                   d.getFieldValues("dest_s"));
-      assertEquals(chain + ": dest boost changed", 
-                   2.3f, d.getField("dest_s").getBoost(), 0.0f);
-    }
-
-    // should be equivilent for any chain matching source1_s and source2_s
-    for (String chain : Arrays.asList("clone-multi",
-                                      "clone-array","clone-selector" )) {
-
-      // simple clone
-      d = processAdd(chain,       
-                     doc(f("id", "1111"),
-                         f("source0_s", "NOT COPIED"),
-                         f("source1_s", "123456789", "", 42, "abcd"),
-                         f("source2_s", "xxx", 999)));
-      assertNotNull(chain, d);
-      assertEquals(chain,
-                   Arrays.asList("123456789", "", 42, "abcd"), 
-                   d.getFieldValues("source1_s"));
-      assertEquals(chain,
-                   Arrays.asList("xxx", 999),
-                   d.getFieldValues("source2_s"));
-      assertEquals(chain,
-                   Arrays.asList("123456789", "", 42, "abcd", "xxx", 999), 
-                   d.getFieldValues("dest_s"));
-
-      // append to existing values, preserve boost
-      d = processAdd(chain,       
-                     doc(f("id", "1111"),
-                         field("dest_s", 2.3f, "orig1", "orig2"),
-                         f("source0_s", "NOT COPIED"),
-                         f("source1_s", "123456789", "", 42, "abcd"),
-                         f("source2_s", "xxx", 999)));
-      assertNotNull(chain, d);
-      assertEquals(chain,
-                   Arrays.asList("123456789", "", 42, "abcd"), 
-                   d.getFieldValues("source1_s"));
-      assertEquals(chain,
-                   Arrays.asList("xxx", 999),
-                   d.getFieldValues("source2_s"));
-      assertEquals(chain,
-                   Arrays.asList("orig1", "orig2", 
-                                 "123456789", "", 42, "abcd",
-                                 "xxx", 999),
-                   d.getFieldValues("dest_s"));
-      assertEquals(chain + ": dest boost changed", 
-                   2.3f, d.getField("dest_s").getBoost(), 0.0f);
-    }
-  }
-
-  public void testCloneFieldExample() throws Exception {
-
-    SolrInputDocument d = null;
-
-    // test example from the javadocs
-    d = processAdd("multiple-clones",       
-                   doc(f("id", "1111"),
-                       f("category", "misc"),
-                       f("authors", "Isaac Asimov", "John Brunner"),
-                       f("editors", "John W. Campbell"),
-                       f("store1_price", 87),
-                       f("store2_price", 78),
-                       f("store3_price", (Object) null),
-                       f("list_price", 1000)));
-    assertNotNull(d);
-    assertEquals("misc",d.getFieldValue("category"));
-    assertEquals("misc",d.getFieldValue("category_s"));
-    assertEquals(Arrays.asList("Isaac Asimov", "John Brunner"),
-                 d.getFieldValues("authors"));
-    assertEquals(Arrays.asList("John W. Campbell"),
-                 d.getFieldValues("editors"));
-    assertEquals(Arrays.asList("Isaac Asimov", "John Brunner", 
-                               "John W. Campbell"),
-                 d.getFieldValues("contributors"));
-    assertEquals(87,d.getFieldValue("store1_price"));
-    assertEquals(78,d.getFieldValue("store2_price"));
-    assertEquals(1000,d.getFieldValue("list_price"));
-    assertEquals(Arrays.asList(87, 78),
-                 d.getFieldValues("all_prices"));
-
-  } 
 
   public void testCountValues() throws Exception {
 
@@ -895,47 +781,6 @@ public class FieldMutatingUpdateProcesso
 
   } 
 
-  public void testCloneCombinations() throws Exception {
-
-    SolrInputDocument d = null;
-
-    // maxChars
-    d = processAdd("clone-max-chars",
-                   doc(f("id", "1111"),
-                       f("field1", "text")));
-    assertNotNull(d);
-    assertEquals("text",d.getFieldValue("field1"));
-    assertEquals("tex",d.getFieldValue("toField"));
-
-    // move
-    d = processAdd("clone-move",
-                   doc(f("id", "1111"),
-                       f("field1", "text")));
-    assertNotNull(d);
-    assertEquals("text",d.getFieldValue("toField"));
-    assertFalse(d.containsKey("field1"));
-
-    // replace
-    d = processAdd("clone-replace",
-                   doc(f("id", "1111"),
-                       f("toField", "IGNORED"),
-                       f("field1", "text")));
-    assertNotNull(d);
-    assertEquals("text", d.getFieldValue("field1"));
-    assertEquals("text", d.getFieldValue("toField"));
-
-    // append
-    d = processAdd("clone-append",
-                   doc(f("id", "1111"),
-                       f("toField", "aaa"),
-                       f("field1", "bbb"),
-                       f("field2", "ccc")));
-    assertNotNull(d);
-    assertEquals("bbb", d.getFieldValue("field1"));
-    assertEquals("ccc", d.getFieldValue("field2"));
-    assertEquals("aaa; bbb; ccc", d.getFieldValue("toField"));
-  } 
-
   public void testConcatDefaults() throws Exception {
     SolrInputDocument d = null;
     d = processAdd("concat-defaults",