You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ro...@apache.org on 2012/11/23 13:01:26 UTC

svn commit: r1412849 [10/13] - in /lucene/dev/branches/LUCENE-2878: ./ dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/ dev-tools/idea/solr/contrib/dataimporthandler/ dev-tools/maven/ dev-tools/maven/solr/contrib/dataimporthandler/ dev-too...

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SimplePropertiesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SimplePropertiesWriter.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SimplePropertiesWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SimplePropertiesWriter.java Fri Nov 23 12:00:32 2012
@@ -1,4 +1,5 @@
 package org.apache.solr.handler.dataimport;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -16,108 +17,213 @@ package org.apache.solr.handler.dataimpo
  * limitations under the License.
  */
 
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
+
 import java.io.File;
 import java.io.FileInputStream;
-import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
 import java.io.InputStream;
 import java.io.OutputStream;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.Locale;
+import java.util.Map;
 import java.util.Properties;
 
 import org.apache.solr.core.SolrCore;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
-
-public class SimplePropertiesWriter implements DIHPropertiesWriter {
-  private static final Logger log = LoggerFactory.getLogger(SimplePropertiesWriter.class);
-
-  static final String IMPORTER_PROPERTIES = "dataimport.properties";
-
+/**
+ * <p>
+ *  Writes properties using {@link Properties#store} .
+ *  The special property "last_index_time" is converted to a formatted date.
+ *  Users can configure the location, filename, locale and date format to use.
+ * </p> 
+ */
+public class SimplePropertiesWriter extends DIHProperties {
+  private static final Logger log = LoggerFactory
+      .getLogger(SimplePropertiesWriter.class);
+  
   static final String LAST_INDEX_KEY = "last_index_time";
-
-  private String persistFilename = IMPORTER_PROPERTIES;
-
-  private String configDir = null;
-
-
-
-    public void init(DataImporter dataImporter) {
+  
+  protected String filename = null;
+  
+  protected String configDir = null;
+  
+  protected Locale locale = null;
+  
+  protected SimpleDateFormat dateFormat = null;
+  
+  /**
+   * The locale to use when writing the properties file.  Default is {@link Locale#ROOT}
+   */
+  public static final String LOCALE = "locale";
+  /**
+   * The date format to use when writing values for "last_index_time" to the properties file.
+   * See {@link SimpleDateFormat} for patterns.  Default is yyyy-MM-dd HH:mm:ss .
+   */
+  public static final String DATE_FORMAT = "dateFormat";
+  /**
+   * The directory to save the properties file in. Default is the current core's "config" directory.
+   */
+  public static final String DIRECTORY = "directory";
+  /**
+   * The filename to save the properties file to.  Default is this Handler's name from solrconfig.xml.
+   */
+  public static final String FILENAME = "filename";
+  
+  @Override
+  public void init(DataImporter dataImporter, Map<String, String> params) {
+    if(params.get(FILENAME) != null) {
+      filename = params.get(FILENAME);
+    } else if(dataImporter.getHandlerName()!=null) {
+      filename = dataImporter.getHandlerName() +  ".properties";
+    } else {
+      filename = "dataimport.properties";
+    }
+    if(params.get(DIRECTORY) != null) {
+      configDir = params.get(DIRECTORY);
+    } else {
       SolrCore core = dataImporter.getCore();
-      String configDir = core ==null ? ".": core.getResourceLoader().getConfigDir();
-      String persistFileName = dataImporter.getHandlerName();
-
-      this.configDir = configDir;
-    if(persistFileName != null){
-        persistFilename = persistFileName + ".properties";
-      }
+      configDir = (core == null ? "." : core.getResourceLoader().getConfigDir());
     }
-
-
-
-
+    if(params.get(LOCALE) != null) {
+      String localeStr = params.get(LOCALE);
+      for (Locale l : Locale.getAvailableLocales()) {
+        if(localeStr.equals(l.getDisplayName())) {
+          locale = l;
+          break;
+        }
+      }
+      if(locale==null) {
+        throw new DataImportHandlerException(SEVERE, "Unsupported locale for PropertWriter: " + localeStr);
+      }
+    } else {
+      locale = Locale.ROOT;
+    }    
+    if(params.get(DATE_FORMAT) != null) {
+      dateFormat = new SimpleDateFormat(params.get(DATE_FORMAT), locale);
+    } else {
+      dateFormat = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", locale);
+    }    
+  }
+  
   private File getPersistFile() {
     String filePath = configDir;
-    if (configDir != null && !configDir.endsWith(File.separator))
-      filePath += File.separator;
-    filePath += persistFilename;
+    if (configDir != null && !configDir.endsWith(File.separator)) filePath += File.separator;
+    filePath += filename;
     return new File(filePath);
   }
-
-    public boolean isWritable() {
-        File persistFile =  getPersistFile();
-        return persistFile.exists() ? persistFile.canWrite() : persistFile.getParentFile().canWrite();
-
+  @Override
+  public boolean isWritable() {
+    File persistFile = getPersistFile();
+    return persistFile.exists() ? persistFile.canWrite() : persistFile
+        .getParentFile().canWrite();
+    
+  }
+  
+  protected String convertDateToString(Date d) {
+    return dateFormat.format(d);
+  }
+  protected Date convertStringToDate(String s) {
+    try {
+      return dateFormat.parse(s);
+    } catch (ParseException e) {
+      throw new DataImportHandlerException(SEVERE, "Value for "
+          + LAST_INDEX_KEY + " is invalid for date format "
+          + dateFormat.toLocalizedPattern() + " : " + s);
     }
-
-    @Override
-  public void persist(Properties p) {
-    OutputStream propOutput = null;
-
-    Properties props = readIndexerProperties();
-
+  }
+  /**
+   * {@link DocBuilder} sends the date as an Object because 
+   * this class knows how to convert it to a String
+   */
+  protected Properties mapToProperties(Map<String,Object> propObjs) {
+    Properties p = new Properties();
+    for(Map.Entry<String,Object> entry : propObjs.entrySet()) {
+      String key = entry.getKey();
+      String val = null;
+      String lastKeyPart = key;
+      int lastDotPos = key.lastIndexOf('.');
+      if(lastDotPos!=-1 && key.length() > lastDotPos+1) {
+        lastKeyPart = key.substring(lastDotPos + 1);
+      }
+      if(LAST_INDEX_KEY.equals(lastKeyPart) && entry.getValue() instanceof Date) {
+        val = convertDateToString((Date) entry.getValue());
+      } else {
+        val = entry.getValue().toString();
+      }
+      p.put(key, val);
+    }
+    return p;
+  }
+  /**
+   * We'll send everything back as Strings as this class has
+   * already converted them.
+   */
+  protected Map<String,Object> propertiesToMap(Properties p) {
+    Map<String,Object> theMap = new HashMap<String,Object>();
+    for(Map.Entry<Object,Object> entry : p.entrySet()) {
+      String key = entry.getKey().toString();
+      Object val = entry.getValue().toString();
+      theMap.put(key, val);
+    }
+    return theMap;
+  }
+  
+  @Override
+  public void persist(Map<String, Object> propObjs) {
+    OutputStream propOutput = null;    
+    Properties existingProps = mapToProperties(readIndexerProperties());    
+    Properties newProps = mapToProperties(propObjs);
     try {
-      props.putAll(p);
+      existingProps.putAll(newProps);
       String filePath = configDir;
-      if (configDir != null && !configDir.endsWith(File.separator))
+      if (configDir != null && !configDir.endsWith(File.separator)) {
         filePath += File.separator;
-      filePath += persistFilename;
+      }
+      filePath += filename;
       propOutput = new FileOutputStream(filePath);
-      props.store(propOutput, null);
-      log.info("Wrote last indexed time to " + persistFilename);
+      existingProps.store(propOutput, null);
+      log.info("Wrote last indexed time to " + filename);
     } catch (Exception e) {
-      throw new DataImportHandlerException(DataImportHandlerException.SEVERE, "Unable to persist Index Start Time", e);
+      throw new DataImportHandlerException(DataImportHandlerException.SEVERE,
+          "Unable to persist Index Start Time", e);
     } finally {
       try {
-        if (propOutput != null)
-          propOutput.close();
+        if (propOutput != null) propOutput.close();
       } catch (IOException e) {
         propOutput = null;
       }
     }
   }
-
+  
   @Override
-  public Properties readIndexerProperties() {
+  public Map<String, Object> readIndexerProperties() {
     Properties props = new Properties();
-    InputStream propInput = null;
-
+    InputStream propInput = null;    
     try {
-      propInput = new FileInputStream(configDir + persistFilename);
+      String filePath = configDir;
+      if (configDir != null && !configDir.endsWith(File.separator)) {
+        filePath += File.separator;
+      }
+      filePath += filename;
+      propInput = new FileInputStream(filePath);
       props.load(propInput);
-      log.info("Read " + persistFilename);
+      log.info("Read " + filename);
     } catch (Exception e) {
-      log.warn("Unable to read: " + persistFilename);
+      log.warn("Unable to read: " + filename);
     } finally {
       try {
-        if (propInput != null)
-          propInput.close();
+        if (propInput != null) propInput.close();
       } catch (IOException e) {
         propInput = null;
       }
-    }
-
-    return props;
+    }    
+    return propertiesToMap(props);
   }
-
+  
 }

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/TemplateTransformer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/TemplateTransformer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/TemplateTransformer.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/TemplateTransformer.java Fri Nov 23 12:00:32 2012
@@ -55,7 +55,7 @@ public class TemplateTransformer extends
   @SuppressWarnings("unchecked")
   public Object transformRow(Map<String, Object> row, Context context) {
 
-    VariableResolverImpl resolver = (VariableResolverImpl) context
+    VariableResolver resolver = (VariableResolver) context
             .getVariableResolver();
     // Add current row to the copy of resolver map
 //    for (Map.Entry<String, Object> entry : row.entrySet())
@@ -69,7 +69,11 @@ public class TemplateTransformer extends
 
       // Verify if all variables can be resolved or not
       boolean resolvable = true;
-      List<String> variables = getVars(expr);
+      List<String> variables = this.templateVsVars.get(expr);
+      if(variables == null){
+        variables = resolver.getVariables(expr);
+        this.templateVsVars.put(expr, variables);
+      }
       for (String v : variables) {
         if (resolver.resolve(v) == null) {
           LOG.warn("Unable to resolve variable: " + v
@@ -91,15 +95,5 @@ public class TemplateTransformer extends
 
     return row;
   }
-
-  private List<String> getVars(String expr) {
-    List<String> result = this.templateVsVars.get(expr);
-    if(result == null){
-      result = TemplateString.getVariables(expr);
-      this.templateVsVars.put(expr, result);
-    }
-    return result;
-  }
-
   public static final String TEMPLATE = "template";
 }

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/VariableResolver.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/VariableResolver.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/VariableResolver.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/VariableResolver.java Fri Nov 23 12:00:32 2012
@@ -16,37 +16,205 @@
  */
 package org.apache.solr.handler.dataimport;
 
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.WeakHashMap;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
 /**
  * <p>
- * This class is more or less like a Map. But has more intelligence to resolve
- * namespaces. Namespaces are delimited with '.' (period)
+ * A set of nested maps that can resolve variables by namespaces. Variables are
+ * enclosed with a dollar sign then an opening curly brace, ending with a
+ * closing curly brace. Namespaces are delimited with '.' (period).
+ * </p>
+ * <p>
+ * This class also has special logic to resolve evaluator calls by recognizing
+ * the reserved function namespace: dataimporter.functions.xxx
  * </p>
- * <p/>
  * <p>
- * Refer to <a
- * href="http://wiki.apache.org/solr/DataImportHandler">http://wiki.apache.org/solr/DataImportHandler</a>
- * for more details.
+ * This class caches strings that have already been resolved from the current
+ * dih import.
  * </p>
- * <p/>
  * <b>This API is experimental and may change in the future.</b>
- *
- *
+ * 
+ * 
  * @since solr 1.3
  */
-public abstract class VariableResolver {
-
+public class VariableResolver {
+  
+  private static final Pattern DOT_PATTERN = Pattern.compile("[.]");
+  private static final Pattern PLACEHOLDER_PATTERN = Pattern
+      .compile("[$][{](.*?)[}]");
+  private static final Pattern EVALUATOR_FORMAT_PATTERN = Pattern
+      .compile("^(\\w*?)\\((.*?)\\)$");
+  private Map<String,Object> rootNamespace;
+  private Map<String,Evaluator> evaluators;
+  private Map<String,Resolved> cache = new WeakHashMap<String,Resolved>();
+  
+  class Resolved {
+    List<Integer> startIndexes = new ArrayList<Integer>(2);
+    List<Integer> endOffsets = new ArrayList<Integer>(2);
+    List<String> variables = new ArrayList<String>(2);
+  }
+  
+  public static final String FUNCTIONS_NAMESPACE = "dataimporter.functions.";
+  
+  public VariableResolver() {
+    rootNamespace = new HashMap<String,Object>();
+  }
+  
+  public VariableResolver(Properties defaults) {
+    rootNamespace = new HashMap<String,Object>();
+    for (Map.Entry<Object,Object> entry : defaults.entrySet()) {
+      rootNamespace.put(entry.getKey().toString(), entry.getValue());
+    }
+  }
+  
+  public VariableResolver(Map<String,Object> defaults) {
+    rootNamespace = new HashMap<String,Object>(defaults);
+  }
+  
   /**
    * Resolves a given value with a name
-   *
-   * @param name the String to be resolved
+   * 
+   * @param name
+   *          the String to be resolved
    * @return an Object which is the result of evaluation of given name
    */
-  public abstract Object resolve(String name);
-
+  public Object resolve(String name) {
+    Object r = null;
+    if (name != null) {
+      String[] nameParts = DOT_PATTERN.split(name);
+      Map<String,Object> currentLevel = currentLevelMap(nameParts,
+          rootNamespace, false);
+      r = currentLevel.get(nameParts[nameParts.length - 1]);
+      if (r == null && name.startsWith(FUNCTIONS_NAMESPACE)
+          && name.length() > FUNCTIONS_NAMESPACE.length()) {
+        return resolveEvaluator(name);
+      }
+      if (r == null) {
+        r = System.getProperty(name);
+      }
+    }
+    return r == null ? "" : r;
+  }
+  
+  private Object resolveEvaluator(String name) {
+    if (evaluators == null) {
+      return "";
+    }
+    Matcher m = EVALUATOR_FORMAT_PATTERN.matcher(name
+        .substring(FUNCTIONS_NAMESPACE.length()));
+    if (m.find()) {
+      String fname = m.group(1);
+      Evaluator evaluator = evaluators.get(fname);
+      if (evaluator == null) return "";
+      ContextImpl ctx = new ContextImpl(null, this, null, null, null, null,
+          null);
+      String g2 = m.group(2);
+      return evaluator.evaluate(g2, ctx);
+    } else {
+      return "";
+    }
+  }
+  
   /**
    * Given a String with place holders, replace them with the value tokens.
-   *
+   * 
    * @return the string with the placeholders replaced with their values
    */
-  public abstract String replaceTokens(String template);
+  public String replaceTokens(String template) {
+    if (template == null) {
+      return null;
+    }
+    Resolved r = getResolved(template);
+    if (r.startIndexes != null) {
+      StringBuilder sb = new StringBuilder(template);
+      for (int i = r.startIndexes.size() - 1; i >= 0; i--) {
+        String replacement = resolve(r.variables.get(i)).toString();
+        sb.replace(r.startIndexes.get(i), r.endOffsets.get(i), replacement);
+      }
+      return sb.toString();
+    } else {
+      return template;
+    }
+  }
+  
+  private Resolved getResolved(String template) {
+    Resolved r = cache.get(template);
+    if (r == null) {
+      r = new Resolved();
+      Matcher m = PLACEHOLDER_PATTERN.matcher(template);
+      while (m.find()) {
+        String variable = m.group(1);
+        r.startIndexes.add(m.start(0));
+        r.endOffsets.add(m.end(0));
+        r.variables.add(variable);
+      }
+      cache.put(template, r);
+    }
+    return r;
+  }
+  /**
+   * Get a list of variables embedded in the template string.
+   */
+  public List<String> getVariables(String template) {
+    Resolved r = getResolved(template);
+    if (r == null) {
+      return Collections.emptyList();
+    }
+    return new ArrayList<String>(r.variables);
+  }
+  
+  public void addNamespace(String name, Map<String,Object> newMap) {
+    if (newMap != null) {
+      if (name != null) {
+        String[] nameParts = DOT_PATTERN.split(name);
+        Map<String,Object> nameResolveLevel = currentLevelMap(nameParts,
+            rootNamespace, false);
+        nameResolveLevel.put(nameParts[nameParts.length - 1], newMap);
+      } else {
+        for (Map.Entry<String,Object> entry : newMap.entrySet()) {
+          String[] keyParts = DOT_PATTERN.split(entry.getKey());
+          Map<String,Object> currentLevel = rootNamespace;
+          currentLevel = currentLevelMap(keyParts, currentLevel, false);
+          currentLevel.put(keyParts[keyParts.length - 1], entry.getValue());
+        }
+      }
+    }
+  }
+  
+  private Map<String,Object> currentLevelMap(String[] keyParts,
+      Map<String,Object> currentLevel, boolean includeLastLevel) {
+    int j = includeLastLevel ? keyParts.length : keyParts.length - 1;
+    for (int i = 0; i < j; i++) {
+      Object o = currentLevel.get(keyParts[i]);
+      if (o == null) {
+        Map<String,Object> nextLevel = new HashMap<String,Object>();
+        currentLevel.put(keyParts[i], nextLevel);
+        currentLevel = nextLevel;
+      } else if (o instanceof Map<?,?>) {
+        @SuppressWarnings("unchecked")
+        Map<String,Object> nextLevel = (Map<String,Object>) o;
+        currentLevel = nextLevel;
+      } else {
+        throw new AssertionError(
+            "Non-leaf nodes should be of type java.util.Map");
+      }
+    }
+    return currentLevel;
+  }
+  
+  public void removeNamespace(String name) {
+    rootNamespace.remove(name);
+  }
+  
+  public void setEvaluators(Map<String,Evaluator> evaluators) {
+    this.evaluators = evaluators;
+  }
 }

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java Fri Nov 23 12:00:32 2012
@@ -91,14 +91,14 @@ public class XPathEntityProcessor extend
   public void init(Context context) {
     super.init(context);
     if (xpathReader == null)
-      initXpathReader();
+      initXpathReader(context.getVariableResolver());
     pk = context.getEntityAttribute("pk");
     dataSource = context.getDataSource();
     rowIterator = null;
 
   }
 
-  private void initXpathReader() {
+  private void initXpathReader(VariableResolver resolver) {
     useSolrAddXml = Boolean.parseBoolean(context
             .getEntityAttribute(USE_SOLR_ADD_SCHEMA));
     streamRows = Boolean.parseBoolean(context
@@ -175,7 +175,7 @@ public class XPathEntityProcessor extend
       }
     }
     String url = context.getEntityAttribute(URL);
-    List<String> l = url == null ? Collections.EMPTY_LIST : TemplateString.getVariables(url);
+    List<String> l = url == null ? Collections.EMPTY_LIST : resolver.getVariables(url);
     for (String s : l) {
       if (s.startsWith(entityName + ".")) {
         if (placeHolderVariables == null)
@@ -259,7 +259,7 @@ public class XPathEntityProcessor extend
       Object val = context.getSessionAttribute(name, Context.SCOPE_ENTITY);
       if (val != null) namespace.put(name, val);
     }
-    ((VariableResolverImpl)context.getVariableResolver()).addNamespace(entityName, namespace);
+    ((VariableResolver)context.getVariableResolver()).addNamespace(entityName, namespace);
   }
 
   private void addCommonFields(Map<String, Object> r) {

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ZKPropertiesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ZKPropertiesWriter.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ZKPropertiesWriter.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ZKPropertiesWriter.java Fri Nov 23 12:00:32 2012
@@ -18,6 +18,7 @@ package org.apache.solr.handler.dataimpo
 
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
+import java.util.Map;
 import java.util.Properties;
 
 import org.apache.solr.common.cloud.SolrZkClient;
@@ -25,7 +26,13 @@ import org.apache.zookeeper.KeeperExcept
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-public class ZKPropertiesWriter implements DIHPropertiesWriter {
+/**
+ * <p>
+ *  A SolrCloud-friendly extension of {@link SimplePropertiesWriter}.  
+ *  This implementation ignores the "directory" parameter, saving
+ *  the properties file under /configs/[solrcloud collection name]/
+ */
+public class ZKPropertiesWriter extends SimplePropertiesWriter {
   
   private static final Logger log = LoggerFactory
       .getLogger(ZKPropertiesWriter.class);
@@ -34,16 +41,11 @@ public class ZKPropertiesWriter implemen
   private SolrZkClient zkClient;
   
   @Override
-  public void init(DataImporter dataImporter) {
+  public void init(DataImporter dataImporter, Map<String, String> params) {
+    super.init(dataImporter, params);
     String collection = dataImporter.getCore().getCoreDescriptor()
         .getCloudDescriptor().getCollectionName();
-    String persistFilename;
-    if(dataImporter.getHandlerName() != null){
-      persistFilename = dataImporter.getHandlerName() + ".properties";
-    } else {
-      persistFilename = SimplePropertiesWriter.IMPORTER_PROPERTIES;
-    }
-    path = "/configs/" + collection + "/" + persistFilename;
+    path = "/configs/" + collection + "/" + filename;
     zkClient = dataImporter.getCore().getCoreDescriptor().getCoreContainer()
         .getZkController().getZkClient();
   }
@@ -54,9 +56,9 @@ public class ZKPropertiesWriter implemen
   }
   
   @Override
-  public void persist(Properties props) {
-    Properties existing = readIndexerProperties();
-    existing.putAll(props);
+  public void persist(Map<String, Object> propObjs) {
+    Properties existing = mapToProperties(readIndexerProperties());
+    existing.putAll(mapToProperties(propObjs));
     ByteArrayOutputStream output = new ByteArrayOutputStream();
     try {
       existing.store(output, "");
@@ -78,7 +80,7 @@ public class ZKPropertiesWriter implemen
   }
   
   @Override
-  public Properties readIndexerProperties() {
+  public Map<String, Object> readIndexerProperties() {
     Properties props = new Properties();
     try {
       byte[] data = zkClient.getData(path, null, null, false);
@@ -90,6 +92,6 @@ public class ZKPropertiesWriter implemen
       log.warn(
           "Could not read DIH properties from " + path + " :" + e.getClass(), e);
     }
-    return props;
+    return propertiesToMap(props);
   }
 }

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/ConfigNameConstants.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/ConfigNameConstants.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/ConfigNameConstants.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/ConfigNameConstants.java Fri Nov 23 12:00:32 2012
@@ -28,6 +28,8 @@ public class ConfigNameConstants {
   public static final String NAME = "name";
 
   public static final String PROCESSOR = "processor";
+  
+  public static final String PROPERTY_WRITER = "propertyWriter";
 
   /**
    * @deprecated use IMPORTER_NS_SHORT instead

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/DIHConfiguration.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/DIHConfiguration.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/DIHConfiguration.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/DIHConfiguration.java Fri Nov 23 12:00:32 2012
@@ -43,13 +43,18 @@ import org.w3c.dom.Element;
 public class DIHConfiguration {
   // TODO - remove from here and add it to entity
   private final String deleteQuery;
+  
   private final List<Entity> entities;
   private final String onImportStart;
   private final String onImportEnd;
   private final List<Map<String, String>> functions;
   private final Script script;
   private final Map<String, Map<String,String>> dataSources;
-  public DIHConfiguration(Element element, DataImporter di, List<Map<String, String>> functions, Script script, Map<String, Map<String,String>> dataSources) {
+  private final PropertyWriter propertyWriter;
+  
+  public DIHConfiguration(Element element, DataImporter di,
+      List<Map<String,String>> functions, Script script,
+      Map<String,Map<String,String>> dataSources, PropertyWriter pw) {
     this.deleteQuery = ConfigParseUtil.getStringAttribute(element, "deleteQuery", null);
     this.onImportStart = ConfigParseUtil.getStringAttribute(element, "onImportStart", null);
     this.onImportEnd = ConfigParseUtil.getStringAttribute(element, "onImportEnd", null);
@@ -73,6 +78,7 @@ public class DIHConfiguration {
     this.functions = Collections.unmodifiableList(modFunc);
     this.script = script;
     this.dataSources = Collections.unmodifiableMap(dataSources);
+    this.propertyWriter = pw;
   }
   public String getDeleteQuery() {
     return deleteQuery;
@@ -95,4 +101,7 @@ public class DIHConfiguration {
   public Script getScript() {
     return script;
   }
+  public PropertyWriter getPropertyWriter() {
+    return propertyWriter;
+  }
 }
\ No newline at end of file

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHCacheTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHCacheTestCase.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHCacheTestCase.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHCacheTestCase.java Fri Nov 23 12:00:32 2012
@@ -226,7 +226,7 @@ public class AbstractDIHCacheTestCase {
   }
 
   public static Context getContext(final Map<String, String> entityAttrs) {
-    VariableResolverImpl resolver = new VariableResolverImpl();
+    VariableResolver resolver = new VariableResolver();
     final Context delegate = new ContextImpl(null, resolver, null, null, new HashMap<String, Object>(), null, null);
     return new TestContext(entityAttrs, delegate, null, true);
   }

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHJdbcTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHJdbcTestCase.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHJdbcTestCase.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHJdbcTestCase.java Fri Nov 23 12:00:32 2012
@@ -1,4 +1,5 @@
 package org.apache.solr.handler.dataimport;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements. See the NOTICE file distributed with this
@@ -16,155 +17,194 @@ package org.apache.solr.handler.dataimpo
  * the License.
  */
 
+import java.io.OutputStream;
 import java.sql.Connection;
 import java.sql.DriverManager;
-import java.sql.PreparedStatement;
 import java.sql.SQLException;
 import java.sql.Statement;
+import java.util.Locale;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
 
 import junit.framework.Assert;
 
+import org.apache.derby.iapi.error.StandardException;
+import org.apache.solr.request.LocalSolrQueryRequest;
+import org.junit.After;
 import org.junit.AfterClass;
+import org.junit.Assume;
+import org.junit.Before;
 import org.junit.BeforeClass;
+
 /**
- * This sets up an in-memory Derby Sql database with a little sample data.
- * The schema here is poorly-designed to illustrate DIH's ability to 
- * overcome these kinds of challenges.
+ * This sets up an in-memory Sql database with a little sample data.
  */
-public abstract class AbstractDIHJdbcTestCase extends AbstractDataImportHandlerTestCase {
+public abstract class AbstractDIHJdbcTestCase extends
+    AbstractDataImportHandlerTestCase {
+  
+  protected Database dbToUse;
+  
+  public enum Database {
+    RANDOM, DERBY, HSQLDB
+  }
+  
+  protected boolean skipThisTest = false;
+  
+  private static final Pattern totalRequestsPattern = Pattern
+      .compile(".str name..Total Requests made to DataSource..(\\d+)..str.");
+    
   @BeforeClass
   public static void beforeClassDihJdbcTest() throws Exception {
     try {
       Class.forName("org.hsqldb.jdbcDriver").newInstance();
+      String oldProp = System.getProperty("derby.stream.error.field");
+      System.setProperty("derby.stream.error.field", "DerbyUtil.DEV_NULL");
+      Class.forName("org.apache.derby.jdbc.EmbeddedDriver").newInstance();
+      if (oldProp != null) {
+        System.setProperty("derby.stream.error.field", oldProp);
+      }
     } catch (Exception e) {
       throw e;
     }
+    initCore("dataimport-solrconfig.xml", "dataimport-schema.xml");
+  }
+  
+  @AfterClass
+  public static void afterClassDihJdbcTest() throws Exception {
+    try {
+      DriverManager.getConnection("jdbc:derby:;shutdown=true");
+    } catch (SQLException e) {
+      // ignore...we might not even be using derby this time...
+    }
+  }
+  
+  protected Database setAllowedDatabases() {
+    return Database.RANDOM;
+  }
+  
+  @Before
+  public void beforeDihJdbcTest() throws Exception {  
+    skipThisTest = false;
+    dbToUse = setAllowedDatabases();
+    if (dbToUse == Database.RANDOM) {
+      if (random().nextBoolean()) {
+        dbToUse = Database.DERBY;
+      } else {
+        dbToUse = Database.HSQLDB;
+      }
+    }
     
+    clearIndex();
+    assertU(commit());
+    buildDatabase();
+  }
+  
+  @After
+  public void afterDihJdbcTest() throws Exception {
     Connection conn = null;
     Statement s = null;
-    PreparedStatement ps = null;    
-    try {    
-      conn = DriverManager.getConnection("jdbc:hsqldb:mem:.");    
-      s = conn.createStatement();
-      s.executeUpdate("create table countries(code char(2) not null primary key, country_name varchar(50))");
-      s.executeUpdate("create table people(id int not null primary key, name varchar(50), country_codes varchar(100))");
-      s.executeUpdate("create table people_sports(id int not null primary key, person_id int, sport_name varchar(50))");
-      
-      ps = conn.prepareStatement("insert into countries values (?,?)");
-      for(String[] country : countries) {
-        ps.setString(1, country[0]);
-        ps.setString(2, country[1]);
-        Assert.assertEquals(1, ps.executeUpdate());
-      }
-      ps.close();
-            
-      ps = conn.prepareStatement("insert into people values (?,?,?)");
-      for(Object[] person : people) {
-        ps.setInt(1, (Integer) person[0]);
-        ps.setString(2, (String) person[1]);
-        ps.setString(3, (String) person[2]);
-        Assert.assertEquals(1, ps.executeUpdate());
-      }
-      ps.close(); 
-      
-      ps = conn.prepareStatement("insert into people_sports values (?,?,?)");
-      for(Object[] sport : people_sports) {
-        ps.setInt(1, (Integer) sport[0]);
-        ps.setInt(2, (Integer) sport[1]);
-        ps.setString(3, (String) sport[2]);
-        Assert.assertEquals(1, ps.executeUpdate());
-      }
-      ps.close();
-      conn.close();    
-    } catch(Exception e) {
-      throw e;
+    try {
+      if (dbToUse == Database.DERBY) {
+        try {
+          conn = DriverManager
+              .getConnection("jdbc:derby:memory:derbyDB;drop=true");
+        } catch (SQLException e) {
+          if (!"08006".equals(e.getSQLState())) {
+            throw e;
+          }
+        }
+      } else if (dbToUse == Database.HSQLDB) {
+        conn = DriverManager.getConnection("jdbc:hsqldb:mem:.");
+        s = conn.createStatement();
+        s.executeUpdate("shutdown");
+      }
+    } catch (SQLException e) {
+      if(!skipThisTest) {
+        throw e;
+      }
     } finally {
-      if(s!=null) { s.close(); }
-      if(ps!=null) { ps.close(); }
-      if(conn!=null) { conn.close(); }
+      try {
+        s.close();
+      } catch (Exception ex) {}
+      try {
+        conn.close();
+      } catch (Exception ex) {}
     }
   }
   
-  @AfterClass
-  public static void afterClassDihJdbcTest() throws Exception {  
+  protected Connection newConnection() throws Exception {
+    if (dbToUse == Database.DERBY) {
+      return DriverManager.getConnection("jdbc:derby:memory:derbyDB;");
+    } else if (dbToUse == Database.HSQLDB) {
+      return DriverManager.getConnection("jdbc:hsqldb:mem:.");
+    }
+    throw new AssertionError("Invalid database to use: " + dbToUse);
+  }
+  
+  protected void buildDatabase() throws Exception {
     Connection conn = null;
-    Statement s = null;
-    try {      
-      conn = DriverManager.getConnection("jdbc:hsqldb:mem:.");    
-      s = conn.createStatement();
-      s.executeUpdate("shutdown");
-    } catch (SQLException e) {
-      throw e;
+    try {
+      if (dbToUse == Database.DERBY) {
+        conn = DriverManager
+            .getConnection("jdbc:derby:memory:derbyDB;create=true");
+      } else if (dbToUse == Database.HSQLDB) {
+        conn = DriverManager.getConnection("jdbc:hsqldb:mem:.");
+      } else {
+        throw new AssertionError("Invalid database to use: " + dbToUse);
+      }
+      populateData(conn);
+    } catch (SQLException sqe) {
+      Throwable cause = sqe;
+      while(cause.getCause()!=null) {
+        cause = cause.getCause();
+      }
+      String message = cause.getMessage();
+      if(cause instanceof StandardException) {
+        message = ((StandardException) cause).getMessageId();
+      }
+      //Derby INVALID_LOCALE_DESCRIPTION
+      if("XBM0X.D".equals(message)) {
+        log.warn("Skipping test because Database " + dbToUse + " does not support the locale " + Locale.getDefault());
+        skipThisTest = true;
+        Assume.assumeNoException(sqe); 
+        throw sqe;
+      }      
     } finally {
-      if(s!=null) { s.close(); }
-      if(conn!=null) { conn.close(); }
+      try {
+        conn.close();
+      } catch (Exception e1) {}
     }
   }
   
-  public static final String[][] countries = {
-    {"NA",   "Namibia"},
-    {"NC",   "New Caledonia"},
-    {"NE",   "Niger"},
-    {"NF",   "Norfolk Island"},
-    {"NG",   "Nigeria"},
-    {"NI",   "Nicaragua"},
-    {"NL",   "Netherlands"},
-    {"NO",   "Norway"},
-    {"NP",   "Nepal"},
-    {"NR",   "Nauru"},
-    {"NU",   "Niue"},
-    {"NZ",   "New Zealand"}
-  };
-  
-  public static final Object[][] people = {
-    {1,"Jacob","NZ"},
-    {2,"Ethan","NU,NA,NE"},
-    {3,"Michael","NR"},
-    {4,"Jayden","NP"},
-    {5,"William","NO"},
-    {6,"Alexander","NL"},
-    {7,"Noah","NI"},
-    {8,"Daniel","NG"},
-    {9,"Aiden","NF"},
-    {10,"Anthony","NE"},
-    {11,"Emma","NL"},
-    {12,"Grace","NI"},
-    {13,"Hailey","NG"},
-    {14,"Isabella","NF"},
-    {15,"Lily","NE"},
-    {16,"Madison","NC"},
-    {17,"Mia","NA"},
-    {18,"Natalie","NP,NR,NU,NZ"},
-    {19,"Olivia","NU"},
-    {20,"Samantha","NR"}
-  };
-  
-  public static final Object[][] people_sports = {
-    {100, 1, "Swimming"},
-    {200, 2, "Triathlon"},
-    {300, 3, "Water polo"},
-    {310, 3, "Underwater rugby"},
-    {320, 3, "Kayaking"},
-    {400, 4, "Snorkeling"},
-    {500, 5, "Synchronized diving"},
-    {600, 6, "Underwater rugby"},
-    {700, 7, "Boating"},
-    {800, 8, "Bodyboarding"},
-    {900, 9, "Canoeing"},
-    {1000, 10, "Fishing"},
-    {1100, 11, "Jet Ski"},
-    {1110, 11, "Rowing"},
-    {1120, 11, "Sailing"},
-    {1200, 12, "Kayaking"},
-    {1210, 12, "Canoeing"},
-    {1300, 13, "Kite surfing"},
-    {1400, 14, "Parasailing"},
-    {1500, 15, "Rafting"},
-    {1600, 16, "Rowing"},
-    {1700, 17, "Sailing"},
-    {1800, 18, "White Water Rafting"},
-    {1900, 19, "Water skiing"},
-    {2000, 20, "Windsurfing"}
-  };  
+  protected void populateData(Connection conn) throws Exception {
+    // no-op
+  }
+  
+  public int totalDatabaseRequests(String dihHandlerName) throws Exception {
+    LocalSolrQueryRequest request = lrf.makeRequest("indent", "true");
+    String response = h.query(dihHandlerName, request);
+    Matcher m = totalRequestsPattern.matcher(response);
+    Assert.assertTrue("The handler " + dihHandlerName
+        + " is not reporting any database requests. ",
+        m.find() && m.groupCount() == 1);
+    return Integer.parseInt(m.group(1));
+  }
+  
+  public int totalDatabaseRequests() throws Exception {
+    return totalDatabaseRequests("/dataimport");
+  }
+  
+  protected LocalSolrQueryRequest generateRequest() {
+    return lrf.makeRequest("command", "full-import", "dataConfig",
+        generateConfig(), "clean", "true", "commit", "true", "synchronous",
+        "true", "indent", "true");
+  }
+  
+  protected abstract String generateConfig();
   
-}
+  public static class DerbyUtil {
+    public static final OutputStream DEV_NULL = new OutputStream() {
+      public void write(int b) {}
+    };
+  }
+}
\ No newline at end of file

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDataImportHandlerTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDataImportHandlerTestCase.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDataImportHandlerTestCase.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDataImportHandlerTestCase.java Fri Nov 23 12:00:32 2012
@@ -133,10 +133,10 @@ public abstract class AbstractDataImport
    */
   @SuppressWarnings("unchecked")
   public static TestContext getContext(EntityProcessorWrapper parent,
-                                   VariableResolverImpl resolver, DataSource parentDataSource,
+                                   VariableResolver resolver, DataSource parentDataSource,
                                    String currProcess, final List<Map<String, String>> entityFields,
                                    final Map<String, String> entityAttrs) {
-    if (resolver == null) resolver = new VariableResolverImpl();
+    if (resolver == null) resolver = new VariableResolver();
     final Context delegate = new ContextImpl(parent, resolver,
             parentDataSource, currProcess,
             new HashMap<String, Object>(), null, null);

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestClobTransformer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestClobTransformer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestClobTransformer.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestClobTransformer.java Fri Nov 23 12:00:32 2012
@@ -42,7 +42,7 @@ public class TestClobTransformer extends
     f.put(ClobTransformer.CLOB, "true");
     f.put(DataImporter.NAME, "description");
     flds.add(f);
-    Context ctx = getContext(null, new VariableResolverImpl(), null, Context.FULL_DUMP, flds, Collections.EMPTY_MAP);
+    Context ctx = getContext(null, new VariableResolver(), null, Context.FULL_DUMP, flds, Collections.EMPTY_MAP);
     Transformer t = new ClobTransformer();
     Map<String, Object> row = new HashMap<String, Object>();
     Clob clob = (Clob) Proxy.newProxyInstance(this.getClass().getClassLoader(), new Class[]{Clob.class}, new InvocationHandler() {

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContextImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContextImpl.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContextImpl.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContextImpl.java Fri Nov 23 12:00:32 2012
@@ -25,7 +25,7 @@ public class TestContextImpl extends Abs
   
   @Test
   public void testEntityScope() {
-    ContextImpl ctx = new ContextImpl(null, new VariableResolverImpl(), null, "something", new HashMap<String,Object>(), null, null);
+    ContextImpl ctx = new ContextImpl(null, new VariableResolver(), null, "something", new HashMap<String,Object>(), null, null);
     String lala = new String("lala");
     ctx.setSessionAttribute("huhu", lala, Context.SCOPE_ENTITY);
     Object got = ctx.getSessionAttribute("huhu", Context.SCOPE_ENTITY);
@@ -38,7 +38,7 @@ public class TestContextImpl extends Abs
     DataImporter di = new DataImporter();
     di.loadAndInit("<dataConfig><document /></dataConfig>");
     DocBuilder db = new DocBuilder(di, new SolrWriter(null, null),new SimplePropertiesWriter(), new RequestInfo(new HashMap<String,Object>(), null));
-    ContextImpl ctx = new ContextImpl(null, new VariableResolverImpl(), null, "something", new HashMap<String,Object>(), null, db);
+    ContextImpl ctx = new ContextImpl(null, new VariableResolver(), null, "something", new HashMap<String,Object>(), null, db);
     String lala = new String("lala");
     ctx.setSessionAttribute("huhu", lala, Context.SCOPE_SOLR_CORE);
     Object got = ctx.getSessionAttribute("huhu", Context.SCOPE_SOLR_CORE);
@@ -47,7 +47,7 @@ public class TestContextImpl extends Abs
   }
   @Test
   public void testDocumentScope() {
-    ContextImpl ctx = new ContextImpl(null, new VariableResolverImpl(), null, "something", new HashMap<String,Object>(), null, null);
+    ContextImpl ctx = new ContextImpl(null, new VariableResolver(), null, "something", new HashMap<String,Object>(), null, null);
     ctx.setDoc(new DocBuilder.DocWrapper());
     String lala = new String("lala");
     ctx.setSessionAttribute("huhu", lala, Context.SCOPE_DOC);
@@ -58,7 +58,7 @@ public class TestContextImpl extends Abs
   }
   @Test
   public void testGlobalScope() {
-    ContextImpl ctx = new ContextImpl(null, new VariableResolverImpl(), null, "something", new HashMap<String,Object>(), null, null);
+    ContextImpl ctx = new ContextImpl(null, new VariableResolver(), null, "something", new HashMap<String,Object>(), null, null);
     String lala = new String("lala");
     ctx.setSessionAttribute("huhu", lala, Context.SCOPE_GLOBAL);
     Object got = ctx.getSessionAttribute("huhu", Context.SCOPE_GLOBAL);

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDateFormatTransformer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDateFormatTransformer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDateFormatTransformer.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDateFormatTransformer.java Fri Nov 23 12:00:32 2012
@@ -34,18 +34,18 @@ public class TestDateFormatTransformer e
   @Test
   @SuppressWarnings("unchecked")
   public void testTransformRow_SingleRow() throws Exception {
-    List fields = new ArrayList();
+    List<Map<String, String>> fields = new ArrayList<Map<String, String>>();
     fields.add(createMap(DataImporter.COLUMN, "lastModified"));
     fields.add(createMap(DataImporter.COLUMN,
             "dateAdded", RegexTransformer.SRC_COL_NAME, "lastModified",
             DateFormatTransformer.DATE_TIME_FMT, "MM/dd/yyyy"));
 
-    SimpleDateFormat format = new SimpleDateFormat("MM/dd/yyyy");
+    SimpleDateFormat format = new SimpleDateFormat("MM/dd/yyyy", Locale.ROOT);
     Date now = format.parse(format.format(new Date()));
 
-    Map row = createMap("lastModified", format.format(now));
+    Map<String,Object> row = createMap("lastModified", format.format(now));
 
-    VariableResolverImpl resolver = new VariableResolverImpl();
+    VariableResolver resolver = new VariableResolver();
     resolver.addNamespace("e", row);
 
     Context context = getContext(null, resolver,
@@ -57,29 +57,29 @@ public class TestDateFormatTransformer e
   @Test
   @SuppressWarnings("unchecked")
   public void testTransformRow_MultipleRows() throws Exception {
-    List fields = new ArrayList();
+    List<Map<String, String>> fields = new ArrayList<Map<String, String>>();
     fields.add(createMap(DataImporter.COLUMN, "lastModified"));
     fields.add(createMap(DataImporter.COLUMN,
             "dateAdded", RegexTransformer.SRC_COL_NAME, "lastModified",
             DateFormatTransformer.DATE_TIME_FMT, "MM/dd/yyyy hh:mm:ss.SSS"));
 
-    SimpleDateFormat format = new SimpleDateFormat("MM/dd/yyyy hh:mm:ss.SSS");
+    SimpleDateFormat format = new SimpleDateFormat("MM/dd/yyyy hh:mm:ss.SSS", Locale.ROOT);
     Date now1 = format.parse(format.format(new Date()));
     Date now2 = format.parse(format.format(new Date()));
 
-    Map row = new HashMap();
-    List list = new ArrayList();
+    Map<String,Object> row = new HashMap<String,Object>();
+    List<String> list = new ArrayList<String>();
     list.add(format.format(now1));
     list.add(format.format(now2));
     row.put("lastModified", list);
 
-    VariableResolverImpl resolver = new VariableResolverImpl();
+    VariableResolver resolver = new VariableResolver();
     resolver.addNamespace("e", row);
 
     Context context = getContext(null, resolver,
             null, Context.FULL_DUMP, fields, null);
     new DateFormatTransformer().transformRow(row, context);
-    List output = new ArrayList();
+    List<Object> output = new ArrayList<Object>();
     output.add(now1);
     output.add(now2);
     assertEquals(output, row.get("dateAdded"));

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDocBuilder2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDocBuilder2.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDocBuilder2.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDocBuilder2.java Fri Nov 23 12:00:32 2012
@@ -247,14 +247,14 @@ public class TestDocBuilder2 extends Abs
 
     Map<String, String> params = createMap("baseDir", tmpdir.getAbsolutePath());
 
-    createFile(tmpdir, "a.xml", "a.xml".getBytes(), true);
-    createFile(tmpdir, "b.xml", "b.xml".getBytes(), true);
-    createFile(tmpdir, "c.props", "c.props".getBytes(), true);
+    createFile(tmpdir, "a.xml", "a.xml".getBytes("UTF-8"), true);
+    createFile(tmpdir, "b.xml", "b.xml".getBytes("UTF-8"), true);
+    createFile(tmpdir, "c.props", "c.props".getBytes("UTF-8"), true);
     runFullImport(dataConfigFileList, params);
     assertQ(req("*:*"), "//*[@numFound='3']");
 
     // Add a new file after a full index is done
-    createFile(tmpdir, "t.xml", "t.xml".getBytes(), false);
+    createFile(tmpdir, "t.xml", "t.xml".getBytes("UTF-8"), false);
     runFullImport(dataConfigFileList, params);
     // we should find only 1 because by default clean=true is passed
     // and this particular import should find only one file t.xml

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java Fri Nov 23 12:00:32 2012
@@ -21,6 +21,7 @@ import org.junit.Test;
 
 import java.io.File;
 import java.io.IOException;
+import java.text.SimpleDateFormat;
 import java.util.*;
 
 /**
@@ -40,14 +41,14 @@ public class TestFileListEntityProcessor
     tmpdir.delete();
     tmpdir.mkdir();
     tmpdir.deleteOnExit();
-    createFile(tmpdir, "a.xml", "a.xml".getBytes(), false);
-    createFile(tmpdir, "b.xml", "b.xml".getBytes(), false);
-    createFile(tmpdir, "c.props", "c.props".getBytes(), false);
+    createFile(tmpdir, "a.xml", "a.xml".getBytes("UTF-8"), false);
+    createFile(tmpdir, "b.xml", "b.xml".getBytes("UTF-8"), false);
+    createFile(tmpdir, "c.props", "c.props".getBytes("UTF-8"), false);
     Map attrs = createMap(
             FileListEntityProcessor.FILE_NAME, "xml$",
             FileListEntityProcessor.BASE_DIR, tmpdir.getAbsolutePath());
     Context c = getContext(null,
-            new VariableResolverImpl(), null, Context.FULL_DUMP, Collections.EMPTY_LIST, attrs);
+            new VariableResolver(), null, Context.FULL_DUMP, Collections.EMPTY_LIST, attrs);
     FileListEntityProcessor fileListEntityProcessor = new FileListEntityProcessor();
     fileListEntityProcessor.init(c);
     List<String> fList = new ArrayList<String>();
@@ -108,14 +109,14 @@ public class TestFileListEntityProcessor
             FileListEntityProcessor.FILE_NAME, ".*",
             FileListEntityProcessor.BASE_DIR, tmpdir.getAbsolutePath(),
             FileListEntityProcessor.SMALLER_THAN, "${a.x}");
-    VariableResolverImpl resolver = new VariableResolverImpl();
+    VariableResolver resolver = new VariableResolver();
     resolver.addNamespace("a", createMap("x", "4"));
     fList = getFiles(resolver, attrs);
     assertEquals(l, new HashSet<String>(fList));
   }
 
   @SuppressWarnings("unchecked")
-  static List<String> getFiles(VariableResolverImpl resolver, Map attrs) {
+  static List<String> getFiles(VariableResolver resolver, Map attrs) {
     Context c = getContext(null,
             resolver, null, Context.FULL_DUMP, Collections.EMPTY_LIST, attrs);
     FileListEntityProcessor fileListEntityProcessor = new FileListEntityProcessor();
@@ -137,9 +138,9 @@ public class TestFileListEntityProcessor
     tmpdir.delete();
     tmpdir.mkdir();
     tmpdir.deleteOnExit();
-    createFile(tmpdir, "a.xml", "a.xml".getBytes(), true);
-    createFile(tmpdir, "b.xml", "b.xml".getBytes(), true);
-    createFile(tmpdir, "c.props", "c.props".getBytes(), true);
+    createFile(tmpdir, "a.xml", "a.xml".getBytes("UTF-8"), true);
+    createFile(tmpdir, "b.xml", "b.xml".getBytes("UTF-8"), true);
+    createFile(tmpdir, "c.props", "c.props".getBytes("UTF-8"), true);
     Map attrs = createMap(
             FileListEntityProcessor.FILE_NAME, "xml$",
             FileListEntityProcessor.BASE_DIR, tmpdir.getAbsolutePath(),
@@ -158,10 +159,10 @@ public class TestFileListEntityProcessor
             FileListEntityProcessor.FILE_NAME, ".xml$",
             FileListEntityProcessor.BASE_DIR, tmpdir.getAbsolutePath(),
             FileListEntityProcessor.NEWER_THAN, "${a.x}");
-    VariableResolverImpl resolver = new VariableResolverImpl();
-    String lastMod = DataImporter.DATE_TIME_FORMAT.get().format(new Date(System.currentTimeMillis() - 50000));
+    VariableResolver resolver = new VariableResolver();
+    String lastMod = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.ROOT).format(new Date(System.currentTimeMillis() - 50000));
     resolver.addNamespace("a", createMap("x", lastMod));
-    createFile(tmpdir, "t.xml", "t.xml".getBytes(), false);
+    createFile(tmpdir, "t.xml", "t.xml".getBytes("UTF-8"), false);
     fList = getFiles(resolver, attrs);
     assertEquals(1, fList.size());
     assertEquals("File name must be t.xml", new File(tmpdir, "t.xml").getAbsolutePath(), fList.get(0));
@@ -176,9 +177,9 @@ public class TestFileListEntityProcessor
     File childdir = new File(tmpdir + "/child" );
     childdir.mkdirs();
     childdir.deleteOnExit();
-    createFile(childdir, "a.xml", "a.xml".getBytes(), true);
-    createFile(childdir, "b.xml", "b.xml".getBytes(), true);
-    createFile(childdir, "c.props", "c.props".getBytes(), true);
+    createFile(childdir, "a.xml", "a.xml".getBytes("UTF-8"), true);
+    createFile(childdir, "b.xml", "b.xml".getBytes("UTF-8"), true);
+    createFile(childdir, "c.props", "c.props".getBytes("UTF-8"), true);
     Map attrs = createMap(
             FileListEntityProcessor.FILE_NAME, "^.*\\.xml$",
             FileListEntityProcessor.BASE_DIR, childdir.getAbsolutePath(),

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListWithLineEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListWithLineEntityProcessor.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListWithLineEntityProcessor.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListWithLineEntityProcessor.java Fri Nov 23 12:00:32 2012
@@ -33,9 +33,9 @@ public class TestFileListWithLineEntityP
     tmpdir.delete();
     tmpdir.mkdir();
     tmpdir.deleteOnExit();
-    createFile(tmpdir, "a.txt", "a line one\na line two\na line three".getBytes(), false);
-    createFile(tmpdir, "b.txt", "b line one\nb line two".getBytes(), false);
-    createFile(tmpdir, "c.txt", "c line one\nc line two\nc line three\nc line four".getBytes(), false);
+    createFile(tmpdir, "a.txt", "a line one\na line two\na line three".getBytes("UTF-8"), false);
+    createFile(tmpdir, "b.txt", "b line one\nb line two".getBytes("UTF-8"), false);
+    createFile(tmpdir, "c.txt", "c line one\nc line two\nc line three\nc line four".getBytes("UTF-8"), false);
     
     String config = generateConfig(tmpdir);
     LocalSolrQueryRequest request = lrf.makeRequest(

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestLineEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestLineEntityProcessor.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestLineEntityProcessor.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestLineEntityProcessor.java Fri Nov 23 12:00:32 2012
@@ -51,7 +51,7 @@ public class TestLineEntityProcessor ext
 
     Context c = getContext(
             null,                          //parentEntity
-            new VariableResolverImpl(),  //resolver
+            new VariableResolver(),  //resolver
             getDataSource(filecontents),   //parentDataSource
             Context.FULL_DUMP,                             //currProcess
             Collections.EMPTY_LIST,        //entityFields
@@ -91,7 +91,7 @@ public class TestLineEntityProcessor ext
 
     Context c = getContext(
             null,                          //parentEntity
-            new VariableResolverImpl(),  //resolver
+            new VariableResolver(),  //resolver
             getDataSource(filecontents),   //parentDataSource
             Context.FULL_DUMP,                             //currProcess
             Collections.EMPTY_LIST,        //entityFields
@@ -129,7 +129,7 @@ public class TestLineEntityProcessor ext
 
     Context c = getContext(
             null,                          //parentEntity
-            new VariableResolverImpl(),  //resolver
+            new VariableResolver(),  //resolver
             getDataSource(filecontents),   //parentDataSource
             Context.FULL_DUMP,                             //currProcess
             Collections.EMPTY_LIST,        //entityFields
@@ -165,7 +165,7 @@ public class TestLineEntityProcessor ext
 
     Context c = getContext(
             null,                          //parentEntity
-            new VariableResolverImpl(),  //resolver
+            new VariableResolver(),  //resolver
             getDataSource(filecontents),   //parentDataSource
             Context.FULL_DUMP,                             //currProcess
             Collections.EMPTY_LIST,        //entityFields

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestNumberFormatTransformer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestNumberFormatTransformer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestNumberFormatTransformer.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestNumberFormatTransformer.java Fri Nov 23 12:00:32 2012
@@ -33,20 +33,20 @@ import java.util.Map;
  * @since solr 1.3
  */
 public class TestNumberFormatTransformer extends AbstractDataImportHandlerTestCase {
-  private char GROUPING_SEP = new DecimalFormatSymbols().getGroupingSeparator();
-  private char DECIMAL_SEP = new DecimalFormatSymbols().getDecimalSeparator();
+  private char GROUPING_SEP = new DecimalFormatSymbols(Locale.ROOT).getGroupingSeparator();
+  private char DECIMAL_SEP = new DecimalFormatSymbols(Locale.ROOT).getDecimalSeparator();
 
-  @Test
   @SuppressWarnings("unchecked")
+  @Test
   public void testTransformRow_SingleNumber() {
     char GERMAN_GROUPING_SEP = new DecimalFormatSymbols(Locale.GERMANY).getGroupingSeparator();
-    List l = new ArrayList();
+    List<Map<String, String>> l = new ArrayList<Map<String, String>>();
     l.add(createMap("column", "num",
             NumberFormatTransformer.FORMAT_STYLE, NumberFormatTransformer.NUMBER));
     l.add(createMap("column", "localizedNum",
             NumberFormatTransformer.FORMAT_STYLE, NumberFormatTransformer.NUMBER, NumberFormatTransformer.LOCALE, "de-DE"));
     Context c = getContext(null, null, null, Context.FULL_DUMP, l, null);
-    Map m = createMap("num", "123" + GROUPING_SEP + "567", "localizedNum", "123" + GERMAN_GROUPING_SEP + "567");
+    Map<String,Object> m = createMap("num", "123" + GROUPING_SEP + "567", "localizedNum", "123" + GERMAN_GROUPING_SEP + "567");
     new NumberFormatTransformer().transformRow(m, c);
     assertEquals(new Long(123567), m.get("num"));
     assertEquals(new Long(123567), m.get("localizedNum"));
@@ -55,27 +55,27 @@ public class TestNumberFormatTransformer
   @Test
   @SuppressWarnings("unchecked")
   public void testTransformRow_MultipleNumbers() throws Exception {
-    List fields = new ArrayList();
+    List<Map<String, String>> fields = new ArrayList<Map<String, String>>();
     fields.add(createMap(DataImporter.COLUMN, "inputs"));
     fields.add(createMap(DataImporter.COLUMN,
             "outputs", RegexTransformer.SRC_COL_NAME, "inputs",
             NumberFormatTransformer.FORMAT_STYLE, NumberFormatTransformer.NUMBER));
 
-    List inputs = new ArrayList();
+    List<String> inputs = new ArrayList<String>();
     inputs.add("123" + GROUPING_SEP + "567");
     inputs.add("245" + GROUPING_SEP + "678");
-    Map row = createMap("inputs", inputs);
+    Map<String, Object> row = createMap("inputs", inputs);
 
-    VariableResolverImpl resolver = new VariableResolverImpl();
+    VariableResolver resolver = new VariableResolver();
     resolver.addNamespace("e", row);
 
     Context context = getContext(null, resolver, null, Context.FULL_DUMP, fields, null);
     new NumberFormatTransformer().transformRow(row, context);
 
-    List output = new ArrayList();
+    List<Long> output = new ArrayList<Long>();
     output.add(new Long(123567));
     output.add(new Long(245678));
-    Map outputRow = createMap("inputs", inputs, "outputs", output);
+    Map<String, Object> outputRow = createMap("inputs", inputs, "outputs", output);
 
     assertEquals(outputRow, row);
   }
@@ -83,77 +83,77 @@ public class TestNumberFormatTransformer
   @Test(expected = DataImportHandlerException.class)
   @SuppressWarnings("unchecked")
   public void testTransformRow_InvalidInput1_Number() {
-    List l = new ArrayList();
+    List<Map<String, String>> l = new ArrayList<Map<String, String>>();
     l.add(createMap("column", "num",
             NumberFormatTransformer.FORMAT_STYLE, NumberFormatTransformer.NUMBER));
     Context c = getContext(null, null, null, Context.FULL_DUMP, l, null);
-    Map m = createMap("num", "123" + GROUPING_SEP + "5a67");
+    Map<String, Object> m = createMap("num", "123" + GROUPING_SEP + "5a67");
     new NumberFormatTransformer().transformRow(m, c);
   }
 
   @Test(expected = DataImportHandlerException.class)
   @SuppressWarnings("unchecked")
   public void testTransformRow_InvalidInput2_Number() {
-    List l = new ArrayList();
+    List<Map<String, String>> l = new ArrayList<Map<String, String>>();
     l.add(createMap("column", "num",
             NumberFormatTransformer.FORMAT_STYLE, NumberFormatTransformer.NUMBER));
     Context c = getContext(null, null, null, Context.FULL_DUMP, l, null);
-    Map m = createMap("num", "123" + GROUPING_SEP + "567b");
+    Map<String, Object> m = createMap("num", "123" + GROUPING_SEP + "567b");
     new NumberFormatTransformer().transformRow(m, c);
   }
 
   @Test(expected = DataImportHandlerException.class)
   @SuppressWarnings("unchecked")
   public void testTransformRow_InvalidInput2_Currency() {
-    List l = new ArrayList();
+    List<Map<String, String>> l = new ArrayList<Map<String, String>>();
     l.add(createMap("column", "num",
             NumberFormatTransformer.FORMAT_STYLE, NumberFormatTransformer.CURRENCY));
     Context c = getContext(null, null, null, Context.FULL_DUMP, l, null);
-    Map m = createMap("num", "123" + GROUPING_SEP + "567b");
+    Map<String, Object> m = createMap("num", "123" + GROUPING_SEP + "567b");
     new NumberFormatTransformer().transformRow(m, c);
   }
 
   @Test(expected = DataImportHandlerException.class)
   @SuppressWarnings("unchecked")
   public void testTransformRow_InvalidInput1_Percent() {
-    List l = new ArrayList();
+    List<Map<String, String>> l = new ArrayList<Map<String, String>>();
     l.add(createMap("column", "num",
             NumberFormatTransformer.FORMAT_STYLE, NumberFormatTransformer.PERCENT));
     Context c = getContext(null, null, null, Context.FULL_DUMP, l, null);
-    Map m = createMap("num", "123" + GROUPING_SEP + "5a67");
+    Map<String, Object> m = createMap("num", "123" + GROUPING_SEP + "5a67");
     new NumberFormatTransformer().transformRow(m, c);
   }
 
   @Test(expected = DataImportHandlerException.class)
   @SuppressWarnings("unchecked")
   public void testTransformRow_InvalidInput3_Currency() {
-    List l = new ArrayList();
+    List<Map<String, String>> l = new ArrayList<Map<String, String>>();
     l.add(createMap("column", "num",
             NumberFormatTransformer.FORMAT_STYLE, NumberFormatTransformer.CURRENCY));
     Context c = getContext(null, null, null, Context.FULL_DUMP, l, null);
-    Map m = createMap("num", "123" + DECIMAL_SEP + "456" + DECIMAL_SEP + "789");
+    Map<String, Object> m = createMap("num", "123" + DECIMAL_SEP + "456" + DECIMAL_SEP + "789");
     new NumberFormatTransformer().transformRow(m, c);
   }
 
   @Test(expected = DataImportHandlerException.class)
   @SuppressWarnings("unchecked")
   public void testTransformRow_InvalidInput3_Number() {
-    List l = new ArrayList();
+    List<Map<String, String>> l = new ArrayList<Map<String, String>>();
     l.add(createMap("column", "num",
             NumberFormatTransformer.FORMAT_STYLE, NumberFormatTransformer.NUMBER));
     Context c = getContext(null, null, null, Context.FULL_DUMP, l, null);
-    Map m = createMap("num", "123" + DECIMAL_SEP + "456" + DECIMAL_SEP + "789");
+    Map<String, Object> m = createMap("num", "123" + DECIMAL_SEP + "456" + DECIMAL_SEP + "789");
     new NumberFormatTransformer().transformRow(m, c);
   }
 
   @Test
   @SuppressWarnings("unchecked")
   public void testTransformRow_MalformedInput_Number() {
-    List l = new ArrayList();
+    List<Map<String, String>> l = new ArrayList<Map<String, String>>();
     l.add(createMap("column", "num",
             NumberFormatTransformer.FORMAT_STYLE, NumberFormatTransformer.NUMBER));
     Context c = getContext(null, null, null, Context.FULL_DUMP, l, null);
-    Map m = createMap("num", "123" + GROUPING_SEP + GROUPING_SEP + "789");
+    Map<String, Object> m = createMap("num", "123" + GROUPING_SEP + GROUPING_SEP + "789");
     new NumberFormatTransformer().transformRow(m, c);
     assertEquals(new Long(123789), m.get("num"));
   }

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestRegexTransformer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestRegexTransformer.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestRegexTransformer.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestRegexTransformer.java Fri Nov 23 12:00:32 2012
@@ -134,7 +134,7 @@ public class TestRegexTransformer extend
     String s = "Fuel Economy Range: 26 mpg Hwy, 19 mpg City";
     row.put("rowdata", s);
 
-    VariableResolverImpl resolver = new VariableResolverImpl();
+    VariableResolver resolver = new VariableResolver();
     resolver.addNamespace("e", row);
     Map<String, String> eAttrs = createMap("name", "e");
     Context context = getContext(null, resolver, null, Context.FULL_DUMP, fields, eAttrs);

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSqlEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSqlEntityProcessor.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSqlEntityProcessor.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSqlEntityProcessor.java Fri Nov 23 12:00:32 2012
@@ -1,3 +1,8 @@
+package org.apache.solr.handler.dataimport;
+
+import org.junit.Ignore;
+import org.junit.Test;
+
 /*
  * Licensed to the Apache Software Foundation (ASF) under one or more
  * contributor license agreements.  See the NOTICE file distributed with
@@ -14,168 +19,44 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-package org.apache.solr.handler.dataimport;
-
-import org.junit.Test;
-import org.junit.Ignore;
-
-import java.util.*;
 
 /**
- * <p>
- * Test for SqlEntityProcessor
- * </p>
- *
- *
- * @since solr 1.3
+ * Test with various combinations of parameters, child entities, caches, transformers.
  */
-@Ignore("FIXME: I fail so often it makes me ill!")
-public class TestSqlEntityProcessor extends AbstractDataImportHandlerTestCase {
-  private static ThreadLocal<Integer> local = new ThreadLocal<Integer>();
-
+public class TestSqlEntityProcessor extends AbstractSqlEntityProcessorTestCase { 
+   
   @Test
-  public void testSingleBatch() {
-    SqlEntityProcessor sep = new SqlEntityProcessor();
-    List<Map<String, Object>> rows = getRows(3);
-    VariableResolverImpl vr = new VariableResolverImpl();
-    HashMap<String, String> ea = new HashMap<String, String>();
-    ea.put("query", "SELECT * FROM A");
-    Context c = getContext(null, vr, getDs(rows), Context.FULL_DUMP, null, ea);
-    sep.init(c);
-    int count = 0;
-    while (true) {
-      Map<String, Object> r = sep.nextRow();
-      if (r == null)
-        break;
-      count++;
-    }
-
-    assertEquals(3, count);
-  }
-
+  public void testSingleEntity() throws Exception {
+    singleEntity(1);
+  }  
   @Test
-  public void testTranformer() {
-    EntityProcessor sep = new EntityProcessorWrapper( new SqlEntityProcessor(), null, null);
-    List<Map<String, Object>> rows = getRows(2);
-    VariableResolverImpl vr = new VariableResolverImpl();
-    HashMap<String, String> ea = new HashMap<String, String>();
-    ea.put("query", "SELECT * FROM A");
-    ea.put("transformer", T.class.getName());
-
-    sep.init(getContext(null, vr, getDs(rows), Context.FULL_DUMP, null, ea));
-    List<Map<String, Object>> rs = new ArrayList<Map<String, Object>>();
-    Map<String, Object> r = null;
-    while (true) {
-      r = sep.nextRow();
-      if (r == null)
-        break;
-      rs.add(r);
-
-    }
-    assertEquals(2, rs.size());
-    assertNotNull(rs.get(0).get("T"));
+  public void testWithSimpleTransformer() throws Exception {
+    simpleTransform(1);   
   }
-
   @Test
-  public void testTranformerWithReflection() {
-    EntityProcessor sep = new EntityProcessorWrapper(new SqlEntityProcessor(), null, null);
-    List<Map<String, Object>> rows = getRows(2);
-    VariableResolverImpl vr = new VariableResolverImpl();
-    HashMap<String, String> ea = new HashMap<String, String>();
-    ea.put("query", "SELECT * FROM A");
-    ea.put("transformer", T3.class.getName());
-
-    sep.init(getContext(null, vr, getDs(rows), Context.FULL_DUMP, null, ea));
-    List<Map<String, Object>> rs = new ArrayList<Map<String, Object>>();
-    Map<String, Object> r = null;
-    while (true) {
-      r = sep.nextRow();
-      if (r == null)
-        break;
-      rs.add(r);
-
-    }
-    assertEquals(2, rs.size());
-    assertNotNull(rs.get(0).get("T3"));
+  public void testWithComplexTransformer() throws Exception {
+    complexTransform(1, 0);
   }
-
   @Test
-  public void testTranformerList() {
-    EntityProcessor sep = new EntityProcessorWrapper(new SqlEntityProcessor(),null, null);
-    List<Map<String, Object>> rows = getRows(2);
-    VariableResolverImpl vr = new VariableResolverImpl();
-
-    HashMap<String, String> ea = new HashMap<String, String>();
-    ea.put("query", "SELECT * FROM A");
-    ea.put("transformer", T2.class.getName());
-    sep.init(getContext(null, vr, getDs(rows), Context.FULL_DUMP, null, ea));
-
-    local.set(0);
-    Map<String, Object> r = null;
-    int count = 0;
-    while (true) {
-      r = sep.nextRow();
-      if (r == null)
-        break;
-      count++;
-    }
-    assertEquals(2, (int) local.get());
-    assertEquals(4, count);
-  }
-
-  private List<Map<String, Object>> getRows(int count) {
-    List<Map<String, Object>> rows = new ArrayList<Map<String, Object>>();
-    for (int i = 0; i < count; i++) {
-      Map<String, Object> row = new HashMap<String, Object>();
-      row.put("id", i);
-      row.put("value", "The value is " + i);
-      rows.add(row);
-    }
-    return rows;
-  }
-
-  private static DataSource<Iterator<Map<String, Object>>> getDs(
-          final List<Map<String, Object>> rows) {
-    return new DataSource<Iterator<Map<String, Object>>>() {
-      @Override
-      public Iterator<Map<String, Object>> getData(String query) {
-        return rows.iterator();
-      }
-
-      @Override
-      public void init(Context context, Properties initProps) {
-      }
-
-      @Override
-      public void close() {
-      }
-    };
+  public void testChildEntities() throws Exception {
+    withChildEntities(false, true);
   }
-
-  public static class T extends Transformer {
-    @Override
-    public Object transformRow(Map<String, Object> aRow, Context context) {
-      aRow.put("T", "Class T");
-      return aRow;
-    }
-  }
-
-  public static class T3 {
-    public Object transformRow(Map<String, Object> aRow) {
-      aRow.put("T3", "T3 class");
-      return aRow;
-    }
-  }
-
-  public static class T2 extends Transformer {
-    @Override
-    public Object transformRow(Map<String, Object> aRow, Context context) {
-      Integer count = local.get();
-      local.set(count + 1);
-      List<Map<String, Object>> l = new ArrayList<Map<String, Object>>();
-      l.add(aRow);
-      l.add(aRow);
-      return l;
-    }
+  @Test
+  public void testCachedChildEntities() throws Exception {
+    withChildEntities(true, true);
   }
+  @Test
+  @Ignore("broken see SOLR-3857")
+  public void testSimpleCacheChildEntities() throws Exception {
+    simpleCacheChildEntities(true);
+  }
+   
+  @Override
+  protected String deltaQueriesCountryTable() {
+    return "";
+  }
+  @Override
+  protected String deltaQueriesPersonTable() {
+    return "";
+  }  
 }

Modified: lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSqlEntityProcessor2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSqlEntityProcessor2.java?rev=1412849&r1=1412848&r2=1412849&view=diff
==============================================================================
--- lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSqlEntityProcessor2.java (original)
+++ lucene/dev/branches/LUCENE-2878/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSqlEntityProcessor2.java Fri Nov 23 12:00:32 2012
@@ -23,6 +23,7 @@ import org.junit.Test;
 
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Locale;
 import java.text.SimpleDateFormat;
 import java.text.ParseException;
 
@@ -41,8 +42,9 @@ public class TestSqlEntityProcessor2 ext
   public static void beforeClass() throws Exception {
     initCore("dataimport-solrconfig.xml", "dataimport-schema.xml");
   }
-
-  @Before @Override
+  
+  @Before 
+  @Override
   public void setUp() throws Exception {
     super.setUp();
     clearIndex();
@@ -51,140 +53,13 @@ public class TestSqlEntityProcessor2 ext
   
   @Test
   @SuppressWarnings("unchecked")
-  public void testCompositePk_FullImport() throws Exception {
-    List parentRow = new ArrayList();
-    parentRow.add(createMap("id", "1"));
-    MockDataSource.setIterator("select * from x", parentRow.iterator());
-
-    List childRow = new ArrayList();
-    childRow.add(createMap("desc", "hello"));
-
-    MockDataSource.setIterator("select * from y where y.A=1", childRow
-            .iterator());
-
-    runFullImport(dataConfig);
-
-    assertQ(req("id:1"), "//*[@numFound='1']");
-    assertQ(req("desc:hello"), "//*[@numFound='1']");
-  }
-  
-  @Test
-  @SuppressWarnings("unchecked")
-  public void testCompositePk_FullImportWithoutCommit() throws Exception {
-    List parentRow = new ArrayList();
-    parentRow.add(createMap("id", "10"));
-    MockDataSource.setIterator("select * from x", parentRow.iterator());
-
-    List childRow = new ArrayList();
-    childRow.add(createMap("desc", "hello"));
-
-    MockDataSource.setIterator("select * from y where y.A=10", childRow
-            .iterator());
-
-
-    runFullImport(dataConfig,createMap("commit","false"));
-    assertQ(req("id:10"), "//*[@numFound='0']");
-  }
-
-  @Test
-  @SuppressWarnings("unchecked")
-  public void testCompositePk_DeltaImport() throws Exception {
-    List deltaRow = new ArrayList();
-    deltaRow.add(createMap("id", "5"));
-    MockDataSource.setIterator("select id from x where last_modified > NOW",
-            deltaRow.iterator());
-
-    List parentRow = new ArrayList();
-    parentRow.add(createMap("id", "5"));
-    MockDataSource.setIterator("select * from x where id = '5'", parentRow
-            .iterator());
-
-    List childRow = new ArrayList();
-    childRow.add(createMap("desc", "hello"));
-    MockDataSource.setIterator("select * from y where y.A=5", childRow
-            .iterator());
-
-    runDeltaImport(dataConfig);
-
-    assertQ(req("id:5"), "//*[@numFound='1']");
-    assertQ(req("desc:hello"), "//*[@numFound='1']");
-  }
-
-  @Test
-  @SuppressWarnings("unchecked")
-  public void testCompositePk_DeltaImport_DeletedPkQuery() throws Exception {
-    List parentRow = new ArrayList();
-    parentRow.add(createMap("id", "11"));
-    MockDataSource.setIterator("select * from x", parentRow.iterator());
-
-    List childRow = new ArrayList();
-    childRow.add(createMap("desc", "hello"));
-
-    MockDataSource.setIterator("select * from y where y.A=11", childRow
-            .iterator());
-
-    runFullImport(dataConfig);
-
-    assertQ(req("id:11"), "//*[@numFound='1']");
-
-
-
-    List deltaRow = new ArrayList();
-    deltaRow.add(createMap("id", "15"));
-    deltaRow.add(createMap("id", "17"));
-    MockDataSource.setIterator("select id from x where last_modified > NOW",
-            deltaRow.iterator());
-
-    List deltaDeleteRow = new ArrayList();
-    deltaDeleteRow.add(createMap("id", "11"));
-    deltaDeleteRow.add(createMap("id", "17"));
-    MockDataSource.setIterator("select id from x where last_modified > NOW AND deleted='true'",
-            deltaDeleteRow.iterator());
-
-    parentRow = new ArrayList();
-    parentRow.add(createMap("id", "15"));
-    MockDataSource.setIterator("select * from x where id = '15'", parentRow
-            .iterator());
-
-    parentRow = new ArrayList();
-    parentRow.add(createMap("id", "17"));
-    MockDataSource.setIterator("select * from x where id = '17'", parentRow
-            .iterator());
-
-    runDeltaImport(dataConfig);
-
-    assertQ(req("id:15"), "//*[@numFound='1']");
-    assertQ(req("id:11"), "//*[@numFound='0']");
-    assertQ(req("id:17"), "//*[@numFound='0']");
-  }
-
-  @Test
-  @SuppressWarnings("unchecked")
-  public void testCompositePk_DeltaImport_DeltaImportQuery() throws Exception {
-    List deltaRow = new ArrayList();
-    deltaRow.add(createMap("id", "5"));
-    MockDataSource.setIterator("select id from x where last_modified > NOW",
-            deltaRow.iterator());
-
-    List parentRow = new ArrayList();
-    parentRow.add(createMap("id", "5"));
-    MockDataSource.setIterator("select * from x where id=5", parentRow
-            .iterator());
-
-    List childRow = new ArrayList();
-    childRow.add(createMap("desc", "hello"));
-    MockDataSource.setIterator("select * from y where y.A=5", childRow
-            .iterator());
-
-    runDeltaImport(dataConfig_deltaimportquery);
-
-    assertQ(req("id:5"), "//*[@numFound='1']");
-    assertQ(req("desc:hello"), "//*[@numFound='1']");
-  }
-
-  @Test
-  @SuppressWarnings("unchecked")
   @Ignore("Known Locale/TZ problems: see https://issues.apache.org/jira/browse/SOLR-1916")
+  /**
+   * This test is here for historical purposes only.  
+   * When SOLR-1916 is fixed, it would be best to rewrite this test.
+   * 
+   * @throws Exception
+   */
   public void testLastIndexTime() throws Exception  {
     List row = new ArrayList();
     row.add(createMap("id", 5));
@@ -196,15 +71,15 @@ public class TestSqlEntityProcessor2 ext
   static class DateFormatValidatingEvaluator extends Evaluator {
     @Override
     public String evaluate(String expression, Context context) {
-      List l = EvaluatorBag.parseParams(expression, context.getVariableResolver());
+      List l = new DateFormatEvaluator().parseParams(expression, context.getVariableResolver());
       Object o = l.get(0);
       String dateStr = null;
-      if (o instanceof EvaluatorBag.VariableWrapper) {
-        EvaluatorBag.VariableWrapper wrapper = (EvaluatorBag.VariableWrapper) o;
+      if (o instanceof Evaluator.VariableWrapper) {
+        Evaluator.VariableWrapper wrapper = (Evaluator.VariableWrapper) o;
         o = wrapper.resolve();
         dateStr = o.toString();
       }
-      SimpleDateFormat formatter = DataImporter.DATE_TIME_FORMAT.get();
+      SimpleDateFormat formatter = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.ROOT);
       try {
         formatter.parse(dateStr);
       } catch (ParseException e) {
@@ -220,22 +95,4 @@ public class TestSqlEntityProcessor2 ext
           "\t\t<entity name=\"x\" query=\"select * from x where last_modified > ${dih.functions.checkDateFormat(dih.last_index_time)}\" />\n" +
           "\t</document>\n" +
           "</dataConfig>";
-
-  private static String dataConfig = "<dataConfig><dataSource  type=\"MockDataSource\"/>\n"
-          + "       <document>\n"
-          + "               <entity name=\"x\" pk=\"id\" query=\"select * from x\" deletedPkQuery=\"select id from x where last_modified > NOW AND deleted='true'\" deltaQuery=\"select id from x where last_modified > NOW\">\n"
-          + "                       <field column=\"id\" />\n"
-          + "                       <entity name=\"y\" query=\"select * from y where y.A=${x.id}\">\n"
-          + "                               <field column=\"desc\" />\n"
-          + "                       </entity>\n" + "               </entity>\n"
-          + "       </document>\n" + "</dataConfig>\n";
-
-  private static String dataConfig_deltaimportquery = "<dataConfig><dataSource  type=\"MockDataSource\"/>\n"
-          + "       <document>\n"
-          + "               <entity name=\"x\" deltaImportQuery=\"select * from x where id=${dataimporter.delta.id}\" deltaQuery=\"select id from x where last_modified > NOW\">\n"
-          + "                       <field column=\"id\" />\n"
-          + "                       <entity name=\"y\" query=\"select * from y where y.A=${x.id}\">\n"
-          + "                               <field column=\"desc\" />\n"
-          + "                       </entity>\n" + "               </entity>\n"
-          + "       </document>\n" + "</dataConfig>\n";
 }