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

svn commit: r1411276 [1/2] - in /lucene/dev/trunk/solr: ./ contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/

Author: jdyer
Date: Mon Nov 19 16:23:50 2012
New Revision: 1411276

URL: http://svn.apache.org/viewvc?rev=1411276&view=rev
Log:
SOLR-4086: Refactor DIH - VariableResolver & Evaluator

Added:
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DateFormatEvaluator.java   (with props)
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrQueryEscapingEvaluator.java   (with props)
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SqlEscapingEvaluator.java   (with props)
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/UrlEvaluator.java   (with props)
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestBuiltInEvaluators.java   (with props)
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestNestedChildren.java   (with props)
Removed:
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/EvaluatorBag.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/TemplateString.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/VariableResolverImpl.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestEvaluatorBag.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestTemplateString.java
Modified:
    lucene/dev/trunk/solr/CHANGES.txt
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ContextImpl.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DataImporter.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DocBuilder.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/EntityProcessorWrapper.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/Evaluator.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/FileListEntityProcessor.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/TemplateTransformer.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/VariableResolver.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHCacheTestCase.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDataImportHandlerTestCase.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestClobTransformer.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContextImpl.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDateFormatTransformer.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestLineEntityProcessor.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestNumberFormatTransformer.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestRegexTransformer.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSqlEntityProcessor2.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestTemplateTransformer.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestURLDataSource.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestVariableResolver.java
    lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestXPathEntityProcessor.java

Modified: lucene/dev/trunk/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/CHANGES.txt?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/CHANGES.txt (original)
+++ lucene/dev/trunk/solr/CHANGES.txt Mon Nov 19 16:23:50 2012
@@ -230,6 +230,11 @@ Other Changes
 
 * SOLR-4067: ZkStateReader#getLeaderProps should not return props for a leader
   that it does not think is live. (Mark Miller)
+  
+* SOLR-4086: DIH refactor of VariableResolver and Evaluator.  VariableResolver
+  and each built-in Evaluator are separate concrete classes.  DateFormatEvaluator
+  now defaults with the ROOT Locale. However, users may specify a different 
+  Locale using an optional new third parameter. (James Dyer) 
 
 ==================  4.0.0 ==================
 

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ContextImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ContextImpl.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ContextImpl.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ContextImpl.java Mon Nov 19 16:23:50 2012
@@ -38,7 +38,7 @@ public class ContextImpl extends Context
 
   private ContextImpl parent;
 
-  private VariableResolverImpl resolver;
+  private VariableResolver resolver;
 
   private DataSource ds;
 
@@ -55,7 +55,7 @@ public class ContextImpl extends Context
   DocBuilder docBuilder;
 
 
-  public ContextImpl(EntityProcessorWrapper epw, VariableResolverImpl resolver,
+  public ContextImpl(EntityProcessorWrapper epw, VariableResolver resolver,
                      DataSource ds, String currProcess,
                      Map<String, Object> global, ContextImpl parentContext, DocBuilder docBuilder) {
     this.epw = epw;

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DataImporter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DataImporter.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DataImporter.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DataImporter.java Mon Nov 19 16:23:50 2012
@@ -34,6 +34,10 @@ import org.apache.solr.handler.dataimpor
 
 import static org.apache.solr.handler.dataimport.DataImportHandlerException.wrapAndThrow;
 import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
+import static org.apache.solr.handler.dataimport.DocBuilder.loadClass;
+import static org.apache.solr.handler.dataimport.config.ConfigNameConstants.CLASS;
+import static org.apache.solr.handler.dataimport.config.ConfigNameConstants.NAME;
+
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 import org.w3c.dom.Document;
@@ -511,6 +515,30 @@ public class DataImporter {
   DocBuilder getDocBuilder() {
     return docBuilder;
   }
+  
+  Map<String, Evaluator> getEvaluators() {
+    return getEvaluators(config.getFunctions());
+  }
+  
+  /**
+   * used by tests.
+   */
+  Map<String, Evaluator> getEvaluators(List<Map<String,String>> fn) {
+    Map<String, Evaluator> evaluators = new HashMap<String, Evaluator>();
+    evaluators.put(Evaluator.DATE_FORMAT_EVALUATOR, new DateFormatEvaluator());
+    evaluators.put(Evaluator.SQL_ESCAPE_EVALUATOR, new SqlEscapingEvaluator());
+    evaluators.put(Evaluator.URL_ENCODE_EVALUATOR, new UrlEvaluator());
+    evaluators.put(Evaluator.ESCAPE_SOLR_QUERY_CHARS, new SolrQueryEscapingEvaluator());
+    SolrCore core = docBuilder == null ? null : docBuilder.dataImporter.getCore();
+    for (Map<String, String> map : fn) {
+      try {
+        evaluators.put(map.get(NAME), (Evaluator) loadClass(map.get(CLASS), core).newInstance());
+      } catch (Exception e) {
+        wrapAndThrow(SEVERE, e, "Unable to instantiate evaluator: " + map.get(CLASS));
+      }
+    }
+    return evaluators;    
+  }
 
   static final ThreadLocal<AtomicLong> QUERY_COUNT = new ThreadLocal<AtomicLong>() {
     @Override
@@ -519,12 +547,7 @@ public class DataImporter {
     }
   };
 
-  static final ThreadLocal<SimpleDateFormat> DATE_TIME_FORMAT = new ThreadLocal<SimpleDateFormat>() {
-    @Override
-    protected SimpleDateFormat initialValue() {
-      return new SimpleDateFormat("yyyy-MM-dd HH:mm:ss");
-    }
-  };
+  
 
   static final class MSG {
     public static final String NO_CONFIG_FOUND = "Configuration not found";
@@ -598,4 +621,5 @@ public class DataImporter {
   public static final String RELOAD_CONF_CMD = "reload-config";
 
   public static final String SHOW_CONF_CMD = "show-config";
+  
 }

Added: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DateFormatEvaluator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DateFormatEvaluator.java?rev=1411276&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DateFormatEvaluator.java (added)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DateFormatEvaluator.java Mon Nov 19 16:23:50 2012
@@ -0,0 +1,147 @@
+package org.apache.solr.handler.dataimport;
+
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.wrapAndThrow;
+
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Locale;
+import java.util.Map;
+import java.util.TimeZone;
+import java.util.WeakHashMap;
+
+import org.apache.solr.handler.dataimport.config.EntityField;
+import org.apache.solr.util.DateMathParser;
+
+/*
+ * 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.
+ */
+
+/**
+ * <p>Formats values using a given date format. </p>
+ * <p>Pass three parameters:
+ * <ul>
+ *  <li>An {@link EntityField} or a date expression to be parsed with 
+ *      the {@link DateMathParser} class  If the value is in a String, 
+ *      then it is assumed to be a datemath expression, otherwise it 
+ *      resolved using a {@link VariableResolver} instance</li>
+ *  <li>A date format see {@link SimpleDateFormat} for the syntax.</li>
+ *  <li>The {@link Locale} to parse.  
+ *      (optional. Defaults to the Root Locale) </li>
+ * </ul>
+ * </p>
+ */
+public class DateFormatEvaluator extends Evaluator {
+  
+  public static final String DEFAULT_DATE_FORMAT = "yyyy-MM-dd HH:mm:ss";
+  Map<DateFormatCacheKey, SimpleDateFormat> cache = new WeakHashMap<DateFormatCacheKey, SimpleDateFormat>();
+  Map<String, Locale> availableLocales = new HashMap<String, Locale>();
+  
+  class DateFormatCacheKey {
+    DateFormatCacheKey(Locale l, String df) {
+      this.locale = l;
+      this.dateFormat = df;
+    }
+    Locale locale;
+    String dateFormat;
+  }
+  
+  public DateFormatEvaluator() {  
+    for (Locale locale : Locale.getAvailableLocales()) {
+      availableLocales.put(locale.toString(), locale);
+    }
+  }
+  
+  
+  @Override
+  public String evaluate(String expression, Context context) {
+    List<Object> l = parseParams(expression, context.getVariableResolver());
+    if (l.size() < 2 || l.size() > 3) {
+      throw new DataImportHandlerException(SEVERE, "'formatDate()' must have two or three parameters ");
+    }
+    Object o = l.get(0);
+    Object format = l.get(1);
+    if (format instanceof VariableWrapper) {
+      VariableWrapper wrapper = (VariableWrapper) format;
+      o = wrapper.resolve();
+      format = o.toString();
+    }
+    Locale locale = Locale.ROOT;
+    if(l.size()==3) {
+      Object localeObj = l.get(2);
+      String localeStr = null;
+      if (localeObj  instanceof VariableWrapper) {
+        VariableWrapper wrapper = (VariableWrapper) localeObj;
+        o = wrapper.resolve();
+        localeStr = o.toString();
+      } else {
+        localeStr = localeObj.toString();
+      }
+      availableLocales.get(localeStr);
+      if(locale==null) {
+        throw new DataImportHandlerException(SEVERE, "Unsupported locale: " + localeStr);
+      }
+    }
+    String dateFmt = format.toString();
+    DateFormatCacheKey dfck = new DateFormatCacheKey(locale, dateFmt);
+    SimpleDateFormat sdf = cache.get(dfck);
+    if(sdf==null) {
+      sdf = new SimpleDateFormat(dateFmt, locale);
+      cache.put(dfck, sdf);
+    }
+    Date date = null;
+    if (o instanceof VariableWrapper) {
+      VariableWrapper variableWrapper = (VariableWrapper) o;
+      Object variableval = variableWrapper.resolve();
+      if (variableval instanceof Date) {
+        date = (Date) variableval;
+      } else {
+        String s = variableval.toString();
+        try {
+          dfck = new DateFormatCacheKey(locale, DEFAULT_DATE_FORMAT);
+          sdf = cache.get(dfck);
+          if(sdf==null) {
+            sdf = new SimpleDateFormat(dfck.dateFormat, dfck.locale);
+            cache.put(dfck, sdf);
+          }
+          date = new SimpleDateFormat(DEFAULT_DATE_FORMAT, locale).parse(s);
+        } catch (ParseException exp) {
+          wrapAndThrow(SEVERE, exp, "Invalid expression for date");
+        }
+      }
+    } else {
+      String datemathfmt = o.toString();
+      datemathfmt = datemathfmt.replaceAll("NOW", "");
+      try {
+        date = getDateMathParser().parseMath(datemathfmt);
+      } catch (ParseException e) {
+        wrapAndThrow(SEVERE, e, "Invalid expression for date");
+      }
+    }
+    return sdf.format(date);
+  }
+  static DateMathParser getDateMathParser() {
+    return new DateMathParser(TimeZone.getDefault(), Locale.getDefault()) {
+      @Override
+      public Date getNow() {
+        return new Date();
+      }
+    };
+  }
+}

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DocBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DocBuilder.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DocBuilder.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DocBuilder.java Mon Nov 19 16:23:50 2012
@@ -31,6 +31,7 @@ import org.apache.solr.schema.SchemaFiel
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.text.SimpleDateFormat;
 import java.util.*;
 import java.util.concurrent.atomic.AtomicBoolean;
 import java.util.concurrent.atomic.AtomicLong;
@@ -67,7 +68,7 @@ public class DocBuilder {
   Map<String, Object> session = new HashMap<String, Object>();
 
   static final ThreadLocal<DocBuilder> INSTANCE = new ThreadLocal<DocBuilder>();
-  private Map<String, Object> functionsNamespace;
+  //private Map<String, Object> functionsNamespace;
   private Map<String, Object> persistedProperties;
   
   private DIHProperties propWriter;
@@ -85,8 +86,7 @@ public class DocBuilder {
     DataImporter.QUERY_COUNT.set(importStatistics.queryCount);
     verboseDebug = reqParams.isDebug() && reqParams.getDebugInfo().verbose;
     persistedProperties = propWriter.readIndexerProperties();
-    functionsNamespace = EvaluatorBag.getFunctionsNamespace(this.dataImporter.getConfig().getFunctions(), this, getVariableResolver());
-    
+     
     String writerClassStr = null;
     if(reqParams!=null && reqParams.getRawParams() != null) {
       writerClassStr = (String) reqParams.getRawParams().get(PARAM_WRITER_IMPL);
@@ -113,13 +113,16 @@ public class DocBuilder {
     return debugLogger;
   }
 
-  public VariableResolverImpl getVariableResolver() {
+  private VariableResolver getVariableResolver() {
     try {
-      VariableResolverImpl resolver = null;
+      VariableResolver resolver = null;
       if(dataImporter != null && dataImporter.getCore() != null
           && dataImporter.getCore().getResourceLoader().getCoreProperties() != null){
-        resolver =  new VariableResolverImpl(dataImporter.getCore().getResourceLoader().getCoreProperties());
-      } else resolver = new VariableResolverImpl();
+        resolver =  new VariableResolver(dataImporter.getCore().getResourceLoader().getCoreProperties());
+      } else {
+        resolver = new VariableResolver();
+      }
+      resolver.setEvaluators(dataImporter.getEvaluators());
       Map<String, Object> indexerNamespace = new HashMap<String, Object>();
       if (persistedProperties.get(LAST_INDEX_TIME) != null) {
         indexerNamespace.put(LAST_INDEX_TIME, persistedProperties.get(LAST_INDEX_TIME));
@@ -129,7 +132,6 @@ public class DocBuilder {
       }
       indexerNamespace.put(INDEX_START_TIME, dataImporter.getIndexStartTime());
       indexerNamespace.put("request", reqParams.getRawParams());
-      indexerNamespace.put("functions", functionsNamespace);
       for (Entity entity : dataImporter.getConfig().getEntities()) {
         String key = entity.getName() + "." + SolrWriter.LAST_INDEX_KEY;
         Object lastIndex = persistedProperties.get(key);
@@ -149,12 +151,6 @@ public class DocBuilder {
     }
   }
   
-  private Map<String,Object> getFunctionsNamespace() {
-    if(functionsNamespace==null) {
-      
-    }
-    return functionsNamespace;
-  }
 
   private void invokeEventListener(String className) {
     try {
@@ -241,7 +237,7 @@ public class DocBuilder {
 
       if (stop.get()) {
         // Dont commit if aborted using command=abort
-        statusMessages.put("Aborted", DataImporter.DATE_TIME_FORMAT.get().format(new Date()));
+        statusMessages.put("Aborted", new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.ROOT).format(new Date()));
         rollback();
       } else {
         // Do not commit unnecessarily if this is a delta-import and no documents were created or deleted
@@ -327,7 +323,7 @@ public class DocBuilder {
   @SuppressWarnings("unchecked")
   private void doDelta() {
     addStatusMessage("Delta Dump started");
-    VariableResolverImpl resolver = getVariableResolver();
+    VariableResolver resolver = getVariableResolver();
 
     if (config.getDeleteQuery() != null) {
       writer.deleteByQuery(config.getDeleteQuery());
@@ -350,7 +346,7 @@ public class DocBuilder {
     writer.setDeltaKeys(allPks);
 
     statusMessages.put("Total Changed Documents", allPks.size());
-    VariableResolverImpl vri = getVariableResolver();
+    VariableResolver vri = getVariableResolver();
     Iterator<Map<String, Object>> pkIter = allPks.iterator();
     while (pkIter.hasNext()) {
       Map<String, Object> map = pkIter.next();
@@ -390,7 +386,7 @@ public class DocBuilder {
   
   @SuppressWarnings("unchecked")
   public void addStatusMessage(String msg) {
-    statusMessages.put(msg, DataImporter.DATE_TIME_FORMAT.get().format(new Date()));
+    statusMessages.put(msg, new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.ROOT).format(new Date()));
   }
 
   private void resetEntity(EntityProcessorWrapper epw) {
@@ -401,7 +397,7 @@ public class DocBuilder {
     
   }
   
-  private void buildDocument(VariableResolverImpl vr, DocWrapper doc,
+  private void buildDocument(VariableResolver vr, DocWrapper doc,
       Map<String,Object> pk, EntityProcessorWrapper epw, boolean isRoot,
       ContextImpl parentCtx) {
     List<EntityProcessorWrapper> entitiesToDestroy = new ArrayList<EntityProcessorWrapper>();
@@ -418,7 +414,7 @@ public class DocBuilder {
   }
 
   @SuppressWarnings("unchecked")
-  private void buildDocument(VariableResolverImpl vr, DocWrapper doc,
+  private void buildDocument(VariableResolver vr, DocWrapper doc,
                              Map<String, Object> pk, EntityProcessorWrapper epw, boolean isRoot,
                              ContextImpl parentCtx, List<EntityProcessorWrapper> entitiesToDestroy) {
 
@@ -734,7 +730,7 @@ public class DocBuilder {
    * @return an iterator to the list of keys for which Solr documents should be updated.
    */
   @SuppressWarnings("unchecked")
-  public Set<Map<String, Object>> collectDelta(EntityProcessorWrapper epw, VariableResolverImpl resolver,
+  public Set<Map<String, Object>> collectDelta(EntityProcessorWrapper epw, VariableResolver resolver,
                                                Set<Map<String, Object>> deletedRows) {
     //someone called abort
     if (stop.get())
@@ -816,14 +812,16 @@ public class DocBuilder {
       // identifying deleted rows with deltas
 
       for (Map<String, Object> row : myModifiedPks) {
-        getModifiedParentRows(resolver.addNamespace(epw.getEntity().getName(), row), epw.getEntity().getName(), epw, parentKeyList);
+        resolver.addNamespace(epw.getEntity().getName(), row);
+        getModifiedParentRows(resolver, epw.getEntity().getName(), epw, parentKeyList);
         // check for abort
         if (stop.get())
           return new HashSet();
       }
       // running the same for deletedrows
       for (Map<String, Object> row : deletedSet) {
-        getModifiedParentRows(resolver.addNamespace(epw.getEntity().getName(), row), epw.getEntity().getName(), epw, parentKeyList);
+        resolver.addNamespace(epw.getEntity().getName(), row);
+        getModifiedParentRows(resolver, epw.getEntity().getName(), epw, parentKeyList);
         // check for abort
         if (stop.get())
           return new HashSet();
@@ -838,7 +836,7 @@ public class DocBuilder {
         myModifiedPks : new HashSet<Map<String, Object>>(parentKeyList);
   }
 
-  private void getModifiedParentRows(VariableResolverImpl resolver,
+  private void getModifiedParentRows(VariableResolver resolver,
                                      String entity, EntityProcessor entityProcessor,
                                      Set<Map<String, Object>> parentKeyList) {
     try {

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/EntityProcessorWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/EntityProcessorWrapper.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/EntityProcessorWrapper.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/EntityProcessorWrapper.java Mon Nov 19 16:23:50 2012
@@ -48,7 +48,7 @@ public class EntityProcessorWrapper exte
   private boolean initalized;
   private String onError;
   private Context context;
-  private VariableResolverImpl resolver;
+  private VariableResolver resolver;
   private String entityName;
 
   protected List<Transformer> transformers;
@@ -65,7 +65,7 @@ public class EntityProcessorWrapper exte
   public void init(Context context) {
     rowcache = null;
     this.context = context;
-    resolver = (VariableResolverImpl) context.getVariableResolver();
+    resolver = (VariableResolver) context.getVariableResolver();
     if (entityName == null) {
       onError = resolver.replaceTokens(context.getEntityAttribute(ON_ERROR));
       if (onError == null) onError = ABORT;
@@ -171,7 +171,7 @@ public class EntityProcessorWrapper exte
     Map<String, Object> transformedRow = row;
     List<Map<String, Object>> rows = null;
     boolean stopTransform = checkStopTransform(row);
-    VariableResolverImpl resolver = (VariableResolverImpl) context.getVariableResolver();
+    VariableResolver resolver = (VariableResolver) context.getVariableResolver();
     for (Transformer t : transformers) {
       if (stopTransform) break;
       try {
@@ -288,8 +288,8 @@ public class EntityProcessorWrapper exte
     delegate.destroy();
   }
 
-  public VariableResolverImpl getVariableResolver() {
-    return (VariableResolverImpl) context.getVariableResolver();
+  public VariableResolver getVariableResolver() {
+    return (VariableResolver) context.getVariableResolver();
   }
 
   public Context getContext() {

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/Evaluator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/Evaluator.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/Evaluator.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/Evaluator.java Mon Nov 19 16:23:50 2012
@@ -16,6 +16,18 @@
  */
 package org.apache.solr.handler.dataimport;
 
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.wrapAndThrow;
+
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+import java.util.Locale;
+import java.util.TimeZone;
+import java.util.regex.Pattern;
+
+import org.apache.solr.util.DateMathParser;
+
 /**
  * <p>
  * Pluggable functions for resolving variables
@@ -43,4 +55,88 @@ public abstract class Evaluator {
    * @return the value of the given expression evaluated using the resolver
    */
   public abstract String evaluate(String expression, Context context);
+  
+  /**
+   * Parses a string of expression into separate params. The values are separated by commas. each value will be
+   * translated into one of the following:
+   * &lt;ol&gt;
+   * &lt;li&gt;If it is in single quotes the value will be translated to a String&lt;/li&gt;
+   * &lt;li&gt;If is is not in quotes and is a number a it will be translated into a Double&lt;/li&gt;
+   * &lt;li&gt;else it is a variable which can be resolved and it will be put in as an instance of VariableWrapper&lt;/li&gt;
+   * &lt;/ol&gt;
+   *
+   * @param expression the expression to be parsed
+   * @param vr the VariableResolver instance for resolving variables
+   *
+   * @return a List of objects which can either be a string, number or a variable wrapper
+   */
+  List<Object> parseParams(String expression, VariableResolver vr) {
+    List<Object> result = new ArrayList<Object>();
+    expression = expression.trim();
+    String[] ss = expression.split(",");
+    for (int i = 0; i < ss.length; i++) {
+      ss[i] = ss[i].trim();
+      if (ss[i].startsWith("'")) {//a string param has started
+        StringBuilder sb = new StringBuilder();
+        while (true) {
+          sb.append(ss[i]);
+          if (ss[i].endsWith("'")) break;
+          i++;
+          if (i >= ss.length)
+            throw new DataImportHandlerException(SEVERE, "invalid string at " + ss[i - 1] + " in function params: " + expression);
+          sb.append(",");
+        }
+        String s = sb.substring(1, sb.length() - 1);
+        s = s.replaceAll("\\\\'", "'");
+        result.add(s);
+      } else {
+        if (Character.isDigit(ss[i].charAt(0))) {
+          try {
+            Double doub = Double.parseDouble(ss[i]);
+            result.add(doub);
+          } catch (NumberFormatException e) {
+            if (vr.resolve(ss[i]) == null) {
+              wrapAndThrow(
+                      SEVERE, e, "Invalid number :" + ss[i] +
+                              "in parameters  " + expression);
+            }
+          }
+        } else {
+          result.add(new VariableWrapper(ss[i], vr));
+        }
+      }
+    }
+    return result;
+  }
+
+  static class VariableWrapper {
+    String varName;
+    VariableResolver vr;
+
+    public VariableWrapper(String s, VariableResolver vr) {
+      this.varName = s;
+      this.vr = vr;
+    }
+
+    public Object resolve() {
+      return vr.resolve(varName);
+
+    }
+
+    @Override
+    public String toString() {
+      Object o = vr.resolve(varName);
+      return o == null ? null : o.toString();
+    }
+  }
+
+  static Pattern IN_SINGLE_QUOTES = Pattern.compile("^'(.*?)'$");
+  
+  public static final String DATE_FORMAT_EVALUATOR = "formatDate";
+
+  public static final String URL_ENCODE_EVALUATOR = "encodeUrl";
+
+  public static final String ESCAPE_SOLR_QUERY_CHARS = "escapeQueryChars";
+
+  public static final String SQL_ESCAPE_EVALUATOR = "escapeSql";
 }

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/FileListEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/FileListEntityProcessor.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/FileListEntityProcessor.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/FileListEntityProcessor.java Mon Nov 19 16:23:50 2012
@@ -19,6 +19,7 @@ package org.apache.solr.handler.dataimpo
 import java.io.File;
 import java.io.FilenameFilter;
 import java.text.ParseException;
+import java.text.SimpleDateFormat;
 import java.util.*;
 import java.util.regex.Matcher;
 import java.util.regex.Pattern;
@@ -151,19 +152,19 @@ public class FileListEntityProcessor ext
     } else  {
       dateStr = context.replaceTokens(dateStr);
     }
-    m = EvaluatorBag.IN_SINGLE_QUOTES.matcher(dateStr);
+    m = Evaluator.IN_SINGLE_QUOTES.matcher(dateStr);
     if (m.find()) {
       String expr = null;
       expr = m.group(1).replaceAll("NOW", "");
       try {
-        return EvaluatorBag.dateMathParser.parseMath(expr);
+        return DateFormatEvaluator.getDateMathParser().parseMath(expr);
       } catch (ParseException exp) {
         throw new DataImportHandlerException(DataImportHandlerException.SEVERE,
                 "Invalid expression for date", exp);
       }
     }
     try {
-      return DataImporter.DATE_TIME_FORMAT.get().parse(dateStr);
+      return new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.ROOT).parse(dateStr);
     } catch (ParseException exp) {
       throw new DataImportHandlerException(DataImportHandlerException.SEVERE,
               "Invalid expression for date", exp);

Added: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrQueryEscapingEvaluator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrQueryEscapingEvaluator.java?rev=1411276&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrQueryEscapingEvaluator.java (added)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrQueryEscapingEvaluator.java Mon Nov 19 16:23:50 2012
@@ -0,0 +1,35 @@
+package org.apache.solr.handler.dataimport;
+
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
+
+import java.util.List;
+
+import org.apache.solr.client.solrj.util.ClientUtils;
+
+/*
+ * 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.
+ */
+
+public class SolrQueryEscapingEvaluator extends Evaluator {
+  public String evaluate(String expression, Context context) {
+    List<Object> l = parseParams(expression, context.getVariableResolver());
+    if (l.size() != 1) {
+      throw new DataImportHandlerException(SEVERE, "'escapeQueryChars' must have at least one parameter ");
+    }
+    String s = l.get(0).toString();
+    return ClientUtils.escapeQueryChars(s);
+  }
+}

Added: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SqlEscapingEvaluator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SqlEscapingEvaluator.java?rev=1411276&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SqlEscapingEvaluator.java (added)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SqlEscapingEvaluator.java Mon Nov 19 16:23:50 2012
@@ -0,0 +1,42 @@
+package org.apache.solr.handler.dataimport;
+
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
+
+import java.util.List;
+
+/*
+ * 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.
+ */
+
+/**
+ * <p> Escapes values in SQL queries.  It escapes the value of the given expression 
+ * by replacing all occurrences of single-quotes by two single-quotes and similarily 
+ * for double-quotes </p>
+ */
+public class SqlEscapingEvaluator extends Evaluator {
+  @Override
+  public String evaluate(String expression, Context context) {
+    List<Object> l = parseParams(expression, context.getVariableResolver());
+    if (l.size() != 1) {
+      throw new DataImportHandlerException(SEVERE, "'escapeSql' must have at least one parameter ");
+    }
+    String s = l.get(0).toString();
+    // escape single quote with two single quotes, double quote
+    // with two doule quotes, and backslash with double backslash.
+    // See:  http://dev.mysql.com/doc/refman/4.1/en/mysql-real-escape-string.html
+    return s.replaceAll("'", "''").replaceAll("\"", "\"\"").replaceAll("\\\\", "\\\\\\\\");
+  }
+}

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/TemplateTransformer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/TemplateTransformer.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/TemplateTransformer.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/TemplateTransformer.java Mon Nov 19 16:23:50 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";
 }

Added: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/UrlEvaluator.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/UrlEvaluator.java?rev=1411276&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/UrlEvaluator.java (added)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/UrlEvaluator.java Mon Nov 19 16:23:50 2012
@@ -0,0 +1,47 @@
+package org.apache.solr.handler.dataimport;
+
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.wrapAndThrow;
+
+import java.net.URLEncoder;
+import java.util.List;
+
+/*
+ * 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.
+ */
+
+/**
+ * <p>Escapes reserved characters in Solr queries</p>
+ *
+ * @see org.apache.solr.client.solrj.util.ClientUtils#escapeQueryChars(String)
+ */
+public class UrlEvaluator extends Evaluator {
+  @Override
+  public String evaluate(String expression, Context context) {
+    List<Object> l = parseParams(expression, context.getVariableResolver());
+    if (l.size() != 1) {
+      throw new DataImportHandlerException(SEVERE, "'encodeUrl' must have at least one parameter ");
+    }
+    String s = l.get(0).toString();
+
+    try {
+      return URLEncoder.encode(s.toString(), "UTF-8");
+    } catch (Exception e) {
+      wrapAndThrow(SEVERE, e, "Unable to encode expression: " + expression + " with value: " + s);
+      return null;
+    }
+  }
+}

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/VariableResolver.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/VariableResolver.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/VariableResolver.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/VariableResolver.java Mon Nov 19 16:23:50 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/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java Mon Nov 19 16:23:50 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/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHCacheTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHCacheTestCase.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHCacheTestCase.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDIHCacheTestCase.java Mon Nov 19 16:23:50 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/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDataImportHandlerTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDataImportHandlerTestCase.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDataImportHandlerTestCase.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/AbstractDataImportHandlerTestCase.java Mon Nov 19 16:23:50 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);

Added: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestBuiltInEvaluators.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestBuiltInEvaluators.java?rev=1411276&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestBuiltInEvaluators.java (added)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestBuiltInEvaluators.java Mon Nov 19 16:23:50 2012
@@ -0,0 +1,154 @@
+/*
+ * 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.handler.dataimport;
+
+import org.junit.Before;
+import org.junit.Test;
+
+import java.net.URLEncoder;
+import java.text.SimpleDateFormat;
+import java.util.*;
+
+/**
+ * <p> Test for EvaluatorBag </p>
+ *
+ *
+ * @since solr 1.3
+ */
+public class TestBuiltInEvaluators extends AbstractDataImportHandlerTestCase {
+  private static final String ENCODING = "UTF-8";
+
+  VariableResolver resolver;
+
+  Map<String, String> sqlTests;
+
+  Map<String, String> urlTests;
+
+  @Override
+  @Before
+  public void setUp() throws Exception {
+    super.setUp();
+    resolver = new VariableResolver();
+
+    sqlTests = new HashMap<String, String>();
+
+    sqlTests.put("foo\"", "foo\"\"");
+    sqlTests.put("foo\\", "foo\\\\");
+    sqlTests.put("foo'", "foo''");
+    sqlTests.put("foo''", "foo''''");
+    sqlTests.put("'foo\"", "''foo\"\"");
+    sqlTests.put("\"Albert D'souza\"", "\"\"Albert D''souza\"\"");
+
+    urlTests = new HashMap<String, String>();
+
+    urlTests.put("*:*", URLEncoder.encode("*:*", ENCODING));
+    urlTests.put("price:[* TO 200]", URLEncoder.encode("price:[* TO 200]",
+            ENCODING));
+    urlTests.put("review:\"hybrid sedan\"", URLEncoder.encode(
+            "review:\"hybrid sedan\"", ENCODING));
+  }
+
+  
+  @Test
+  public void testSqlEscapingEvaluator() {
+    Evaluator sqlEscaper = new SqlEscapingEvaluator();
+    runTests(sqlTests, sqlEscaper);
+  }
+
+  
+  @Test
+  public void testUrlEvaluator() throws Exception {
+    Evaluator urlEvaluator = new UrlEvaluator();
+    runTests(urlTests, urlEvaluator);
+  }
+
+  @Test
+  public void parseParams() {
+    Map<String,Object> m = new HashMap<String,Object>();
+    m.put("b","B");
+    VariableResolver vr = new VariableResolver();
+    vr.addNamespace("a",m);
+    List<Object> l = (new Evaluator() {      
+      @Override
+      public String evaluate(String expression, Context context) {
+        return null;
+      }
+    }).parseParams(" 1 , a.b, 'hello!', 'ds,o,u\'za',",vr);
+    assertEquals(new Double(1),l.get(0));
+    assertEquals("B",((Evaluator.VariableWrapper)l.get(1)).resolve());
+    assertEquals("hello!",l.get(2));
+    assertEquals("ds,o,u'za",l.get(3));
+  }
+
+  @Test
+  public void testEscapeSolrQueryFunction() {
+    final VariableResolver resolver = new VariableResolver();    
+    Map<String,Object> m= new HashMap<String,Object>();
+    m.put("query","c:t");
+    resolver.setEvaluators(new DataImporter().getEvaluators(Collections.<Map<String,String>>emptyList()));
+    
+    resolver.addNamespace("e",m);
+    String s = resolver
+            .replaceTokens("${dataimporter.functions.escapeQueryChars(e.query)}");
+    org.junit.Assert.assertEquals("c\\:t", s);
+    
+  }
+
+  
+  @Test
+  public void testDateFormatEvaluator() {
+    Evaluator dateFormatEval = new DateFormatEvaluator();
+    ContextImpl context = new ContextImpl(null, resolver, null, Context.FULL_DUMP, Collections.<String, Object>emptyMap(), null, null);
+    
+    Calendar calendar = new GregorianCalendar();
+    calendar.add(Calendar.DAY_OF_YEAR, -2);
+    
+    String currentLocale = Locale.getDefault().toString();
+
+    assertEquals(new SimpleDateFormat("yyyy-MM-dd HH:mm", Locale.ROOT).format(calendar.getTime()),
+            dateFormatEval.evaluate("'NOW-2DAYS','yyyy-MM-dd HH:mm'", context));    
+    assertEquals(new SimpleDateFormat("yyyy-MM-dd HH:mm", Locale.getDefault()).format(calendar.getTime()),
+        dateFormatEval.evaluate("'NOW-2DAYS','yyyy-MM-dd HH:mm','" + currentLocale + "'", context));
+    
+    calendar = new GregorianCalendar();
+    Date date = calendar.getTime();
+    
+    Map<String, Object> map = new HashMap<String, Object>();
+    map.put("key", date);
+    resolver.addNamespace("A", map);
+
+    assertEquals(new SimpleDateFormat("yyyy-MM-dd HH:mm", Locale.ROOT).format(date),
+            dateFormatEval.evaluate("A.key, 'yyyy-MM-dd HH:mm'", context));
+    assertEquals(new SimpleDateFormat("yyyy-MM-dd HH:mm", Locale.getDefault()).format(date),
+       dateFormatEval.evaluate("A.key, 'yyyy-MM-dd HH:mm','" + currentLocale + "'", context));
+   
+  }
+
+  private void runTests(Map<String, String> tests, Evaluator evaluator) {
+    ContextImpl ctx = new ContextImpl(null, resolver, null, Context.FULL_DUMP, Collections.<String, Object>emptyMap(), null, null);    
+    for (Map.Entry<String, String> entry : tests.entrySet()) {
+      Map<String, Object> values = new HashMap<String, Object>();
+      values.put("key", entry.getKey());
+      resolver.addNamespace("A", values);
+
+      String expected = entry.getValue();
+      String actual = evaluator.evaluate("A.key", ctx);
+      assertEquals(expected, actual);
+    }
+    
+  }
+}

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestClobTransformer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestClobTransformer.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestClobTransformer.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestClobTransformer.java Mon Nov 19 16:23:50 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/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContextImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContextImpl.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContextImpl.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestContextImpl.java Mon Nov 19 16:23:50 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/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDateFormatTransformer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDateFormatTransformer.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDateFormatTransformer.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDateFormatTransformer.java Mon Nov 19 16:23:50 2012
@@ -45,7 +45,7 @@ public class TestDateFormatTransformer e
 
     Map row = createMap("lastModified", format.format(now));
 
-    VariableResolverImpl resolver = new VariableResolverImpl();
+    VariableResolver resolver = new VariableResolver();
     resolver.addNamespace("e", row);
 
     Context context = getContext(null, resolver,
@@ -73,7 +73,7 @@ public class TestDateFormatTransformer e
     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,

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFileListEntityProcessor.java Mon Nov 19 16:23:50 2012
@@ -21,6 +21,7 @@ import org.junit.Test;
 
 import java.io.File;
 import java.io.IOException;
+import java.text.SimpleDateFormat;
 import java.util.*;
 
 /**
@@ -47,7 +48,7 @@ public class TestFileListEntityProcessor
             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();
@@ -158,8 +159,8 @@ 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);
     fList = getFiles(resolver, attrs);

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestLineEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestLineEntityProcessor.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestLineEntityProcessor.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestLineEntityProcessor.java Mon Nov 19 16:23:50 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

Added: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestNestedChildren.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestNestedChildren.java?rev=1411276&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestNestedChildren.java (added)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestNestedChildren.java Mon Nov 19 16:23:50 2012
@@ -0,0 +1,60 @@
+package org.apache.solr.handler.dataimport;
+
+import org.junit.Test;
+
+/*
+ * 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.
+ */
+
+public class TestNestedChildren extends AbstractDIHJdbcTestCase {
+
+  @Test
+  public void test() throws Exception {
+    h.query("/dataimport", generateRequest());
+    assertQ(req("*:*"), "//*[@numFound='1']");
+    assertQ(req("third_s:CHICKEN"), "//*[@numFound='1']");
+  } 
+  
+  @Override
+  protected String generateConfig() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("<dataConfig> \n");
+    sb.append("<dataSource name=\"derby\" driver=\"org.apache.derby.jdbc.EmbeddedDriver\" url=\"jdbc:derby:memory:derbyDB;\" /> \n");
+    sb.append("<document name=\"TestSimplePropertiesWriter\"> \n");
+    sb.append("<entity name=\"FIRST\" processor=\"SqlEntityProcessor\" dataSource=\"derby\" ");
+    sb.append(" query=\"select 1 as id, 'PORK' as FIRST_S from sysibm.sysdummy1 \" >\n");
+    sb.append("  <field column=\"FIRST_S\" name=\"first_s\" /> \n");
+    sb.append("  <entity name=\"SECOND\" processor=\"SqlEntityProcessor\" dataSource=\"derby\" ");
+    sb.append("   query=\"select 1 as id, 2 as SECOND_ID, 'BEEF' as SECOND_S from sysibm.sysdummy1 WHERE 1=${FIRST.ID}\" >\n");
+    sb.append("   <field column=\"SECOND_S\" name=\"second_s\" /> \n");
+    sb.append("   <entity name=\"SECOND\" processor=\"SqlEntityProcessor\" dataSource=\"derby\" ");
+    sb.append("    query=\"select 1 as id, 'CHICKEN' as THIRD_S from sysibm.sysdummy1 WHERE 2=${SECOND.SECOND_ID}\" >\n");
+    sb.append("    <field column=\"THIRD_S\" name=\"third_s\" /> \n");
+    sb.append("   </entity>\n");
+    sb.append("  </entity>\n");
+    sb.append("</entity>\n");
+    sb.append("</document> \n");
+    sb.append("</dataConfig> \n");
+    String config = sb.toString();
+    log.debug(config); 
+    return config;
+  }
+  
+  @Override
+  protected Database setAllowedDatabases() {
+    return Database.DERBY;
+  }   
+}

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestNumberFormatTransformer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestNumberFormatTransformer.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestNumberFormatTransformer.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestNumberFormatTransformer.java Mon Nov 19 16:23:50 2012
@@ -66,7 +66,7 @@ public class TestNumberFormatTransformer
     inputs.add("245" + GROUPING_SEP + "678");
     Map 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);

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestRegexTransformer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestRegexTransformer.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestRegexTransformer.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestRegexTransformer.java Mon Nov 19 16:23:50 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/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSqlEntityProcessor2.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSqlEntityProcessor2.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSqlEntityProcessor2.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSqlEntityProcessor2.java Mon Nov 19 16:23:50 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;
 
@@ -70,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) {

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestTemplateTransformer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestTemplateTransformer.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestTemplateTransformer.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestTemplateTransformer.java Mon Nov 19 16:23:50 2012
@@ -58,7 +58,7 @@ public class TestTemplateTransformer ext
             "lastName", "Mangar",
             "mail", mails);
 
-    VariableResolverImpl resolver = new VariableResolverImpl();
+    VariableResolver resolver = new VariableResolver();
     resolver.addNamespace("e", row);
     Map<String, String> entityAttrs = createMap("name", "e");
 

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestURLDataSource.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestURLDataSource.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestURLDataSource.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestURLDataSource.java Mon Nov 19 16:23:50 2012
@@ -27,7 +27,7 @@ import org.junit.Test;
 public class TestURLDataSource extends AbstractDataImportHandlerTestCase {
   private List<Map<String, String>> fields = new ArrayList<Map<String, String>>();
   private URLDataSource dataSource = new URLDataSource();
-  private VariableResolverImpl variableResolver = new VariableResolverImpl();
+  private VariableResolver variableResolver = new VariableResolver();
   private Context context = AbstractDataImportHandlerTestCase.getContext(null, variableResolver,
       dataSource, Context.FULL_DUMP, fields, null);
   private Properties initProps = new Properties();

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestVariableResolver.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestVariableResolver.java?rev=1411276&r1=1411275&r2=1411276&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestVariableResolver.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestVariableResolver.java Mon Nov 19 16:23:50 2012
@@ -34,7 +34,7 @@ public class TestVariableResolver extend
 
   @Test
   public void testSimpleNamespace() {
-    VariableResolverImpl vri = new VariableResolverImpl();
+    VariableResolver vri = new VariableResolver();
     Map<String, Object> ns = new HashMap<String, Object>();
     ns.put("world", "WORLD");
     vri.addNamespace("hello", ns);
@@ -47,9 +47,9 @@ public class TestVariableResolver extend
     System.setProperty(TestVariableResolver.class.getName(),"hello");
     // System.out.println("s.gP()"+ System.getProperty(TestVariableResolver.class.getName())); 
 
-    HashMap m = new HashMap();
-    m.put("hello","world");
-    VariableResolverImpl vri = new VariableResolverImpl(m);
+    Properties p = new Properties();
+    p.put("hello","world");
+    VariableResolver vri = new VariableResolver(p);
     Object val = vri.resolve(TestVariableResolver.class.getName());
     // System.out.println("val = " + val);
     assertEquals("hello", val);
@@ -58,7 +58,7 @@ public class TestVariableResolver extend
 
   @Test
   public void testNestedNamespace() {
-    VariableResolverImpl vri = new VariableResolverImpl();
+    VariableResolver vri = new VariableResolver();
     Map<String, Object> ns = new HashMap<String, Object>();
     ns.put("world", "WORLD");
     vri.addNamespace("hello", ns);
@@ -70,7 +70,7 @@ public class TestVariableResolver extend
 
   @Test
   public void test3LevelNestedNamespace() {
-    VariableResolverImpl vri = new VariableResolverImpl();
+    VariableResolver vri = new VariableResolver();
     Map<String, Object> ns = new HashMap<String, Object>();
     ns.put("world", "WORLD");
     vri.addNamespace("hello", ns);
@@ -82,9 +82,8 @@ public class TestVariableResolver extend
 
   @Test
   public void dateNamespaceWithValue() {
-    VariableResolverImpl vri = new VariableResolverImpl();
-    vri.addNamespace("dataimporter.functions", EvaluatorBag
-            .getFunctionsNamespace(Collections.EMPTY_LIST, null, vri));
+    VariableResolver vri = new VariableResolver();
+    vri.setEvaluators(new DataImporter().getEvaluators(Collections.<Map<String,String>>emptyList()));
     Map<String, Object> ns = new HashMap<String, Object>();
     Date d = new Date();
     ns.put("dt", d);
@@ -95,12 +94,10 @@ public class TestVariableResolver extend
 
   @Test
   public void dateNamespaceWithExpr() throws Exception {
-    VariableResolverImpl vri = new VariableResolverImpl();
-    vri.addNamespace("dataimporter.functions", EvaluatorBag
-            .getFunctionsNamespace(Collections.EMPTY_LIST,null, vri));
+    VariableResolver vri = new VariableResolver();
+    vri.setEvaluators(new DataImporter().getEvaluators(Collections.<Map<String,String>>emptyList()));
     SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss'Z'");
     format.setTimeZone(TimeZone.getTimeZone("UTC"));
-    resetEvaluatorBagDateMathParser();
     DateMathParser dmp = new DateMathParser(TimeZone.getDefault(), Locale.getDefault());
 
     String s = vri.replaceTokens("${dataimporter.functions.formatDate('NOW/DAY','yyyy-MM-dd HH:mm')}");
@@ -109,7 +106,7 @@ public class TestVariableResolver extend
 
   @Test
   public void testDefaultNamespace() {
-    VariableResolverImpl vri = new VariableResolverImpl();
+    VariableResolver vri = new VariableResolver();
     Map<String, Object> ns = new HashMap<String, Object>();
     ns.put("world", "WORLD");
     vri.addNamespace(null, ns);
@@ -118,7 +115,7 @@ public class TestVariableResolver extend
 
   @Test
   public void testDefaultNamespace1() {
-    VariableResolverImpl vri = new VariableResolverImpl();
+    VariableResolver vri = new VariableResolver();
     Map<String, Object> ns = new HashMap<String, Object>();
     ns.put("world", "WORLD");
     vri.addNamespace(null, ns);
@@ -127,22 +124,21 @@ public class TestVariableResolver extend
 
   @Test
   public void testFunctionNamespace1() throws Exception {
-    VariableResolverImpl resolver = new VariableResolverImpl();
-    ContextImpl context = new ContextImpl(null, resolver, null, Context.FULL_DUMP, Collections.EMPTY_MAP, null, null);
+    VariableResolver resolver = new VariableResolver();
     final List<Map<String ,String >> l = new ArrayList<Map<String, String>>();
     Map<String ,String > m = new HashMap<String, String>();
     m.put("name","test");
     m.put("class",E.class.getName());
     l.add(m);
+    resolver.setEvaluators(new DataImporter().getEvaluators(l));
+    ContextImpl context = new ContextImpl(null, resolver, null, Context.FULL_DUMP, Collections.EMPTY_MAP, null, null);
+    
 
     SimpleDateFormat format = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss'Z'");
     format.setTimeZone(TimeZone.getTimeZone("UTC"));
-    resetEvaluatorBagDateMathParser();
-    DateMathParser dmp = new DateMathParser(TimeZone.getDefault(), Locale.getDefault());
-
-    resolver.addNamespace("dataimporter.functions", EvaluatorBag
-            .getFunctionsNamespace(l,null, resolver));
-    String s = resolver
+     DateMathParser dmp = new DateMathParser(TimeZone.getDefault(), Locale.getDefault());
+		
+		String s = resolver
             .replaceTokens("${dataimporter.functions.formatDate('NOW/DAY','yyyy-MM-dd HH:mm')}");
     assertEquals(new SimpleDateFormat("yyyy-MM-dd HH:mm")
             .format(dmp.parseMath("/DAY")), s);
@@ -157,13 +153,5 @@ public class TestVariableResolver extend
       }
   }
 
-  private void resetEvaluatorBagDateMathParser() {
-    EvaluatorBag.dateMathParser = new DateMathParser(TimeZone
-            .getDefault(), Locale.getDefault()){
-      @Override
-      public Date getNow() {
-        return new Date();
-      }
-    };
-  }
+  
 }