You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@lucene.apache.org by Robert Muir <rc...@gmail.com> on 2012/02/09 21:46:21 UTC

Re: svn commit: r1242514 [1/2] - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/update/processor/ core/src/test-files/solr/conf/ core/src/test/org/apache/solr/update/processor/

Can we improve this? Both Min and MaxFieldValueUpdateProcessorFactory
show up as a compile error in eclipse, which is frustrating to people
who use those IDEs.

While it could be a bug in the eclipse compiler, this code is
definitely on shaky ground, I don't understand how a
ClassCastException is OK?!

  @Override
  @SuppressWarnings("unchecked")
  public Collection<Object> pickSubset(Collection<Object> values) {
    Collection<Object> result = values;
    try {
      result = Collections.singletonList
        (Collections.max((Collection)values));
    } catch (ClassCastException e) {
      /* NOOP */
    }
    return result;
  }


On Thu, Feb 9, 2012 at 3:41 PM,  <ho...@apache.org> wrote:
> Author: hossman
> Date: Thu Feb  9 20:41:21 2012
> New Revision: 1242514
>
> URL: http://svn.apache.org/viewvc?rev=1242514&view=rev
> Log:
> SOLR-2802: several new UpdateProcessorFactories for modifing fields of documents, along with base classes to make writing these types of classes easier for users
>
> Added:
>    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/ConcatFieldUpdateProcessorFactory.java   (with props)
>    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldLengthUpdateProcessorFactory.java   (with props)
>    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java   (with props)
>    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessorFactory.java   (with props)
>    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldValueMutatingUpdateProcessor.java   (with props)
>    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldValueSubsetUpdateProcessorFactory.java   (with props)
>    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FirstFieldValueUpdateProcessorFactory.java   (with props)
>    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/HTMLStripFieldUpdateProcessorFactory.java   (with props)
>    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/LastFieldValueUpdateProcessorFactory.java   (with props)
>    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/MaxFieldValueUpdateProcessorFactory.java   (with props)
>    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/MinFieldValueUpdateProcessorFactory.java   (with props)
>    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/RegexReplaceProcessorFactory.java   (with props)
>    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/RemoveBlankFieldUpdateProcessorFactory.java   (with props)
>    lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/TrimFieldUpdateProcessorFactory.java   (with props)
>    lucene/dev/trunk/solr/core/src/test-files/solr/conf/solrconfig-update-processor-chains.xml   (with props)
>    lucene/dev/trunk/solr/core/src/test/org/apache/solr/update/processor/FieldMutatingUpdateProcessorTest.java   (with props)
> Modified:
>    lucene/dev/trunk/solr/CHANGES.txt
>
> Modified: lucene/dev/trunk/solr/CHANGES.txt
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1242514&r1=1242513&r2=1242514&view=diff
> ==============================================================================
> --- lucene/dev/trunk/solr/CHANGES.txt (original)
> +++ lucene/dev/trunk/solr/CHANGES.txt Thu Feb  9 20:41:21 2012
> @@ -207,6 +207,21 @@ New Features
>  * SOLR-1726: Added deep paging support to search (sort by score only) which should use less memory when paging deeply into results
>  by keeping the priority queue small. (Manojkumar Rangasamy Kannadasan, gsingers)
>
> +* SOLR-2802: New FieldMutatingUpdateProcessor and Factory to simlify the
> +  development of UpdateProcessors that modify field values of documents as
> +  they are indexed. Also includes several useful new implementations:
> +      RemoveBlankFieldUpdateProcessorFactory
> +      TrimFieldUpdateProcessorFactory
> +      HTMLStripFieldUpdateProcessorFactory
> +      RegexReplaceProcessorFactory
> +      FieldLengthUpdateProcessorFactory
> +      ConcatFieldUpdateProcessorFactory
> +      FirstFieldValueUpdateProcessorFactory
> +      LastFieldValueUpdateProcessorFactory
> +      MinFieldValueUpdateProcessorFactory
> +      MaxFieldValueUpdateProcessorFactory
> +  (hossman, janhoy)
> +
>  Optimizations
>  ----------------------
>
>
> Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/ConcatFieldUpdateProcessorFactory.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/ConcatFieldUpdateProcessorFactory.java?rev=1242514&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/ConcatFieldUpdateProcessorFactory.java (added)
> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/ConcatFieldUpdateProcessorFactory.java Thu Feb  9 20:41:21 2012
> @@ -0,0 +1,124 @@
> +/**
> + * 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.solr.core.SolrCore;
> +import org.apache.solr.schema.IndexSchema;
> +import org.apache.solr.schema.FieldType;
> +import org.apache.solr.schema.SchemaField;
> +import org.apache.solr.schema.TextField;
> +import org.apache.solr.schema.StrField;
> +
> +import org.apache.solr.common.SolrInputField;
> +import org.apache.solr.common.util.NamedList;
> +import org.apache.solr.request.SolrQueryRequest;
> +import org.apache.solr.response.SolrQueryResponse;
> +
> +import org.apache.commons.lang.StringUtils;
> +
> +/**
> + * Concatenates multiple values for fields matching the specified
> + * conditions using a configurable <code>delimiter</code> which defaults
> + * to "<code> ,</code>".
> + * <p>
> + * By default, this processor concatenates the values for any field name
> + * which according to the schema is <code>multiValued="false"</code>
> + * and uses <code>TextField</code> or <code>StrField</code>
> + * </p>
> + *
> + * <p>
> + * For example, in the configuration below, any "single valued" string and
> + * text field which is found to contain multiple values <i>except</i> for
> + * the <code>primary_author</code> field will be concatenated using the
> + * string "<code> ;</code>" as a delimeter.  For the
> + * <code>primary_author</code> field, the multiple values will be left
> + * alone for <code>FirstFieldValueUpdateProcessorFactory</code> to deal with.
> + * </p>
> + *
> + * <pre class="prettyprint">
> + *  &lt;updateRequestProcessorChain&gt;
> + *    &lt;processor class="solr.ConcatFieldUpdateProcessorFactory"&gt;
> + *      &lt;str name="delimiter"&gt;; &lt;/str&gt;
> + *      &lt;lst name="exclude"&gt;
> + *        &lt;str name="fieldName"&gt;primary_author&lt;/str&gt;
> + *      &lt;/lst&gt;
> + *    &lt;/processor&gt;
> + *    &lt;processor class="solr.FirstFieldValueUpdateProcessorFactory"&gt;
> + *      &lt;str name="fieldName"&gt;primary_author&lt;/str&gt;
> + *    &lt;/processor&gt;
> + *  &lt;/updateRequestProcessorChain&gt;
> + * </pre>
> + */
> +public final class ConcatFieldUpdateProcessorFactory extends FieldMutatingUpdateProcessorFactory {
> +
> +  String delimiter = ", ";
> +
> +  @SuppressWarnings("unchecked")
> +  @Override
> +  public void init(NamedList args) {
> +    Object d = args.remove("delimiter");
> +    if (null != d) delimiter = d.toString();
> +
> +    super.init(args);
> +  }
> +
> +  @Override
> +  public UpdateRequestProcessor getInstance(SolrQueryRequest req,
> +                                            SolrQueryResponse rsp,
> +                                            UpdateRequestProcessor next) {
> +    return new FieldMutatingUpdateProcessor(getSelector(), next) {
> +      protected SolrInputField mutate(final SolrInputField src) {
> +        if (src.getValueCount() <= 1) return src;
> +
> +        SolrInputField result = new SolrInputField(src.getName());
> +        result.setValue(StringUtils.join(src.getValues(), delimiter),
> +                        src.getBoost());
> +        return result;
> +      }
> +    };
> +  }
> +
> +  @Override
> +  public FieldMutatingUpdateProcessor.FieldNameSelector
> +    getDefaultSelector(final SolrCore core) {
> +
> +    final IndexSchema schema = core.getSchema();
> +    return new FieldMutatingUpdateProcessor.FieldNameSelector() {
> +      public boolean shouldMutate(final String fieldName) {
> +
> +        // first check type since it should be fastest
> +        FieldType type = schema.getFieldTypeNoEx(fieldName);
> +        if (null == type) return false;
> +
> +        if (! (TextField.class.isInstance(type)
> +               || StrField.class.isInstance(type))) {
> +          return false;
> +        }
> +
> +        // only ask for SchemaField if we passed the type check.
> +        SchemaField sf = schema.getFieldOrNull(fieldName);
> +        // shouldn't be null since since type wasn't, but just in case
> +        if (null == sf) return false;
> +
> +        return ! sf.multiValued();
> +      }
> +    };
> +  }
> +
> +}
> +
>
> Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldLengthUpdateProcessorFactory.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldLengthUpdateProcessorFactory.java?rev=1242514&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldLengthUpdateProcessorFactory.java (added)
> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldLengthUpdateProcessorFactory.java Thu Feb  9 20:41:21 2012
> @@ -0,0 +1,80 @@
> +/**
> + * 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.solr.common.util.NamedList;
> +import org.apache.solr.request.SolrQueryRequest;
> +import org.apache.solr.response.SolrQueryResponse;
> +import org.apache.solr.core.SolrCore;
> +
> +
> +/**
> + * Replaces any CharSequence values found in fields matching the specified
> + * conditions with the lengths of those CharSequences (as an Integer).
> + * <p>
> + * By default, this processor matches no fields.
> + * </p>
> + * <p>For example, with the configuration listed below any documents
> + * containing  String values (such as "<code>abcdef</code>" or
> + * "<code>xyz</code>") in a field declared in the schema using
> + * <code>TrieIntField</code> or <code>TrieLongField</code>
> + * would have those Strings replaced with the length of those fields as an
> + * Integer
> + * (ie: <code>6</code> and <code>3</code> respectively)
> + * </p>
> + * <pre class="prettyprint">
> + * &lt;processor class="solr.FieldLengthUpdateProcessorFactory"&gt;
> + *   &lt;arr name="typeClass"&gt;
> + *     &lt;str&gt;solr.TrieIntField&lt;/str&gt;
> + *     &lt;str&gt;solr.TrieLongField&lt;/str&gt;
> + *   &lt;/arr&gt;
> + * &lt;/processor&gt;
> + * </pre>
> + */
> +public final class FieldLengthUpdateProcessorFactory extends FieldMutatingUpdateProcessorFactory {
> +
> +  @SuppressWarnings("unchecked")
> +  @Override
> +  public void init(NamedList args) {
> +    // no length specific init args
> +    super.init(args);
> +  }
> +
> +  @Override
> +  public FieldMutatingUpdateProcessor.FieldNameSelector
> +    getDefaultSelector(final SolrCore core) {
> +
> +    return FieldMutatingUpdateProcessor.SELECT_NO_FIELDS;
> +
> +  }
> +
> +  @Override
> +  public UpdateRequestProcessor getInstance(SolrQueryRequest req,
> +                                            SolrQueryResponse rsp,
> +                                            UpdateRequestProcessor next) {
> +    return new FieldValueMutatingUpdateProcessor(getSelector(), next) {
> +      protected Object mutateValue(final Object src) {
> +        if (src instanceof CharSequence) {
> +          return new Integer(((CharSequence)src).length());
> +        }
> +        return src;
> +      }
> +    };
> +  }
> +}
> +
>
> Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java?rev=1242514&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java (added)
> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessor.java Thu Feb  9 20:41:21 2012
> @@ -0,0 +1,283 @@
> +/**
> + * 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 java.io.IOException;
> +import java.util.Arrays;
> +import java.util.ArrayList;
> +import java.util.Collection;
> +import java.util.Collections;
> +import java.util.HashSet;
> +import java.util.List;
> +import java.util.Set;
> +import java.util.regex.Pattern;
> +import java.util.regex.PatternSyntaxException;
> +
> +import static org.apache.solr.common.SolrException.ErrorCode.*;
> +
> +import org.apache.solr.common.SolrInputDocument;
> +import org.apache.solr.common.SolrInputField;
> +import org.apache.solr.common.SolrException;
> +import org.apache.solr.core.SolrResourceLoader;
> +import org.apache.solr.schema.IndexSchema;
> +import org.apache.solr.schema.FieldType;
> +
> +import org.apache.solr.request.SolrQueryRequest;
> +import org.apache.solr.response.SolrQueryResponse;
> +import org.apache.solr.update.AddUpdateCommand;
> +
> +/**
> + * Reusable base class for UpdateProcessors that will consider
> + * AddUpdateCommands and mutate the values assocaited with configured
> + * fields.
> + * <p>
> + * Subclasses should override the mutate method to specify how individual
> + * SolrInputFields identified by the selector associated with this instance
> + * will be mutated.
> + * </p>
> + *
> + * @see FieldMutatingUpdateProcessorFactory
> + * @see FieldValueMutatingUpdateProcessor
> + * @see FieldNameSelector
> + */
> +public abstract class FieldMutatingUpdateProcessor
> +  extends UpdateRequestProcessor {
> +
> +  private final FieldNameSelector selector;
> +  public FieldMutatingUpdateProcessor(FieldNameSelector selector,
> +                                      UpdateRequestProcessor next) {
> +    super(next);
> +    this.selector = selector;
> +  }
> +
> +  /**
> +   * Method for mutating SolrInputFields associated with fields identified
> +   * by the FieldNameSelector associated with this processor
> +   * @param src the SolrInputField to mutate, may be modified in place and
> +   *            returned
> +   * @return the SolrInputField to use in replacing the original (src) value.
> +   *         If null the field will be removed.
> +   */
> +  protected abstract SolrInputField mutate(final SolrInputField src);
> +
> +  @Override
> +  public void processAdd(AddUpdateCommand cmd) throws IOException {
> +    final SolrInputDocument doc = cmd.getSolrInputDocument();
> +
> +    // make a copy we can iterate over while mutating the doc
> +    final Collection<String> fieldNames
> +      = new ArrayList<String>(doc.getFieldNames());
> +
> +    for (final String fname : fieldNames) {
> +
> +      if (! selector.shouldMutate(fname)) continue;
> +
> +      final SolrInputField src = doc.get(fname);
> +      final SolrInputField dest = mutate(src);
> +      if (null == dest) {
> +        doc.remove(fname);
> +      } else {
> +        // semantics of what happens if dest has diff name are hard
> +        // we could treat it as a copy, or a rename
> +        // for now, don't allow it.
> +        if (! fname.equals(dest.getName()) ) {
> +          throw new SolrException(SERVER_ERROR,
> +                                  "mutute returned field with different name: "
> +                                  + fname + " => " + dest.getName());
> +        }
> +        doc.put(dest.getName(), dest);
> +      }
> +    }
> +    super.processAdd(cmd);
> +  }
> +
> +  /**
> +   * Interface for idenfifying which fileds should be mutated
> +   */
> +  public static interface FieldNameSelector {
> +    public boolean shouldMutate(final String fieldName);
> +  }
> +
> +  /** Singleton indicating all fields should be mutated */
> +  public static final FieldNameSelector SELECT_ALL_FIELDS
> +    = new FieldNameSelector() {
> +        public boolean shouldMutate(final String fieldName) {
> +          return true;
> +        }
> +      };
> +
> +  /** Singleton indicating no fields should be mutated */
> +  public static final FieldNameSelector SELECT_NO_FIELDS
> +    = new FieldNameSelector() {
> +        public boolean shouldMutate(final String fieldName) {
> +          return false;
> +        }
> +      };
> +
> +  /**
> +   * Wraps two FieldNameSelectors such that the FieldNameSelector
> +   * returned matches all fields specified by the "includes" unless they
> +   * are matched by "excludes"
> +   * @param includes a selector identifying field names that should be selected
> +   * @param excludes a selector identifying field names that should be
> +   *        <i>not</i> be selected, even if they are matched by the 'includes'
> +   *        selector
> +   * @return Either a new FieldNameSelector or one of the input selecors
> +   *         if the combination lends itself to optimization.
> +   */
> +  public static FieldNameSelector wrap(final FieldNameSelector includes,
> +                                       final FieldNameSelector excludes) {
> +
> +    if (SELECT_NO_FIELDS == excludes) {
> +      return includes;
> +    }
> +
> +    if (SELECT_ALL_FIELDS == excludes) {
> +      return SELECT_NO_FIELDS;
> +    }
> +
> +    if (SELECT_ALL_FIELDS == includes) {
> +      return new FieldNameSelector() {
> +        public boolean shouldMutate(final String fieldName) {
> +          return ! excludes.shouldMutate(fieldName);
> +        }
> +      };
> +    }
> +
> +    return new FieldNameSelector() {
> +      public boolean shouldMutate(final String fieldName) {
> +        return (includes.shouldMutate(fieldName)
> +                && ! excludes.shouldMutate(fieldName));
> +      }
> +    };
> +  }
> +
> +  /**
> +   * Utility method that can be used to define a FieldNameSelector
> +   * using the same types of rules as the FieldMutatingUpdateProcessor init
> +   * code.  This may be useful for Factories that wish to define default
> +   * selectors in similar terms to what the configuration would look like.
> +   * @lucene.internal
> +   */
> +  public static FieldNameSelector createFieldNameSelector
> +    (final SolrResourceLoader loader,
> +     final IndexSchema schema,
> +     final Set<String> fields,
> +     final Set<String> typeNames,
> +     final Collection<String> typeClasses,
> +     final Collection<Pattern> regexes,
> +     final FieldNameSelector defSelector) {
> +
> +    final Collection<Class> classes
> +      = new ArrayList<Class>(typeClasses.size());
> +
> +    for (String t : typeClasses) {
> +      try {
> +        classes.add(loader.findClass(t));
> +      } catch (Exception e) {
> +        throw new SolrException(SERVER_ERROR,
> +                                "Can't resolve typeClass: " + t, e);
> +      }
> +    }
> +
> +    if (classes.isEmpty() &&
> +        typeNames.isEmpty() &&
> +        regexes.isEmpty() &&
> +        fields.isEmpty()) {
> +      return defSelector;
> +    }
> +
> +    return new ConfigurableFieldNameSelector
> +      (schema, fields, typeNames, classes, regexes);
> +  }
> +
> +  private static final class ConfigurableFieldNameSelector
> +    implements FieldNameSelector {
> +
> +    final IndexSchema schema;
> +    final Set<String> fields;
> +    final Set<String> typeNames;
> +    final Collection<Class> classes;
> +    final Collection<Pattern> regexes;
> +
> +    private ConfigurableFieldNameSelector(final IndexSchema schema,
> +                                          final Set<String> fields,
> +                                          final Set<String> typeNames,
> +                                          final Collection<Class> classes,
> +                                          final Collection<Pattern> regexes) {
> +      this.schema = schema;
> +      this.fields = fields;
> +      this.typeNames = typeNames;
> +      this.classes = classes;
> +      this.regexes = regexes;
> +    }
> +
> +    public boolean shouldMutate(final String fieldName) {
> +
> +      // order of checks is bsaed on what should be quicker
> +      // (ie: set lookups faster the looping over instanceOf / matches tests
> +
> +      if ( ! (fields.isEmpty() || fields.contains(fieldName)) ) {
> +        return false;
> +      }
> +
> +      // do not consider it an error if the fieldName has no type
> +      // there might be another processor dealing with it later
> +      FieldType t = schema.getFieldTypeNoEx(fieldName);
> +      if (null != t) {
> +        if (! (typeNames.isEmpty() || typeNames.contains(t.getTypeName())) ) {
> +          return false;
> +        }
> +
> +        if (! (classes.isEmpty() || instanceOfAny(t, classes)) ) {
> +          return false;
> +          }
> +      }
> +
> +      if (! (regexes.isEmpty() || matchesAny(fieldName, regexes)) ) {
> +        return false;
> +      }
> +
> +      return true;
> +    }
> +
> +    /**
> +     * returns true if the Object 'o' is an instance of any class in
> +     * the Collection
> +     */
> +    private static boolean instanceOfAny(Object o, Collection<Class> classes) {
> +      for (Class c : classes) {
> +        if ( c.isInstance(o) ) return true;
> +      }
> +      return false;
> +    }
> +
> +    /**
> +     * returns true if the CharSequence 's' matches any Pattern in the
> +     * Collection
> +     */
> +    private static boolean matchesAny(CharSequence s,
> +                                      Collection<Pattern> regexes) {
> +      for (Pattern p : regexes) {
> +        if (p.matcher(s).matches()) return true;
> +      }
> +      return false;
> +    }
> +   }
> +}
> +
>
> Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessorFactory.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessorFactory.java?rev=1242514&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessorFactory.java (added)
> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldMutatingUpdateProcessorFactory.java Thu Feb  9 20:41:21 2012
> @@ -0,0 +1,284 @@
> +/**
> + * 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 java.io.IOException;
> +import java.util.Arrays;
> +import java.util.ArrayList;
> +import java.util.Collection;
> +import java.util.Collections;
> +import java.util.HashSet;
> +import java.util.List;
> +import java.util.Set;
> +import java.util.regex.Pattern;
> +import java.util.regex.PatternSyntaxException;
> +
> +import org.apache.solr.core.SolrCore;
> +import org.apache.solr.common.SolrException;
> +import static org.apache.solr.common.SolrException.ErrorCode.*;
> +import org.apache.solr.common.util.NamedList;
> +import org.apache.solr.request.SolrQueryRequest;
> +import org.apache.solr.response.SolrQueryResponse;
> +import org.apache.solr.update.AddUpdateCommand;
> +import org.apache.solr.schema.IndexSchema;
> +import org.apache.solr.schema.FieldType;
> +import org.apache.solr.util.plugin.SolrCoreAware;
> +
> +
> +/**
> + * Base class for implementing Factories for FieldMutatingUpdateProcessors and
> + * FieldValueMutatingUpdateProcessors.
> + *
> + * <p>
> + * This class provides all of the plumbing for configuring the
> + * FieldNameSelector using the following init params to specify selection
> + * critera...
> + * </p>
> + * <ul>
> + *   <li><code>fieldName</code> - selecting specific fields by field name lookup</li>
> + *   <li><code>fieldRegex</code> - selecting specific fields by field name regex match (regexes are checked in the order specified)</li>
> + *   <li><code>typeName</code> - selecting specific fields by fieldType name lookup</li>
> + *   <li><code>typeClass</code> - selecting specific fields by fieldType class lookup, including inheritence and interfaces</li>
> + * </ul>
> + *
> + * <p>
> + * Each critera can specified as either an &lt;arr&gt; of &lt;str&gt;, or
> + * multiple &lt;str&gt; with the same name.  When multiple criteria of a
> + * single type exist, fields must match <b>at least one</b> to be selected.
> + * If more then one type of critera exist, fields must match
> + * <b>at least one of each</b> to be selected.
> + * </p>
> + * <p>
> + * One or more <code>excludes</code> &lt;lst&gt; params may also be specified,
> + * containing any of the above criteria, identifying fields to be excluded
> + * from seelction even if they match the selection criteria.  As with the main
> + * selection critiera a field must match all of criteria in a single exclusion
> + * in order to be excluded, but multiple exclusions may be specified to get an
> + * <code>OR</code> behavior
> + * </p>
> + *
> + * <p>
> + * In the ExampleFieldMutatingUpdateProcessorFactory configured below,
> + * fields will be mutated if the name starts with "foo" <i>or</i> "bar";
> + * <b>unless</b> the field name contains the substring "SKIP" <i>or</i>
> + * the fieldType is (or subclasses) DateField.  Meaning a field named
> + * "foo_SKIP" is gaurunteed not to be selected, but a field named "bar_smith"
> + * that uses StrField will be selected.
> + * </p>
> + * <pre class="prettyprint">
> + * &lt;processor class="solr.ExampleFieldMutatingUpdateProcessorFactory"&gt;
> + *   &lt;str name="fieldRegex"&gt;foo.*&lt;/str&gt;
> + *   &lt;str name="fieldRegex"&gt;bar.*&lt;/str&gt;
> + *   &lt;!-- each set of exclusions is checked independently --&gt;
> + *   &lt;lst name="exclude"&gt;
> + *     &lt;str name="fieldRegex"&gt;.*SKIP.*&lt;/str&gt;
> + *   &lt;/lst&gt;
> + *   &lt;lst name="exclude"&gt;
> + *     &lt;str name="typeClass"&gt;solr.DateField&lt;/str&gt;
> + *   &lt;/lst&gt;
> + * &lt;/processor&gt;
> + * </pre>
> + *
> + * <p>
> + * Subclasses define the default selection behavior to be applied if no
> + * criteria is configured by the user.  User configured "exclude" criteria
> + * will be applied to the subclass defined default selector.
> + * </p>
> + *
> + * @see FieldMutatingUpdateProcessor
> + * @see FieldValueMutatingUpdateProcessor
> + * @see FieldMutatingUpdateProcessor.FieldNameSelector
> + */
> +public abstract class FieldMutatingUpdateProcessorFactory
> +  extends UpdateRequestProcessorFactory
> +  implements SolrCoreAware {
> +
> +  private static class SelectorParams {
> +    public Set<String> fieldName = Collections.emptySet();
> +    public Set<String> typeName = Collections.emptySet();
> +    public Collection<String> typeClass = Collections.emptyList();
> +    public Collection<Pattern> fieldRegex = Collections.emptyList();
> +  }
> +
> +  private SelectorParams inclusions = new SelectorParams();
> +  private Collection<SelectorParams> exclusions
> +    = new ArrayList<SelectorParams>();
> +
> +  private FieldMutatingUpdateProcessor.FieldNameSelector selector = null;
> +
> +  protected final FieldMutatingUpdateProcessor.FieldNameSelector getSelector() {
> +    if (null != selector) return selector;
> +
> +    throw new SolrException(SERVER_ERROR, "selector was never initialized, "+
> +                            " inform(SolrCore) never called???");
> +  }
> +
> +  @SuppressWarnings("unchecked")
> +  private static final SelectorParams parseSelectorParams(NamedList args) {
> +    SelectorParams params = new SelectorParams();
> +
> +    params.fieldName = new HashSet<String>(oneOrMany(args, "fieldName"));
> +    params.typeName = new HashSet<String>(oneOrMany(args, "typeName"));
> +
> +    // we can compile the patterns now
> +    Collection<String> patterns = oneOrMany(args, "fieldRegex");
> +    if (! patterns.isEmpty()) {
> +      params.fieldRegex = new ArrayList<Pattern>(patterns.size());
> +      for (String s : patterns) {
> +        try {
> +          params.fieldRegex.add(Pattern.compile(s));
> +        } catch (PatternSyntaxException e) {
> +          throw new SolrException
> +            (SERVER_ERROR, "Invalid 'fieldRegex' pattern: " + s, e);
> +        }
> +      }
> +    }
> +
> +    // resolve this into actual Class objects later
> +    params.typeClass = oneOrMany(args, "typeClass");
> +
> +    return params;
> +  }
> +
> +
> +  /**
> +   * Handles common initialization related to source fields for
> +   * constructoring the FieldNameSelector to be used.
> +   *
> +   * Will error if any unexpected init args are found, so subclasses should
> +   * remove any subclass-specific init args before calling this method.
> +   */
> +  @SuppressWarnings("unchecked")
> +  @Override
> +  public void init(NamedList args) {
> +
> +    inclusions = parseSelectorParams(args);
> +
> +    List<Object> excList = args.getAll("exclude");
> +    for (Object excObj : excList) {
> +      if (null == excObj) {
> +        throw new SolrException
> +          (SERVER_ERROR, "'exclude' init param can not be null");
> +      }
> +      if (! (excObj instanceof NamedList) ) {
> +        throw new SolrException
> +          (SERVER_ERROR, "'exclude' init param must be <lst/>");
> +      }
> +      NamedList exc = (NamedList) excObj;
> +      exclusions.add(parseSelectorParams(exc));
> +      if (0 < exc.size()) {
> +        throw new SolrException(SERVER_ERROR,
> +                                "Unexpected 'exclude' init sub-param(s): '" +
> +                                args.getName(0) + "'");
> +      }
> +      // call once per instance
> +      args.remove("exclude");
> +    }
> +    if (0 < args.size()) {
> +      throw new SolrException(SERVER_ERROR,
> +                              "Unexpected init param(s): '" +
> +                              args.getName(0) + "'");
> +    }
> +
> +  }
> +
> +  public void inform(final SolrCore core) {
> +
> +    final IndexSchema schema = core.getSchema();
> +
> +    selector =
> +      FieldMutatingUpdateProcessor.createFieldNameSelector
> +      (core.getResourceLoader(),
> +       core.getSchema(),
> +       inclusions.fieldName,
> +       inclusions.typeName,
> +       inclusions.typeClass,
> +       inclusions.fieldRegex,
> +       getDefaultSelector(core));
> +
> +    for (SelectorParams exc : exclusions) {
> +      selector = FieldMutatingUpdateProcessor.wrap
> +        (selector,
> +         FieldMutatingUpdateProcessor.createFieldNameSelector
> +         (core.getResourceLoader(),
> +          core.getSchema(),
> +          exc.fieldName,
> +          exc.typeName,
> +          exc.typeClass,
> +          exc.fieldRegex,
> +          FieldMutatingUpdateProcessor.SELECT_NO_FIELDS));
> +    }
> +  }
> +
> +  /**
> +   * Defines the default selection behavior when the user has not
> +   * configured any specific criteria for selecting fields. The Default
> +   * implementation matches all fields, and should be overridden by subclasses
> +   * as needed.
> +   *
> +   * @see FieldMutatingUpdateProcessor#SELECT_ALL_FIELDS
> +   */
> +  protected FieldMutatingUpdateProcessor.FieldNameSelector
> +    getDefaultSelector(final SolrCore core) {
> +
> +    return FieldMutatingUpdateProcessor.SELECT_ALL_FIELDS;
> +
> +  }
> +
> +  /**
> +   * Removes all instance of the key from NamedList, returning the Set of
> +   * Strings that key refered to.  Throws an error if the key didn't refer
> +   * to one or more strings (or arrays of strings)
> +   * @exception SolrException invalid arr/str structure.
> +   */
> +  private static Collection<String> oneOrMany(final NamedList args, final String key) {
> +    List<String> result = new ArrayList<String>(args.size() / 2);
> +    final String err = "init arg '" + key + "' must be a string "
> +      + "(ie: 'str'), or an array (ie: 'arr') containing strings; found: ";
> +
> +    for (Object o = args.remove(key); null != o; o = args.remove(key)) {
> +      if (o instanceof String) {
> +        result.add((String)o);
> +        continue;
> +      }
> +
> +      if (o instanceof Object[]) {
> +        o = Arrays.asList((Object[]) o);
> +      }
> +
> +      if (o instanceof Collection) {
> +        for (Object item : (Collection)o) {
> +          if (! (item instanceof String)) {
> +            throw new SolrException(SERVER_ERROR, err + item.getClass());
> +          }
> +          result.add((String)item);
> +        }
> +        continue;
> +      }
> +
> +      // who knows what the hell we have
> +      throw new SolrException(SERVER_ERROR, err + o.getClass());
> +    }
> +
> +    return result;
> +  }
> +
> +}
> +
> +
> +
>
> Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldValueMutatingUpdateProcessor.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldValueMutatingUpdateProcessor.java?rev=1242514&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldValueMutatingUpdateProcessor.java (added)
> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldValueMutatingUpdateProcessor.java Thu Feb  9 20:41:21 2012
> @@ -0,0 +1,81 @@
> +/**
> + * 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.solr.common.SolrInputField;
> +
> +import org.slf4j.Logger;
> +import org.slf4j.LoggerFactory;
> +
> +/**
> + * Abstract subclass of FieldMutatingUpdateProcessor for implementing
> + * UpdateProcessors that will mutate all individual values of a selected
> + * field independently
> + *
> + * @see FieldMutatingUpdateProcessorFactory
> + */
> +public abstract class FieldValueMutatingUpdateProcessor
> +  extends FieldMutatingUpdateProcessor {
> +
> +  private static final Logger log = LoggerFactory.getLogger(FieldValueMutatingUpdateProcessor.class);
> +
> +
> +  public static final Object DELETE_VALUE_SINGLETON = new Object() {
> +      public String toString() {
> +        return "!!Singleton Object Triggering Value Deletion!!";
> +      }
> +    };
> +
> +  public FieldValueMutatingUpdateProcessor(FieldNameSelector selector,
> +                                           UpdateRequestProcessor next) {
> +    super(selector, next);
> +  }
> +
> +  /**
> +   * Mutates individual values of a field as needed, or returns the original
> +   * value.
> +   *
> +   * @param src a value from a matched field which should be mutated
> +   * @return the value to use as a replacement for src, or
> +   *         <code>DELETE_VALUE_SINGLETON</code> to indicate that the value
> +   *         should be removed completely.
> +   * @see #DELETE_VALUE_SINGLETON
> +   */
> +  protected abstract Object mutateValue(final Object src);
> +
> +  protected final SolrInputField mutate(final SolrInputField src) {
> +    SolrInputField result = new SolrInputField(src.getName());
> +    for (final Object srcVal : src.getValues()) {
> +      final Object destVal = mutateValue(srcVal);
> +      if (DELETE_VALUE_SINGLETON == destVal) {
> +        /* NOOP */
> +        log.debug("removing value from field '{}': {}",
> +                  src.getName(), srcVal);
> +      } else {
> +        if (destVal != srcVal) {
> +          log.debug("replace value from field '{}': {} with {}",
> +                    new Object[] { src.getName(), srcVal, destVal });
> +        }
> +        result.addValue(destVal, 1.0F);
> +      }
> +    }
> +    result.setBoost(src.getBoost());
> +    return 0 == result.getValueCount() ? null : result;
> +  }
> +}
> +
>
> Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldValueSubsetUpdateProcessorFactory.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldValueSubsetUpdateProcessorFactory.java?rev=1242514&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldValueSubsetUpdateProcessorFactory.java (added)
> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FieldValueSubsetUpdateProcessorFactory.java Thu Feb  9 20:41:21 2012
> @@ -0,0 +1,59 @@
> +/**
> + * 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.solr.core.SolrCore;
> +
> +import org.apache.solr.common.SolrInputField;
> +import org.apache.solr.request.SolrQueryRequest;
> +import org.apache.solr.response.SolrQueryResponse;
> +
> +import java.util.Collection;
> +
> +/**
> + * Base class for processors that want to mutate selected fields to only
> + * keep a subset of the original values.
> + * @see #pickSubset
> + */
> +public abstract class FieldValueSubsetUpdateProcessorFactory extends FieldMutatingUpdateProcessorFactory {
> +
> +  @Override
> +  public final UpdateRequestProcessor getInstance(SolrQueryRequest req,
> +                                                  SolrQueryResponse rsp,
> +                                                  UpdateRequestProcessor next) {
> +    return new FieldMutatingUpdateProcessor(getSelector(), next) {
> +      protected SolrInputField mutate(final SolrInputField src) {
> +        if (src.getValueCount() <= 1) return src;
> +
> +        SolrInputField result = new SolrInputField(src.getName());
> +        result.setValue(pickSubset(src.getValues()),
> +                        src.getBoost());
> +        return result;
> +      }
> +    };
> +  }
> +
> +  /**
> +   * Method subclasses must override to specify which values should be kept.
> +   * This method will not be called unless the collection contains more then
> +   * one value.
> +   */
> +  protected abstract Collection<Object> pickSubset(Collection<Object> values);
> +
> +}
> +
>
> Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FirstFieldValueUpdateProcessorFactory.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FirstFieldValueUpdateProcessorFactory.java?rev=1242514&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FirstFieldValueUpdateProcessorFactory.java (added)
> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/FirstFieldValueUpdateProcessorFactory.java Thu Feb  9 20:41:21 2012
> @@ -0,0 +1,65 @@
> +/**
> + * 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.solr.core.SolrCore;
> +
> +import java.util.Collections;
> +import java.util.Collection;
> +import java.util.Iterator;
> +
> +/**
> + * Keeps only the first value of fields matching the specified
> + * conditions.  Correct behavior assumes that the SolrInputFields being mutated
> + * are either single valued, or use an ordered Collection (ie: not a Set).
> + * <p>
> + * By default, this processor matches no fields.
> + * </p>
> + *
> + * <p>
> + * For example, in the configuration below, if a field named
> + * <code>primary_author</code> contained multiple values (ie:
> + * <code>"Adam Doe", "Bob Smith", "Carla Jones"</code>) then only the first
> + * value (ie:  <code>"Adam Doe"</code>) will be kept
> + * </p>
> + *
> + * <pre class="prettyprint">
> + * &lt;processor class="solr.FirstFieldValueUpdateProcessorFactory"&gt;
> + *   &lt;str name="fieldName"&gt;primary_author&lt;/str&gt;
> + * &lt;/processor&gt;
> + * </pre>
> + *
> + * @see LastFieldValueUpdateProcessorFactory
> + */
> +public final class FirstFieldValueUpdateProcessorFactory extends FieldValueSubsetUpdateProcessorFactory {
> +
> +  @Override
> +  public Collection<Object> pickSubset(Collection<Object> values) {
> +    // trust the iterator
> +    return Collections.singletonList(values.iterator().next());
> +  }
> +
> +  @Override
> +  public FieldMutatingUpdateProcessor.FieldNameSelector
> +    getDefaultSelector(final SolrCore core) {
> +
> +    return FieldMutatingUpdateProcessor.SELECT_NO_FIELDS;
> +  }
> +
> +}
> +
>
> Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/HTMLStripFieldUpdateProcessorFactory.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/HTMLStripFieldUpdateProcessorFactory.java?rev=1242514&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/HTMLStripFieldUpdateProcessorFactory.java (added)
> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/HTMLStripFieldUpdateProcessorFactory.java Thu Feb  9 20:41:21 2012
> @@ -0,0 +1,88 @@
> +/**
> + * 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.solr.core.SolrCore;
> +import org.apache.solr.request.SolrQueryRequest;
> +import org.apache.solr.response.SolrQueryResponse;
> +
> +import org.apache.lucene.analysis.CharReader;
> +import org.apache.lucene.analysis.charfilter.HTMLStripCharFilter;
> +
> +import org.apache.commons.io.IOUtils;
> +import java.io.IOException;
> +import java.io.Reader;
> +import java.io.StringReader;
> +import java.io.StringWriter;
> +
> +/**
> + * Strips all HTML Markup in any CharSequence values
> + * found in fields matching the specified conditions.
> + * <p>
> + * By default this processor matches no fields
> + * </p>
> + *
> + * <p>For example, with the configuration listed below any documents
> + * containing HTML markup in any field declared in the schema using
> + * <code>StrField</code> will have that HTML striped away.
> + * </p>
> + * <pre class="prettyprint">
> + * &lt;processor class="solr.HTMLStripFieldUpdateProcessorFactory"&gt;
> + *   &lt;str name="typeClass"&gt;solr.StrField&lt;/str&gt;
> + * &lt;/processor&gt;
> + * </pre>
> + */
> +public final class HTMLStripFieldUpdateProcessorFactory extends FieldMutatingUpdateProcessorFactory {
> +
> +  @Override
> +  public FieldMutatingUpdateProcessor.FieldNameSelector
> +    getDefaultSelector(final SolrCore core) {
> +
> +    return FieldMutatingUpdateProcessor.SELECT_NO_FIELDS;
> +
> +  }
> +
> +  @Override
> +  public UpdateRequestProcessor getInstance(SolrQueryRequest req,
> +                                            SolrQueryResponse rsp,
> +                                            UpdateRequestProcessor next) {
> +    return new FieldValueMutatingUpdateProcessor(getSelector(), next) {
> +      protected Object mutateValue(final Object src) {
> +        if (src instanceof CharSequence) {
> +          CharSequence s = (CharSequence)src;
> +          StringWriter result = new StringWriter(s.length());
> +          Reader in = null;
> +          try {
> +            in = new HTMLStripCharFilter
> +              (CharReader.get(new StringReader(s.toString())));
> +            IOUtils.copy(in, result);
> +            return result.toString();
> +          } catch (IOException e) {
> +            // we tried and failed
> +            return s;
> +          } finally {
> +            IOUtils.closeQuietly(in);
> +          }
> +
> +        }
> +        return src;
> +      }
> +    };
> +  }
> +}
> +
>
> Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/LastFieldValueUpdateProcessorFactory.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/LastFieldValueUpdateProcessorFactory.java?rev=1242514&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/LastFieldValueUpdateProcessorFactory.java (added)
> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/LastFieldValueUpdateProcessorFactory.java Thu Feb  9 20:41:21 2012
> @@ -0,0 +1,81 @@
> +/**
> + * 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.solr.core.SolrCore;
> +
> +import java.util.Collections;
> +import java.util.Collection;
> +import java.util.List;
> +import java.util.SortedSet;
> +import java.util.Iterator;
> +
> +/**
> + * Keeps only the last value of fields matching the specified
> + * conditions.  Correct behavior assumes that the SolrInputFields being mutated
> + * are either single valued, or use an ordered Collection (ie: not a Set).
> + * <p>
> + * By default, this processor matches no fields.
> + * </p>
> + *
> + * <p>
> + * For example, in the configuration below, if a field named
> + * <code>primary_author</code> contained multiple values (ie:
> + * <code>"Adam Doe", "Bob Smith", "Carla Jones"</code>) then only the last
> + * value (ie:  <code>"Carla Jones"</code>) will be kept
> + * </p>
> + *
> + * <pre class="prettyprint">
> + * &lt;processor class="solr.LastFieldValueUpdateProcessorFactory"&gt;
> + *   &lt;str name="fieldName"&gt;primary_author&lt;/str&gt;
> + * &lt;/processor&gt;
> + * </pre>
> + *
> + * @see FirstFieldValueUpdateProcessorFactory
> + */
> +public final class LastFieldValueUpdateProcessorFactory extends FieldValueSubsetUpdateProcessorFactory {
> +
> +  @Override
> +  public Collection<Object> pickSubset(Collection<Object> values) {
> +
> +    Object result = null;
> +
> +    if (values instanceof List) {
> +      // optimize index lookup
> +      List l = (List)values;
> +      result = l.get(l.size()-1);
> +    } else if (values instanceof SortedSet) {
> +      // optimize tail lookup
> +      result = ((SortedSet)values).last();
> +    } else {
> +      // trust the iterator
> +      for (Object o : values) { result = o; }
> +    }
> +
> +    return Collections.singletonList(result);
> +  }
> +
> +  @Override
> +  public FieldMutatingUpdateProcessor.FieldNameSelector
> +    getDefaultSelector(final SolrCore core) {
> +
> +    return FieldMutatingUpdateProcessor.SELECT_NO_FIELDS;
> +  }
> +
> +}
> +
>
> Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/MaxFieldValueUpdateProcessorFactory.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/MaxFieldValueUpdateProcessorFactory.java?rev=1242514&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/MaxFieldValueUpdateProcessorFactory.java (added)
> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/MaxFieldValueUpdateProcessorFactory.java Thu Feb  9 20:41:21 2012
> @@ -0,0 +1,75 @@
> +/**
> + * 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.solr.core.SolrCore;
> +
> +import java.util.Collections;
> +import java.util.Collection;
> +import java.util.Iterator;
> +
> +/**
> + * An update processor that keeps only the the maximum value from any selected
> + * fields where multiple values are found.  Correct behavior assumes that all
> + * of the values in the SolrInputFields being mutated are mutually comparable;
> + * If this is not the case, then the full list of all values found will be
> + * used as is.
> + * <p>
> + * By default, this processor matches no fields.
> + * </p>
> + *
> + * <p>
> + * In the example configuration below, if a document contains multiple integer
> + * values (ie: <code>64, 128, 1024</code>) in the field
> + * <code>largestFileSize</code> then only the biggest value
> + * (ie: <code>1024</code>) will be kept in that field.
> + * <p>
> + *
> + * <pre class="prettyprint">
> + *  &lt;processor class="solr.MaxFieldValueUpdateProcessorFactory"&gt;
> + *    &lt;str name="fieldName"&gt;largestFileSize&lt;/str&gt;
> + *  &lt;/processor&gt;
> + * </pre>
> + *
> + * @see MinFieldValueUpdateProcessorFactory
> + * @see Collections#max
> + */
> +public final class MaxFieldValueUpdateProcessorFactory extends FieldValueSubsetUpdateProcessorFactory {
> +
> +  @Override
> +  @SuppressWarnings("unchecked")
> +  public Collection<Object> pickSubset(Collection<Object> values) {
> +    Collection<Object> result = values;
> +    try {
> +      result = Collections.singletonList
> +        (Collections.max((Collection)values));
> +    } catch (ClassCastException e) {
> +      /* NOOP */
> +    }
> +    return result;
> +  }
> +
> +  @Override
> +  public FieldMutatingUpdateProcessor.FieldNameSelector
> +    getDefaultSelector(final SolrCore core) {
> +
> +    return FieldMutatingUpdateProcessor.SELECT_NO_FIELDS;
> +  }
> +
> +}
> +
>
> Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/MinFieldValueUpdateProcessorFactory.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/MinFieldValueUpdateProcessorFactory.java?rev=1242514&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/MinFieldValueUpdateProcessorFactory.java (added)
> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/MinFieldValueUpdateProcessorFactory.java Thu Feb  9 20:41:21 2012
> @@ -0,0 +1,75 @@
> +/**
> + * 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.solr.core.SolrCore;
> +
> +import java.util.Collections;
> +import java.util.Collection;
> +import java.util.Iterator;
> +
> +/**
> + * An update processor that keeps only the the minimum value from any selected
> + * fields where multiple values are found.  Correct behavior assumes that all
> + * of the values in the SolrInputFields being mutated are mutually comparable;
> + * If this is not the case, then the full list of all values found will be
> + * used as is.
> + * <p>
> + * By default, this processor matches no fields.
> + * </p>
> + *
> + * <p>
> + * In the example configuration below, if a document contains multiple integer
> + * values (ie: <code>64, 128, 1024</code>) in the field
> + * <code>smallestFileSize</code> then only the smallest value
> + * (ie: <code>64</code>) will be kept in that field.
> + * <p>
> + *
> + * <pre class="prettyprint">
> + *  &lt;processor class="solr.MinFieldValueUpdateProcessorFactory"&gt;
> + *    &lt;str name="fieldName"&gt;smallestFileSize&lt;/str&gt;
> + *  &lt;/processor&gt;
> + * </pre>
> + *
> + * @see MaxFieldValueUpdateProcessorFactory
> + * @see Collections#min
> + */
> +public final class MinFieldValueUpdateProcessorFactory extends FieldValueSubsetUpdateProcessorFactory {
> +
> +  @Override
> +  @SuppressWarnings("unchecked")
> +  public Collection<Object> pickSubset(Collection<Object> values) {
> +    Collection<Object> result = values;
> +    try {
> +      result = Collections.singletonList
> +        (Collections.min((Collection)values));
> +    } catch (ClassCastException e) {
> +      /* NOOP */
> +    }
> +    return result;
> +  }
> +
> +  @Override
> +  public FieldMutatingUpdateProcessor.FieldNameSelector
> +    getDefaultSelector(final SolrCore core) {
> +
> +    return FieldMutatingUpdateProcessor.SELECT_NO_FIELDS;
> +  }
> +
> +}
> +
>
> Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/RegexReplaceProcessorFactory.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/RegexReplaceProcessorFactory.java?rev=1242514&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/RegexReplaceProcessorFactory.java (added)
> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/RegexReplaceProcessorFactory.java Thu Feb  9 20:41:21 2012
> @@ -0,0 +1,121 @@
> +/**
> + * 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.solr.core.SolrCore;
> +
> +import org.apache.solr.common.SolrException;
> +import org.apache.solr.common.SolrException.ErrorCode;
> +import org.apache.solr.common.util.NamedList;
> +
> +import org.apache.solr.request.SolrQueryRequest;
> +import org.apache.solr.response.SolrQueryResponse;
> +
> +import java.util.regex.PatternSyntaxException;
> +import java.util.regex.Pattern;
> +import java.util.regex.Matcher;
> +
> +import org.slf4j.Logger;
> +import org.slf4j.LoggerFactory;
> +
> +/**
> + * An updated processor that applies a configured regex to any
> + * CharSequence values found in the selected fields, and replaces
> + * any matches with the configured replacement string
> + * <p>
> + * By default this processor applies itself to no fields.
> + * </p>
> + *
> + * <p>
> + * For example, with the configuration listed below, any sequence of multiple
> + * whitespace characters found in values for field named <code>title</code>
> + * or <code>content</code> will be replaced by a single space character.
> + * </p>
> + *
> + * <pre class="prettyprint">
> + * &lt;processor class="solr.RegexReplaceProcessorFactory"&gt;
> + *   &lt;str name="fieldName"&gt;content&lt;/str&gt;
> + *   &lt;str name="fieldName"&gt;title&lt;/str&gt;
> + *   &lt;str name="pattern"&gt;\s+&lt;/str&gt;
> + *   &lt;str name="replacement"&gt; &lt;/str&gt;
> + * &lt;/processor&gt;
> + * </pre>
> + *
> + * @see java.util.regex.Pattern
> + */
> +public final class RegexReplaceProcessorFactory extends FieldMutatingUpdateProcessorFactory {
> +
> +  private static final Logger log = LoggerFactory.getLogger(RegexReplaceProcessorFactory.class);
> +
> +  private static final String REPLACEMENT_PARAM = "replacement";
> +  private static final String PATTERN_PARAM = "pattern";
> +
> +  private Pattern pattern;
> +  private String replacement;
> +
> +  @SuppressWarnings("unchecked")
> +  @Override
> +  public void init(NamedList args) {
> +
> +    String patternParam = args.remove(PATTERN_PARAM).toString();
> +
> +    if(patternParam == null) {
> +      throw new SolrException(ErrorCode.SERVER_ERROR,
> +                              "Missing required init parameter: " + PATTERN_PARAM);
> +    }
> +    try {
> +      pattern = Pattern.compile(patternParam);
> +    } catch (PatternSyntaxException e) {
> +      throw new SolrException(ErrorCode.SERVER_ERROR,
> +                              "Invalid regex: " + patternParam, e);
> +    }
> +
> +    String replacementParam = args.remove(REPLACEMENT_PARAM).toString();
> +    if(replacementParam == null) {
> +      throw new SolrException(ErrorCode.SERVER_ERROR,
> +                              "Missing required init parameter: " + REPLACEMENT_PARAM);
> +    }
> +    replacement = Matcher.quoteReplacement(replacementParam);
> +
> +    super.init(args);
> +  }
> +
> +  /**
> +   * @see FieldMutatingUpdateProcessor#SELECT_NO_FIELDS
> +   */
> +  protected FieldMutatingUpdateProcessor.FieldNameSelector
> +    getDefaultSelector(final SolrCore core) {
> +
> +    return FieldMutatingUpdateProcessor.SELECT_NO_FIELDS;
> +
> +  }
> +
> +  @Override
> +  public UpdateRequestProcessor getInstance(SolrQueryRequest request,
> +                                            SolrQueryResponse response,
> +                                            UpdateRequestProcessor next) {
> +    return new FieldValueMutatingUpdateProcessor(getSelector(), next) {
> +      protected Object mutateValue(final Object src) {
> +        if (src instanceof CharSequence) {
> +          CharSequence txt = (CharSequence)src;
> +          return pattern.matcher(txt).replaceAll(replacement);
> +        }
> +        return src;
> +      }
> +    };
> +  }
> +}
>
> Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/RemoveBlankFieldUpdateProcessorFactory.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/RemoveBlankFieldUpdateProcessorFactory.java?rev=1242514&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/RemoveBlankFieldUpdateProcessorFactory.java (added)
> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/RemoveBlankFieldUpdateProcessorFactory.java Thu Feb  9 20:41:21 2012
> @@ -0,0 +1,70 @@
> +/**
> + * 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.solr.common.util.NamedList;
> +import org.apache.solr.request.SolrQueryRequest;
> +import org.apache.solr.response.SolrQueryResponse;
> +
> +/**
> + * Removes any values found which are CharSequence with a length of 0.
> + * (ie: empty strings)
> + * <p>
> + * By default this processor applies itself to all fields.
> + * </p>
> + *
> + * <p>
> + * For example, with the configuration listed below, blank strings will be
> + * removed from all fields except those whose name ends with
> + * "<code>_literal</code>".
> + * </p>
> + *
> + * <pre class="prettyprint">
> + * &lt;processor class="solr.RemoveBlankFieldUpdateProcessorFactory"&gt;
> + *   &lt;lst name="exclude"&gt;
> + *     &lt;str name="fieldRegex"&gt;.*_literal&lt;/str&gt;
> + *   &lt;/lst&gt;
> + * &lt;/processor&gt;
> + * </pre>
> + *
> + */
> +public final class RemoveBlankFieldUpdateProcessorFactory extends FieldMutatingUpdateProcessorFactory {
> +
> +  @SuppressWarnings("unchecked")
> +  @Override
> +  public void init(NamedList args) {
> +    // no trim specific init args
> +    super.init(args);
> +  }
> +
> +  @Override
> +  public UpdateRequestProcessor getInstance(SolrQueryRequest req,
> +                                            SolrQueryResponse rsp,
> +                                            UpdateRequestProcessor next) {
> +    return new FieldValueMutatingUpdateProcessor(getSelector(), next) {
> +      protected Object mutateValue(final Object src) {
> +        if (src instanceof CharSequence
> +            && 0 == ((CharSequence)src).length()) {
> +          return DELETE_VALUE_SINGLETON;
> +        }
> +        return src;
> +      }
> +    };
> +  }
> +}
> +
>
> Added: lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/TrimFieldUpdateProcessorFactory.java
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/TrimFieldUpdateProcessorFactory.java?rev=1242514&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/TrimFieldUpdateProcessorFactory.java (added)
> +++ lucene/dev/trunk/solr/core/src/java/org/apache/solr/update/processor/TrimFieldUpdateProcessorFactory.java Thu Feb  9 20:41:21 2012
> @@ -0,0 +1,68 @@
> +/**
> + * 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.solr.common.util.NamedList;
> +import org.apache.solr.request.SolrQueryRequest;
> +import org.apache.solr.response.SolrQueryResponse;
> +
> +
> +/**
> + * Trims leading and trailing whitespace from any CharSequence values
> + * found in fields matching the specified conditions and returns the
> + * resulting String.
> + * <p>
> + * By default this processor matches all fields
> + * </p>
> + *
> + * <p>For example, with the configuration listed all String field values
> + * will have leading and trailing spaces removed except for fields whose
> + * named ends with "<code>_literal</code>".
> + * </p>
> + * <pre class="prettyprint">
> + * &lt;processor class="solr.TrimFieldUpdateProcessorFactory"&gt;
> + *   &lt;lst name="exclude"&gt;
> + *     &lt;str name="fieldRegex"&gt;.*_literal&lt;/str&gt;
> + *   &lt;/lst&gt;
> + * &lt;/processor&gt;
> + * </pre>
> + */
> +public final class TrimFieldUpdateProcessorFactory extends FieldMutatingUpdateProcessorFactory {
> +
> +  @SuppressWarnings("unchecked")
> +  @Override
> +  public void init(NamedList args) {
> +    // no trim specific init args
> +    super.init(args);
> +  }
> +
> +  @Override
> +  public UpdateRequestProcessor getInstance(SolrQueryRequest req,
> +                                            SolrQueryResponse rsp,
> +                                            UpdateRequestProcessor next) {
> +    return new FieldValueMutatingUpdateProcessor(getSelector(), next) {
> +      protected Object mutateValue(final Object src) {
> +        if (src instanceof CharSequence) {
> +          return ((CharSequence)src).toString().trim();
> +        }
> +        return src;
> +      }
> +    };
> +  }
> +}
> +
>
> Added: lucene/dev/trunk/solr/core/src/test-files/solr/conf/solrconfig-update-processor-chains.xml
> URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/core/src/test-files/solr/conf/solrconfig-update-processor-chains.xml?rev=1242514&view=auto
> ==============================================================================
> --- lucene/dev/trunk/solr/core/src/test-files/solr/conf/solrconfig-update-processor-chains.xml (added)
> +++ lucene/dev/trunk/solr/core/src/test-files/solr/conf/solrconfig-update-processor-chains.xml Thu Feb  9 20:41:21 2012
> @@ -0,0 +1,235 @@
> +<?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 enumerates many different update processor chain
> +   configurations.
> +
> +  -->
> +<config>
> +  <luceneMatchVersion>${tests.luceneMatchVersion:LUCENE_CURRENT}</luceneMatchVersion>
> +  <requestHandler name="standard" class="solr.StandardRequestHandler"></requestHandler>
> +
> +  <updateRequestProcessorChain name="comprehensive">
> +    <processor class="solr.FieldLengthUpdateProcessorFactory">
> +      <arr name="typeClass">
> +        <str>solr.TrieIntField</str>
> +        <str>solr.TrieLongField</str>
> +      </arr>
> +    </processor>
> +    <processor class="solr.MinFieldValueUpdateProcessorFactory">
> +      <str name="fieldName">min_foo_l</str>
> +    </processor>
> +    <processor class="solr.MaxFieldValueUpdateProcessorFactory">
> +      <str name="fieldName">max_foo_l</str>
> +    </processor>
> +    <processor class="solr.ConcatFieldUpdateProcessorFactory">
> +      <str name="delimiter">; </str>
> +      <lst name="exclude">
> +        <str name="fieldName">primary_author_s1</str>
> +      </lst>
> +    </processor>
> +    <processor class="solr.FirstFieldValueUpdateProcessorFactory">
> +      <str name="fieldName">primary_author_s1</str>
> +      <str name="fieldName">first_foo_l</str>
> +    </processor>
> +    <processor class="solr.RunUpdateProcessorFactory" />
> +  </updateRequestProcessorChain>
> +
> +
> +  <updateRequestProcessorChain name="trim-all">
> +    <!-- no specific src field configs, so all fields should get trimmed -->
> +    <processor class="solr.TrimFieldUpdateProcessorFactory">
> +    </processor>
> +  </updateRequestProcessorChain>
> +
> +  <updateRequestProcessorChain name="trim-field">
> +    <processor class="solr.TrimFieldUpdateProcessorFactory">
> +      <str name="fieldName">foo_t</str>
> +    </processor>
> +  </updateRequestProcessorChain>
> +
> +
> +  <updateRequestProcessorChain name="trim-most">
> +    <!-- all fields except the exclusions should be trimmed -->
> +    <processor class="solr.TrimFieldUpdateProcessorFactory">
> +      <lst name="exclude">
> +        <str name="fieldName">foo_t</str>
> +      </lst>
> +    </processor>
> +  </updateRequestProcessorChain>
> +  <updateRequestProcessorChain name="trim-many">
> +    <processor class="solr.TrimFieldUpdateProcessorFactory">
> +      <str name="fieldRegex">foo.*</str>
> +      <str name="fieldRegex">bar.*</str>
> +      <lst name="exclude">
> +        <str name="fieldRegex">.*HOSS.*</str>
> +      </lst>
> +    </processor>
> +  </updateRequestProcessorChain>
> +  <updateRequestProcessorChain name="trim-few">
> +    <processor class="solr.TrimFieldUpdateProcessorFactory">
> +      <str name="fieldRegex">foo.*</str>
> +      <str name="fieldRegex">bar.*</str>
> +      <!-- each set of exclusions is checked independently -->
> +      <lst name="exclude">
> +        <str name="typeClass">solr.DateField</str>
> +      </lst>
> +      <lst name="exclude">
> +        <str name="fieldRegex">.*HOSS.*</str>
> +      </lst>
> +    </processor>
> +  </updateRequestProcessorChain>
> +  <updateRequestProcessorChain name="trim-some">
> +    <processor class="solr.TrimFieldUpdateProcessorFactory">
> +      <str name="fieldRegex">foo.*</str>
> +      <str name="fieldRegex">bar.*</str>
> +      <!-- only excluded if it matches all in set -->
> +      <lst name="exclude">
> +        <str name="typeClass">solr.DateField</str>
> +        <str name="fieldRegex">.*HOSS.*</str>
> +      </lst>
> +    </processor>
> +  </updateRequestProcessorChain>
> +
> +  <updateRequestProcessorChain name="trim-fields">
> +    <processor class="solr.TrimFieldUpdateProcessorFactory">
> +      <str name="fieldName">name</str>
> +      <str name="fieldName">foo_t</str>
> +    </processor>
> +  </updateRequestProcessorChain>
> +  <updateRequestProcessorChain name="trim-fields-arr">
> +    <processor class="solr.TrimFieldUpdateProcessorFactory">
> +      <arr name="fieldName">
> +        <str>name</str>
> +        <str>foo_t</str>
> +      </arr>
> +    </processor>
> +  </updateRequestProcessorChain>
> +
> +  <updateRequestProcessorChain name="trim-field-regexes">
> +    <processor class="solr.TrimFieldUpdateProcessorFactory">
> +      <str name="fieldRegex">foo.*</str>
> +      <str name="fieldRegex">bar.*_s</str>
> +    </processor>
> +  </updateRequestProcessorChain>
> +
> +  <updateRequestProcessorChain name="trim-types">
> +    <processor class="solr.TrimFieldUpdateProcessorFactory">
> +      <str name="typeName">nametext</str>
> +      <str name="typeName">text_sw</str>
> +    </processor>
> +  </updateRequestProcessorChain>
> +
> +  <updateRequestProcessorChain name="trim-classes">
> +    <processor class="solr.TrimFieldUpdateProcessorFactory">
> +      <str name="typeClass">solr.DateField</str>
> +      <str name="typeClass">solr.StrField</str>
> +    </processor>
> +  </updateRequestProcessorChain>
> +
> +  <updateRequestProcessorChain name="trim-multi">
> +    <processor class="solr.TrimFieldUpdateProcessorFactory">
> +      <str name="typeClass">solr.DateField</str>
> +      <str name="typeClass">solr.StrField</str>
> +      <arr name="fieldRegex">
> +        <str>foo.*</str>
> +      </arr>
> +    </processor>
> +  </updateRequestProcessorChain>
> +
> +  <updateRequestProcessorChain name="remove-all-blanks">
> +    <processor class="solr.RemoveBlankFieldUpdateProcessorFactory">
> +    </processor>
> +  </updateRequestProcessorChain>
> +
> +  <updateRequestProcessorChain name="length-none">
> +    <processor class="solr.FieldLengthUpdateProcessorFactory">
> +      <!-- by default, the processor doesn't touch anything -->
> +    </processor>
> +  </updateRequestProcessorChain>
> +  <updateRequestProcessorChain name="length-some">
> +    <processor class="solr.FieldLengthUpdateProcessorFactory">
> +      <arr name="fieldRegex">
> +        <str>foo.*</str>
> +        <str>yak.*</str>
> +      </arr>
> +    </processor>
> +  </updateRequestProcessorChain>
> +
> +  <updateRequestProcessorChain name="concat-defaults">
> +    <processor class="solr.ConcatFieldUpdateProcessorFactory">
> +    </processor>
> +  </updateRequestProcessorChain>
> +  <updateRequestProcessorChain name="concat-field">
> +    <processor class="solr.ConcatFieldUpdateProcessorFactory">
> +      <str name="fieldName">foo_s</str>
> +    </processor>
> +  </updateRequestProcessorChain>
> +  <updateRequestProcessorChain name="concat-type-delim">
> +    <processor class="solr.ConcatFieldUpdateProcessorFactory">
> +      <str name="typeName">string</str>
> +      <str name="delimiter">; </str>
> +    </processor>
> +  </updateRequestProcessorChain>
> +
> +  <updateRequestProcessorChain name="first-value">
> +    <processor class="solr.FirstFieldValueUpdateProcessorFactory">
> +      <str name="fieldName">foo_s</str>
> +      <str name="fieldName">bar_s</str>
> +    </processor>
> +  </updateRequestProcessorChain>
> +  <updateRequestProcessorChain name="last-value">
> +    <processor class="solr.LastFieldValueUpdateProcessorFactory">
> +      <str name="fieldName">foo_s</str>
> +      <str name="fieldName">bar_s</str>
> +    </processor>
> +  </updateRequestProcessorChain>
> +  <updateRequestProcessorChain name="min-value">
> +    <processor class="solr.MinFieldValueUpdateProcessorFactory">
> +      <str name="fieldName">foo_i</str>
> +      <str name="fieldName">foo_s</str>
> +      <str name="fieldName">bar_s</str>
> +    </processor>
> +  </updateRequestProcessorChain>
> +  <updateRequestProcessorChain name="max-value">
> +    <processor class="solr.MaxFieldValueUpdateProcessorFactory">
> +      <str name="fieldName">foo_i</str>
> +      <str name="fieldName">foo_s</str>
> +      <str name="fieldName">bar_s</str>
> +    </processor>
> +  </updateRequestProcessorChain>
> +
> +  <updateRequestProcessorChain name="html-strip">
> +    <processor class="solr.HTMLStripFieldUpdateProcessorFactory">
> +      <str name="fieldName">html_s</str>
> +    </processor>
> +    <processor class="solr.TrimFieldUpdateProcessorFactory"/>
> +  </updateRequestProcessorChain>
> +
> +  <updateRequestProcessorChain name="regex-replace">
> +    <processor class="solr.RegexReplaceProcessorFactory">
> +      <str name="fieldName">content</str>
> +      <str name="fieldName">title</str>
> +      <str name="pattern">\s+</str>
> +      <str name="replacement">X</str>
> +    </processor>
> +  </updateRequestProcessorChain>
> +
> +</config>
>
>



-- 
lucidimagination.com

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


Re: svn commit: r1242514 [1/2] - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/update/processor/ core/src/test-files/solr/conf/ core/src/test/org/apache/solr/update/processor/

Posted by Chris Hostetter <ho...@fucit.org>.
: > really???? what is the compile error?
: 
: Type mismatch: cannot convert from List<Comparable> to Collection<Object>

FYI: mruir and i syned up on IRC, this is fixed in r1242534.

-Hoss

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


Re: svn commit: r1242514 [1/2] - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/update/processor/ core/src/test-files/solr/conf/ core/src/test/org/apache/solr/update/processor/

Posted by Robert Muir <rc...@gmail.com>.
On Thu, Feb 9, 2012 at 3:57 PM, Chris Hostetter
<ho...@fucit.org> wrote:
>
> : Can we improve this? Both Min and MaxFieldValueUpdateProcessorFactory
> : show up as a compile error in eclipse, which is frustrating to people
> : who use those IDEs.
>
> really???? what is the compile error?

Type mismatch: cannot convert from List<Comparable> to Collection<Object>

>
> ClassCastException is what Collections.max/min throw if you give them a
> Collection that is not mutually comparable, per the docs of this class, if
> you give it a Collection that is not mutually comparable, it ruturns the
> original Collection w/o selecting a subset.
>

OK, but can we please do this in a cleaner way? I'm sorry, I have a
hard time being angry at eclipse's compiler about this!

-- 
lucidimagination.com

---------------------------------------------------------------------
To unsubscribe, e-mail: dev-unsubscribe@lucene.apache.org
For additional commands, e-mail: dev-help@lucene.apache.org


Re: svn commit: r1242514 [1/2] - in /lucene/dev/trunk/solr: ./ core/src/java/org/apache/solr/update/processor/ core/src/test-files/solr/conf/ core/src/test/org/apache/solr/update/processor/

Posted by Chris Hostetter <ho...@fucit.org>.
: Can we improve this? Both Min and MaxFieldValueUpdateProcessorFactory
: show up as a compile error in eclipse, which is frustrating to people
: who use those IDEs.

really???? what is the compile error?

: While it could be a bug in the eclipse compiler, this code is
: definitely on shaky ground, I don't understand how a
: ClassCastException is OK?!

ClassCastException is what Collections.max/min throw if you give them a 
Collection that is not mutually comparable, per the docs of this class, if 
you give it a Collection that is not mutually comparable, it ruturns the 
original Collection w/o selecting a subset.

: > + * An update processor that keeps only the the maximum value from any selected
: > + * fields where multiple values are found.  Correct behavior assumes that all
: > + * of the values in the SolrInputFields being mutated are mutually comparable;
: > + * If this is not the case, then the full list of all values found will be
: > + * used as is.

	...

: > +    Collection<Object> result = values;
: > +    try {
: > +      result = Collections.singletonList
: > +        (Collections.max((Collection)values));
: > +    } catch (ClassCastException e) {
: > +      /* NOOP */
: > +    }
: > +    return result;


-Hoss