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/04/30 18:59:31 UTC

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

Added: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/RequestInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/RequestInfo.java?rev=1332292&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/RequestInfo.java (added)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/RequestInfo.java Mon Apr 30 16:59:30 2012
@@ -0,0 +1,149 @@
+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 org.apache.solr.common.util.ContentStream;
+import org.apache.solr.common.util.StrUtils;
+
+public class RequestInfo {
+  private final String command;
+  private final boolean debug;  
+  private final boolean syncMode;
+  private final boolean commit; 
+  private final boolean optimize;
+  private final int start;
+  private final long rows; 
+  private final boolean clean; 
+  private final List<String> entitiesToRun;
+  private final Map<String,Object> rawParams;
+  private final String dataConfig;  
+  
+  //TODO:  find a different home for these two...
+  private final ContentStream contentStream;  
+  private final DebugInfo debugInfo;
+  
+  public RequestInfo(Map<String,Object> requestParams, ContentStream stream) {
+    this.contentStream = stream;    
+    if (requestParams.containsKey("command")) { 
+      command = (String) requestParams.get("command");
+    } else {
+      command = null;
+    }    
+    boolean debugMode = StrUtils.parseBool((String) requestParams.get("debug"), false);    
+    if (debugMode) {
+      debug = true;
+      debugInfo = new DebugInfo(requestParams);
+    } else {
+      debug = false;
+      debugInfo = null;
+    }       
+    if (requestParams.containsKey("clean")) {
+      clean = StrUtils.parseBool( (String) requestParams.get("clean"), true);
+    } else if (DataImporter.DELTA_IMPORT_CMD.equals(command) || DataImporter.IMPORT_CMD.equals(command)) {
+      clean = false;
+    } else  {
+      clean = debug ? false : true;
+    }    
+    optimize = StrUtils.parseBool((String) requestParams.get("optimize"), false);
+    if(optimize) {
+      commit = true;
+    } else {
+      commit = StrUtils.parseBool( (String) requestParams.get("commit"), (debug ? false : true));
+    }      
+    if (requestParams.containsKey("rows")) {
+      rows = Integer.parseInt((String) requestParams.get("rows"));
+    } else {
+      rows = debug ? 10 : Long.MAX_VALUE;
+    }      
+    
+    if (requestParams.containsKey("start")) {
+      start = Integer.parseInt((String) requestParams.get("start"));
+    } else {
+      start = 0;
+    }
+    syncMode = StrUtils.parseBool((String) requestParams.get("synchronous"), false);    
+    
+    Object o = requestParams.get("entity");     
+    List<String> modifiableEntities = null;
+    if(o != null) {
+      if (o instanceof String) {
+        modifiableEntities = new ArrayList<String>();
+        modifiableEntities.add((String) o);
+      } else if (o instanceof List<?>) {
+        @SuppressWarnings("unchecked")
+        List<String> modifiableEntities1 = new ArrayList<String>((List<String>) o);
+        modifiableEntities = modifiableEntities1;
+      } 
+      entitiesToRun = Collections.unmodifiableList(modifiableEntities);
+    } else {
+      entitiesToRun = null;
+    }
+    
+    String dataConfigParam = (String) requestParams.get("dataConfig");
+    if (dataConfigParam != null && dataConfigParam.trim().length() == 0) {
+      // Empty data-config param is not valid, change it to null
+      dataConfigParam = null;
+    }
+    dataConfig = dataConfigParam;
+    this.rawParams = Collections.unmodifiableMap(new HashMap<String,Object>(requestParams));
+  }
+
+  public String getCommand() {
+    return command;
+  }
+
+  public boolean isDebug() {
+    return debug;
+  }
+
+  public boolean isSyncMode() {
+    return syncMode;
+  }
+
+  public boolean isCommit() {
+    return commit;
+  }
+
+  public boolean isOptimize() {
+    return optimize;
+  }
+
+  public int getStart() {
+    return start;
+  }
+
+  public long getRows() {
+    return rows;
+  }
+
+  public boolean isClean() {
+    return clean;
+  }
+  /**
+   * Returns null if we are to run all entities, otherwise just run the entities named in the list.
+   * @return
+   */
+  public List<String> getEntitiesToRun() {
+    return entitiesToRun;
+  }
+
+   public String getDataConfig() {
+    return dataConfig;
+  }
+
+  public Map<String,Object> getRawParams() {
+    return rawParams;
+  }
+
+  public ContentStream getContentStream() {
+    return contentStream;
+  }
+
+  public DebugInfo getDebugInfo() {
+    return debugInfo;
+  }
+}
\ No newline at end of file

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrWriter.java?rev=1332292&r1=1332291&r2=1332292&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrWriter.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrWriter.java Mon Apr 30 16:59:30 2012
@@ -39,7 +39,7 @@ import java.io.*;
 public class SolrWriter extends DIHWriterBase implements DIHWriter {
   private static final Logger log = LoggerFactory.getLogger(SolrWriter.class);
 
-  static final String LAST_INDEX_KEY = "last_index_time";
+  public static final String LAST_INDEX_KEY = "last_index_time";
 
   private final UpdateRequestProcessor processor;
   private final int commitWithin;

Added: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/ConfigNameConstants.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/ConfigNameConstants.java?rev=1332292&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/ConfigNameConstants.java (added)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/ConfigNameConstants.java Mon Apr 30 16:59:30 2012
@@ -0,0 +1,60 @@
+/**
+ * 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.config;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.apache.solr.handler.dataimport.SolrWriter;
+
+public class ConfigNameConstants {
+  public static final String SCRIPT = "script";
+
+  public static final String NAME = "name";
+
+  public static final String PROCESSOR = "processor";
+
+  /**
+   * @deprecated use IMPORTER_NS_SHORT instead
+   */
+  @Deprecated
+  public static final String IMPORTER_NS = "dataimporter";
+
+  public static final String IMPORTER_NS_SHORT = "dih";
+
+  public static final String ROOT_ENTITY = "rootEntity";
+
+  public static final String FUNCTION = "function";
+
+  public static final String CLASS = "class";
+
+  public static final String DATA_SRC = "dataSource";
+
+  public static final Set<String> RESERVED_WORDS;
+  static{
+    Set<String> rw =  new HashSet<String>();
+    rw.add(IMPORTER_NS);
+    rw.add(IMPORTER_NS_SHORT);
+    rw.add("request");
+    rw.add("delta");
+    rw.add("functions");
+    rw.add("session");
+    rw.add(SolrWriter.LAST_INDEX_KEY);
+    RESERVED_WORDS = Collections.unmodifiableSet(rw);
+  } 
+}

Added: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/ConfigParseUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/ConfigParseUtil.java?rev=1332292&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/ConfigParseUtil.java (added)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/ConfigParseUtil.java Mon Apr 30 16:59:30 2012
@@ -0,0 +1,122 @@
+/**
+ * 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.config;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.solr.handler.dataimport.DataImporter;
+import org.apache.solr.schema.SchemaField;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.w3c.dom.Element;
+import org.w3c.dom.NamedNodeMap;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+public class ConfigParseUtil {
+  private static final Logger LOG = LoggerFactory .getLogger(ConfigParseUtil.class);
+  
+  public static String getStringAttribute(Element e, String name, String def) {
+    String r = e.getAttribute(name);
+    if (r == null || "".equals(r.trim())) r = def;
+    return r;
+  }
+  
+  public static HashMap<String,String> getAllAttributes(Element e) {
+    HashMap<String,String> m = new HashMap<String,String>();
+    NamedNodeMap nnm = e.getAttributes();
+    for (int i = 0; i < nnm.getLength(); i++) {
+      m.put(nnm.item(i).getNodeName(), nnm.item(i).getNodeValue());
+    }
+    return m;
+  }
+  
+  public static String getText(Node elem, StringBuilder buffer) {
+    if (elem.getNodeType() != Node.CDATA_SECTION_NODE) {
+      NodeList childs = elem.getChildNodes();
+      for (int i = 0; i < childs.getLength(); i++) {
+        Node child = childs.item(i);
+        short childType = child.getNodeType();
+        if (childType != Node.COMMENT_NODE
+            && childType != Node.PROCESSING_INSTRUCTION_NODE) {
+          getText(child, buffer);
+        }
+      }
+    } else {
+      buffer.append(elem.getNodeValue());
+    }
+    
+    return buffer.toString();
+  }
+  
+  public static List<Element> getChildNodes(Element e, String byName) {
+    List<Element> result = new ArrayList<Element>();
+    NodeList l = e.getChildNodes();
+    for (int i = 0; i < l.getLength(); i++) {
+      if (e.equals(l.item(i).getParentNode())
+          && byName.equals(l.item(i).getNodeName())) result.add((Element) l
+          .item(i));
+    }
+    return result;
+  }
+  
+  public static void verifyWithSchema(DataImporter di, Map<String,EntityField> fields) {
+    Map<String,SchemaField> schemaFields = null;
+    if (di.getSchema() == null) {
+      schemaFields = Collections.emptyMap();
+    } else {
+      schemaFields = di.getSchema().getFields();
+    }
+    for (Map.Entry<String,SchemaField> entry : schemaFields.entrySet()) {
+      SchemaField sf = entry.getValue();
+      if (!fields.containsKey(sf.getName())) {
+        if (sf.isRequired()) {
+          LOG
+              .info(sf.getName()
+                  + " is a required field in SolrSchema . But not found in DataConfig");
+        }
+      }
+    }
+    for (Map.Entry<String,EntityField> entry : fields.entrySet()) {
+      EntityField fld = entry.getValue();
+      SchemaField field = di.getSchemaField(fld.getName());
+      if (field == null) {
+        LOG
+            .info("The field :"
+                + fld.getName()
+                + " present in DataConfig does not have a counterpart in Solr Schema");
+      }
+    }
+  }
+  
+  public static Map<String,EntityField> gatherAllFields(DataImporter di, Entity e) {
+    Map<String,EntityField> fields = new HashMap<String,EntityField>();
+    if (e.getFields() != null) {
+      for (EntityField f : e.getFields()) {
+        fields.put(f.getName(), f);
+      }
+    }
+    for (Entity e1 : e.getChildren()) {
+      fields.putAll(gatherAllFields(di, e1));
+    }
+    return fields;
+  }
+}

Added: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/DIHConfiguration.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/DIHConfiguration.java?rev=1332292&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/DIHConfiguration.java (added)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/DIHConfiguration.java Mon Apr 30 16:59:30 2012
@@ -0,0 +1,99 @@
+package org.apache.solr.handler.dataimport.config;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.solr.handler.dataimport.DataImporter;
+import org.w3c.dom.Element;
+
+/**
+ * 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>
+ * Mapping for data-config.xml
+ * </p>
+ * <p/>
+ * <p>
+ * Refer to <a
+ * href="http://wiki.apache.org/solr/DataImportHandler">http://wiki.apache.org/solr/DataImportHandler</a>
+ * for more details.
+ * </p>
+ * <p/>
+ * <b>This API is experimental and subject to change</b>
+ *
+ * @since solr 1.3
+ */
+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, Properties> dataSources;
+  public DIHConfiguration(Element element, DataImporter di, List<Map<String, String>> functions, Script script, Map<String, Properties> dataSources) {
+    this.deleteQuery = ConfigParseUtil.getStringAttribute(element, "deleteQuery", null);
+    this.onImportStart = ConfigParseUtil.getStringAttribute(element, "onImportStart", null);
+    this.onImportEnd = ConfigParseUtil.getStringAttribute(element, "onImportEnd", null);
+    List<Entity> modEntities = new ArrayList<Entity>();
+    List<Element> l = ConfigParseUtil.getChildNodes(element, "entity");
+    boolean docRootFound = false;
+    for (Element e : l) {
+      Entity entity = new Entity(docRootFound, e, di, null);
+      Map<String, EntityField> fields = ConfigParseUtil.gatherAllFields(di, entity);
+      ConfigParseUtil.verifyWithSchema(di, fields);    
+      modEntities.add(entity);
+    }
+    this.entities = Collections.unmodifiableList(modEntities);
+    if(functions==null) {
+      functions = Collections.emptyList();
+    }
+    List<Map<String, String>> modFunc = new ArrayList<Map<String, String>>(functions.size());
+    for(Map<String, String> f : functions) {
+      modFunc.add(Collections.unmodifiableMap(f));
+    }
+    this.functions = Collections.unmodifiableList(modFunc);
+    this.script = script;
+    this.dataSources = Collections.unmodifiableMap(dataSources);
+  }
+  public String getDeleteQuery() {
+    return deleteQuery;
+  }
+  public List<Entity> getEntities() {
+    return entities;
+  }
+  public String getOnImportStart() {
+    return onImportStart;
+  }
+  public String getOnImportEnd() {
+    return onImportEnd;
+  }
+  public List<Map<String,String>> getFunctions() {
+    return functions;
+  }
+  public Map<String,Properties> getDataSources() {
+    return dataSources;
+  }
+  public Script getScript() {
+    return script;
+  }
+}
\ No newline at end of file

Added: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/Document.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/Document.java?rev=1332292&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/Document.java (added)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/Document.java Mon Apr 30 16:59:30 2012
@@ -0,0 +1,99 @@
+package org.apache.solr.handler.dataimport.config;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import org.apache.solr.handler.dataimport.DataImporter;
+import org.w3c.dom.Element;
+
+/**
+ * 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>
+ * Mapping for data-config.xml
+ * </p>
+ * <p/>
+ * <p>
+ * Refer to <a
+ * href="http://wiki.apache.org/solr/DataImportHandler">http://wiki.apache.org/solr/DataImportHandler</a>
+ * for more details.
+ * </p>
+ * <p/>
+ * <b>This API is experimental and subject to change</b>
+ *
+ * @since solr 1.3
+ */
+public class Document {
+  // 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, Properties> dataSources;
+  public Document(Element element, DataImporter di, List<Map<String, String>> functions, Script script, Map<String, Properties> dataSources) {
+    this.deleteQuery = ConfigParseUtil.getStringAttribute(element, "deleteQuery", null);
+    this.onImportStart = ConfigParseUtil.getStringAttribute(element, "onImportStart", null);
+    this.onImportEnd = ConfigParseUtil.getStringAttribute(element, "onImportEnd", null);
+    List<Entity> modEntities = new ArrayList<Entity>();
+    List<Element> l = ConfigParseUtil.getChildNodes(element, "entity");
+    boolean docRootFound = false;
+    for (Element e : l) {
+      Entity entity = new Entity(docRootFound, e, di, null);
+      Map<String, EntityField> fields = ConfigParseUtil.gatherAllFields(di, entity);
+      ConfigParseUtil.verifyWithSchema(di, fields);    
+      modEntities.add(entity);
+    }
+    this.entities = Collections.unmodifiableList(modEntities);
+    if(functions==null) {
+      functions = Collections.emptyList();
+    }
+    List<Map<String, String>> modFunc = new ArrayList<Map<String, String>>(functions.size());
+    for(Map<String, String> f : functions) {
+      modFunc.add(Collections.unmodifiableMap(f));
+    }
+    this.functions = Collections.unmodifiableList(modFunc);
+    this.script = script;
+    this.dataSources = Collections.unmodifiableMap(dataSources);
+  }
+  public String getDeleteQuery() {
+    return deleteQuery;
+  }
+  public List<Entity> getEntities() {
+    return entities;
+  }
+  public String getOnImportStart() {
+    return onImportStart;
+  }
+  public String getOnImportEnd() {
+    return onImportEnd;
+  }
+  public List<Map<String,String>> getFunctions() {
+    return functions;
+  }
+  public Map<String,Properties> getDataSources() {
+    return dataSources;
+  }
+  public Script getScript() {
+    return script;
+  }
+}
\ No newline at end of file

Added: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/Entity.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/Entity.java?rev=1332292&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/Entity.java (added)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/Entity.java Mon Apr 30 16:59:30 2012
@@ -0,0 +1,220 @@
+/**
+ * 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.config;
+
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.solr.handler.dataimport.DataImportHandlerException;
+import org.apache.solr.handler.dataimport.DataImporter;
+import org.apache.solr.schema.SchemaField;
+import org.w3c.dom.Element;
+
+public class Entity {
+  private final String name;
+  private final String pk;
+  private final String pkMappingFromSchema;
+  private final String dataSourceName;
+  private final String processorName;
+  private final Entity parentEntity;
+  private final boolean docRoot;
+  private final List<Entity> children;
+  private final List<EntityField> fields;
+  private final Map<String,Set<EntityField>> colNameVsField;
+  private final Map<String,String> allAttributes;
+  private final List<Map<String,String>> allFieldAttributes;
+  
+  public Entity(boolean docRootFound, Element element, DataImporter di, Entity parent) {
+    this.parentEntity = parent;
+    
+    String modName = ConfigParseUtil.getStringAttribute(element, ConfigNameConstants.NAME, null);
+    if (modName == null) {
+      throw new DataImportHandlerException(SEVERE, "Entity must have a name.");
+    }
+    if (modName.indexOf(".") != -1) {
+      throw new DataImportHandlerException(SEVERE,
+          "Entity name must not have period (.): '" + modName);
+    }
+    if (ConfigNameConstants.RESERVED_WORDS.contains(modName)) {
+      throw new DataImportHandlerException(SEVERE, "Entity name : '" + modName
+          + "' is a reserved keyword. Reserved words are: " + ConfigNameConstants.RESERVED_WORDS);
+    }
+    this.name = modName;
+    this.pk = ConfigParseUtil.getStringAttribute(element, "pk", null);
+    this.processorName = ConfigParseUtil.getStringAttribute(element, ConfigNameConstants.PROCESSOR,null);
+    this.dataSourceName = ConfigParseUtil.getStringAttribute(element, DataImporter.DATA_SRC, null);
+    
+    String rawDocRootValue = ConfigParseUtil.getStringAttribute(element, ConfigNameConstants.ROOT_ENTITY, null);
+    if (!docRootFound && !"false".equals(rawDocRootValue)) {
+      // if in this chain no document root is found()
+      docRoot = true;
+    } else {
+      docRoot = false;
+    }
+    
+    Map<String,String> modAttributes = ConfigParseUtil
+        .getAllAttributes(element);
+    modAttributes.put(ConfigNameConstants.DATA_SRC, this.dataSourceName);
+    this.allAttributes = Collections.unmodifiableMap(modAttributes);
+    
+    List<Element> n = ConfigParseUtil.getChildNodes(element, "field");
+    List<EntityField> modFields = new ArrayList<EntityField>(n.size());
+    Map<String,Set<EntityField>> modColNameVsField = new HashMap<String,Set<EntityField>>();
+    List<Map<String,String>> modAllFieldAttributes = new ArrayList<Map<String,String>>();
+    for (Element elem : n) {
+      EntityField.Builder fieldBuilder = new EntityField.Builder(elem);
+      if (di.getSchema() != null) {
+        if (fieldBuilder.getNameOrColumn() != null
+            && fieldBuilder.getNameOrColumn().contains("${")) {
+          fieldBuilder.dynamicName = true;
+        } else {
+          SchemaField schemaField = di.getSchemaField(fieldBuilder
+              .getNameOrColumn());
+          if (schemaField != null) {
+            fieldBuilder.name = schemaField.getName();
+            fieldBuilder.multiValued = schemaField.multiValued();
+            fieldBuilder.allAttributes.put(DataImporter.MULTI_VALUED, Boolean
+                .toString(schemaField.multiValued()));
+            fieldBuilder.allAttributes.put(DataImporter.TYPE, schemaField
+                .getType().getTypeName());
+            fieldBuilder.allAttributes.put("indexed", Boolean
+                .toString(schemaField.indexed()));
+            fieldBuilder.allAttributes.put("stored", Boolean
+                .toString(schemaField.stored()));
+            fieldBuilder.allAttributes.put("defaultValue", schemaField
+                .getDefaultValue());
+          } else {
+            fieldBuilder.toWrite = false;
+          }
+        }
+      }
+      Set<EntityField> fieldSet = modColNameVsField.get(fieldBuilder.column);
+      if (fieldSet == null) {
+        fieldSet = new HashSet<EntityField>();
+        modColNameVsField.put(fieldBuilder.column, fieldSet);
+      }
+      fieldBuilder.allAttributes.put("boost", Float
+          .toString(fieldBuilder.boost));
+      fieldBuilder.allAttributes.put("toWrite", Boolean
+          .toString(fieldBuilder.toWrite));
+      modAllFieldAttributes.add(fieldBuilder.allAttributes);
+      fieldBuilder.entity = this;
+      EntityField field = new EntityField(fieldBuilder);
+      fieldSet.add(field);
+      modFields.add(field);
+    }
+    Map<String,Set<EntityField>> modColNameVsField1 = new HashMap<String,Set<EntityField>>();
+    for (Map.Entry<String,Set<EntityField>> entry : modColNameVsField
+        .entrySet()) {
+      if (entry.getValue().size() > 0) {
+        modColNameVsField1.put(entry.getKey(), Collections
+            .unmodifiableSet(entry.getValue()));
+      }
+    }
+    this.colNameVsField = Collections.unmodifiableMap(modColNameVsField1);
+    this.fields = Collections.unmodifiableList(modFields);
+    this.allFieldAttributes = Collections
+        .unmodifiableList(modAllFieldAttributes);
+    
+    String modPkMappingFromSchema = null;
+    if (di.getSchema() != null) {
+      SchemaField uniqueKey = di.getSchema().getUniqueKeyField();
+      if (uniqueKey != null) {
+        modPkMappingFromSchema = uniqueKey.getName();
+        // if no fields are mentioned . solr uniqueKey is same as dih 'pk'
+        for (EntityField field : fields) {
+          if (field.getName().equals(modPkMappingFromSchema)) {
+            modPkMappingFromSchema = field.getColumn();
+            // get the corresponding column mapping for the solr uniqueKey
+            // But if there are multiple columns mapping to the solr uniqueKey,
+            // it will fail
+            // so , in one off cases we may need pk
+            break;
+          }
+        }
+      }
+    }
+    pkMappingFromSchema = modPkMappingFromSchema;
+    n = ConfigParseUtil.getChildNodes(element, "entity");
+    List<Entity> modEntities = new ArrayList<Entity>();
+    for (Element elem : n) {
+      modEntities
+          .add(new Entity((docRootFound || this.docRoot), elem, di, this));
+    }
+    this.children = Collections.unmodifiableList(modEntities);
+  }
+  
+  public String getPk() {
+    return pk == null ? pkMappingFromSchema : pk;
+  }
+  
+  public String getSchemaPk() {
+    return pkMappingFromSchema != null ? pkMappingFromSchema : pk;
+  }
+  
+  public String getName() {
+    return name;
+  }
+  
+  public String getPkMappingFromSchema() {
+    return pkMappingFromSchema;
+  }
+  
+  public String getDataSourceName() {
+    return dataSourceName;
+  }
+  
+  public String getProcessorName() {
+    return processorName;
+  }
+  
+  public Entity getParentEntity() {
+    return parentEntity;
+  }
+  
+  public boolean isDocRoot() {
+    return docRoot;
+  }
+  
+  public List<Entity> getChildren() {
+    return children;
+  }
+  
+  public List<EntityField> getFields() {
+    return fields;
+  }
+  
+  public Map<String,Set<EntityField>> getColNameVsField() {
+    return colNameVsField;
+  }
+  
+  public Map<String,String> getAllAttributes() {
+    return allAttributes;
+  }
+  
+  public List<Map<String,String>> getAllFieldsList() {
+    return allFieldAttributes;
+  }
+}

Added: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/EntityField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/EntityField.java?rev=1332292&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/EntityField.java (added)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/EntityField.java Mon Apr 30 16:59:30 2012
@@ -0,0 +1,109 @@
+/**
+ * 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.config;
+
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.handler.dataimport.ConfigParseUtil;
+import org.apache.solr.handler.dataimport.DataImportHandlerException;
+import org.apache.solr.handler.dataimport.DataImporter;
+import org.w3c.dom.Element;
+
+public class EntityField {
+  private final String column;
+  private final String name;
+  private final float boost;
+  private final boolean toWrite;
+  private final boolean multiValued;
+  private final boolean dynamicName;
+  private final Entity entity;
+  private final Map<String, String> allAttributes;
+
+  public EntityField(Builder b) {
+    this.column = b.column;
+    this.name = b.name;
+    this.boost = b.boost;
+    this.toWrite = b.toWrite;
+    this.multiValued = b.multiValued;
+    this.dynamicName = b.dynamicName;
+    this.entity = b.entity;
+    this.allAttributes = Collections.unmodifiableMap(new HashMap<String,String>(b.allAttributes));
+  }
+
+  public String getName() {
+    return name == null ? column : name;
+  }
+
+  public Entity getEntity() {
+    return entity;
+  }
+
+  public String getColumn() {
+    return column;
+  }
+
+  public float getBoost() {
+    return boost;
+  }
+
+  public boolean isToWrite() {
+    return toWrite;
+  }
+
+  public boolean isMultiValued() {
+    return multiValued;
+  }
+
+  public boolean isDynamicName() {
+    return dynamicName;
+  }
+
+  public Map<String,String> getAllAttributes() {
+    return allAttributes;
+  }
+  
+  public static class Builder {    
+    public String column;
+    public String name;
+    public float boost;
+    public boolean toWrite = true;
+    public boolean multiValued = false;
+    public boolean dynamicName = false;
+    public Entity entity;
+    public Map<String, String> allAttributes = new HashMap<String,String>();
+    
+    public Builder(Element e) {
+      this.name = ConfigParseUtil.getStringAttribute(e, DataImporter.NAME, null);
+      this.column = ConfigParseUtil.getStringAttribute(e, DataImporter.COLUMN, null);
+      if (column == null) {
+        throw new DataImportHandlerException(SEVERE, "Field must have a column attribute");
+      }
+      this.boost = Float.parseFloat(ConfigParseUtil.getStringAttribute(e, "boost", "1.0f"));
+      this.allAttributes = new HashMap<String, String>(ConfigParseUtil.getAllAttributes(e));
+    }
+    
+    public String getNameOrColumn() {
+      return name==null ? column : name;
+    }
+  }
+
+}

Added: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/Field.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/Field.java?rev=1332292&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/Field.java (added)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/Field.java Mon Apr 30 16:59:30 2012
@@ -0,0 +1,109 @@
+/**
+ * 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.config;
+
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
+
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.solr.handler.dataimport.ConfigParseUtil;
+import org.apache.solr.handler.dataimport.DataImportHandlerException;
+import org.apache.solr.handler.dataimport.DataImporter;
+import org.w3c.dom.Element;
+
+public class Field {
+  private final String column;
+  private final String name;
+  private final float boost;
+  private final boolean toWrite;
+  private final boolean multiValued;
+  private final boolean dynamicName;
+  private final Entity entity;
+  private final Map<String, String> allAttributes;
+
+  public Field(Builder b) {
+    this.column = b.column;
+    this.name = b.name;
+    this.boost = b.boost;
+    this.toWrite = b.toWrite;
+    this.multiValued = b.multiValued;
+    this.dynamicName = b.dynamicName;
+    this.entity = b.entity;
+    this.allAttributes = Collections.unmodifiableMap(new HashMap<String,String>(b.allAttributes));
+  }
+
+  public String getName() {
+    return name == null ? column : name;
+  }
+
+  public Entity getEntity() {
+    return entity;
+  }
+
+  public String getColumn() {
+    return column;
+  }
+
+  public float getBoost() {
+    return boost;
+  }
+
+  public boolean isToWrite() {
+    return toWrite;
+  }
+
+  public boolean isMultiValued() {
+    return multiValued;
+  }
+
+  public boolean isDynamicName() {
+    return dynamicName;
+  }
+
+  public Map<String,String> getAllAttributes() {
+    return allAttributes;
+  }
+  
+  public static class Builder {    
+    public String column;
+    public String name;
+    public float boost;
+    public boolean toWrite = true;
+    public boolean multiValued = false;
+    public boolean dynamicName;
+    public Entity entity;
+    public Map<String, String> allAttributes = new HashMap<String,String>();
+    
+    public Builder(Element e) {
+      this.name = ConfigParseUtil.getStringAttribute(e, DataImporter.NAME, null);
+      this.column = ConfigParseUtil.getStringAttribute(e, DataImporter.COLUMN, null);
+      if (column == null) {
+        throw new DataImportHandlerException(SEVERE, "Field must have a column attribute");
+      }
+      this.boost = Float.parseFloat(ConfigParseUtil.getStringAttribute(e, "boost", "1.0f"));
+      this.allAttributes = new HashMap<String, String>(ConfigParseUtil.getAllAttributes(e));
+    }
+    
+    public String getNameOrColumn() {
+      return name==null ? column : name;
+    }
+  }
+
+}

Added: lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/Script.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/Script.java?rev=1332292&view=auto
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/Script.java (added)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/config/Script.java Mon Apr 30 16:59:30 2012
@@ -0,0 +1,42 @@
+/**
+ * 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.config;
+
+import org.w3c.dom.Element;
+
+public class Script {
+  private final String language;
+  private final String text;
+  
+  public Script(Element e) {
+    this.language = ConfigParseUtil.getStringAttribute(e, "language", "JavaScript");
+    StringBuilder buffer = new StringBuilder();
+    String script = ConfigParseUtil.getText(e, buffer);
+    if (script != null) {
+      this.text = script.trim();
+    } else {
+      this.text = null;
+    }
+  }  
+  public String getLanguage() {
+    return language;
+  }  
+  public String getText() {
+    return text;
+  }
+}

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=1332292&r1=1332291&r2=1332292&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 Apr 30 16:59:30 2012
@@ -132,15 +132,15 @@ public abstract class AbstractDataImport
    * Helper for creating a Context instance. Useful for testing Transformers
    */
   @SuppressWarnings("unchecked")
-  public static TestContext getContext(DataConfig.Entity parentEntity,
+  public static TestContext getContext(EntityProcessorWrapper parent,
                                    VariableResolverImpl resolver, DataSource parentDataSource,
                                    String currProcess, final List<Map<String, String>> entityFields,
                                    final Map<String, String> entityAttrs) {
     if (resolver == null) resolver = new VariableResolverImpl();
-    final Context delegate = new ContextImpl(parentEntity, resolver,
+    final Context delegate = new ContextImpl(parent, resolver,
             parentDataSource, currProcess,
             new HashMap<String, Object>(), null, null);
-    return new TestContext(entityAttrs, delegate, entityFields, parentEntity == null);
+    return new TestContext(entityAttrs, delegate, entityFields, parent == null);
   }
 
   /**

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestCachedSqlEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestCachedSqlEntityProcessor.java?rev=1332292&r1=1332291&r2=1332292&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestCachedSqlEntityProcessor.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestCachedSqlEntityProcessor.java Mon Apr 30 16:59:30 2012
@@ -49,7 +49,7 @@ public class TestCachedSqlEntityProcesso
     rows.add(createMap("id", 1, "desc", "one"));
     rows.add(createMap("id", 1, "desc", "another one"));
     MockDataSource.setIterator(vr.replaceTokens(q), rows.iterator());
-    EntityProcessor csep = new EntityProcessorWrapper( new CachedSqlEntityProcessor(), null);
+    EntityProcessor csep = new EntityProcessorWrapper(new CachedSqlEntityProcessor(), null, null);
     csep.init(context);
     rows = new ArrayList<Map<String, Object>>();
     while (true) {
@@ -90,7 +90,7 @@ public class TestCachedSqlEntityProcesso
     rows.add(createMap("id", 1, "desc", "one"));
     rows.add(createMap("id", 1, "desc", "another one"));
     MockDataSource.setIterator(vr.replaceTokens(q), rows.iterator());
-    EntityProcessor csep = new EntityProcessorWrapper( new CachedSqlEntityProcessor(), null);
+    EntityProcessor csep = new EntityProcessorWrapper( new CachedSqlEntityProcessor(), null, null);
     csep.init(context);
     rows = new ArrayList<Map<String, Object>>();
     while (true) {
@@ -132,7 +132,7 @@ public class TestCachedSqlEntityProcesso
     rows.add(createMap("id", 1, "desc", "one"));
     rows.add(createMap("id", 1, "desc", "another one"));
     MockDataSource.setIterator(vr.replaceTokens(q), rows.iterator());
-    EntityProcessor csep = new EntityProcessorWrapper( new CachedSqlEntityProcessor(), null);
+    EntityProcessor csep = new EntityProcessorWrapper( new CachedSqlEntityProcessor(), null, null);
     csep.init(context);
     rows = new ArrayList<Map<String, Object>>();
     while (true) {
@@ -223,7 +223,7 @@ public class TestCachedSqlEntityProcesso
     rows.add(createMap("id", 3, "desc", "another three"));
     rows.add(createMap("id", 3, "desc", "another another three"));
     MockDataSource.setIterator(q, rows.iterator());
-    EntityProcessor csep = new EntityProcessorWrapper(new CachedSqlEntityProcessor(), null);
+    EntityProcessor csep = new EntityProcessorWrapper(new CachedSqlEntityProcessor(), null, null);
     csep.init(context);
     rows = new ArrayList<Map<String, Object>>();
     while (true) {

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=1332292&r1=1332291&r2=1332292&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 Apr 30 16:59:30 2012
@@ -18,15 +18,14 @@ package org.apache.solr.handler.dataimpo
 
 import java.util.HashMap;
 
-import org.apache.solr.handler.dataimport.DataConfig.Entity;
-import org.apache.solr.handler.dataimport.DataImporter.RequestParams;
+import org.apache.solr.handler.dataimport.RequestInfo;
 import org.junit.Test;
 
 public class TestContextImpl extends AbstractDataImportHandlerTestCase {
   
   @Test
   public void testEntityScope() {
-    ContextImpl ctx = new ContextImpl(new Entity(), new VariableResolverImpl(), null, "something", new HashMap<String,Object>(), null, null);
+    ContextImpl ctx = new ContextImpl(null, new VariableResolverImpl(), 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,8 +37,8 @@ public class TestContextImpl extends Abs
   public void testCoreScope() {
     DataImporter di = new DataImporter();
     di.loadAndInit("<dataConfig><document /></dataConfig>");
-    DocBuilder db = new DocBuilder(di, new SolrWriter(null, null),new SimplePropertiesWriter(), new RequestParams());
-    ContextImpl ctx = new ContextImpl(new Entity(), new VariableResolverImpl(), null, "something", new HashMap<String,Object>(), null, db);
+    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);
     String lala = new String("lala");
     ctx.setSessionAttribute("huhu", lala, Context.SCOPE_SOLR_CORE);
     Object got = ctx.getSessionAttribute("huhu", Context.SCOPE_SOLR_CORE);
@@ -48,7 +47,7 @@ public class TestContextImpl extends Abs
   }
   @Test
   public void testDocumentScope() {
-    ContextImpl ctx = new ContextImpl(new Entity(), new VariableResolverImpl(), null, "something", new HashMap<String,Object>(), null, null);
+    ContextImpl ctx = new ContextImpl(null, new VariableResolverImpl(), 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);
@@ -59,7 +58,7 @@ public class TestContextImpl extends Abs
   }
   @Test
   public void testGlobalScope() {
-    ContextImpl ctx = new ContextImpl(new Entity(), new VariableResolverImpl(), null, "something", new HashMap<String,Object>(), null, null);
+    ContextImpl ctx = new ContextImpl(null, new VariableResolverImpl(), 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/TestDataConfig.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDataConfig.java?rev=1332292&r1=1332291&r2=1332292&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDataConfig.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDataConfig.java Mon Apr 30 16:59:30 2012
@@ -16,6 +16,8 @@
  */
 package org.apache.solr.handler.dataimport;
 
+import org.apache.solr.handler.dataimport.config.ConfigNameConstants;
+import org.apache.solr.handler.dataimport.config.DIHConfiguration;
 import org.junit.BeforeClass;
 import org.junit.Test;
 import org.w3c.dom.Document;
@@ -57,10 +59,9 @@ public class TestDataConfig extends Abst
     javax.xml.parsers.DocumentBuilder builder = DocumentBuilderFactory
             .newInstance().newDocumentBuilder();
     Document doc = builder.parse(new InputSource(new StringReader(xml)));
-
-    DataConfig dc = new DataConfig();
-    dc.readFromXml(doc.getDocumentElement());
-    assertEquals("atrimlisting", dc.document.entities.get(0).name);
+    DataImporter di = new DataImporter();
+    DIHConfiguration dc = di.readFromXml(doc);
+    assertEquals("atrimlisting", dc.getEntities().get(0).getName());
   }
 
   private static final String xml = "<dataConfig>\n"

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDocBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDocBuilder.java?rev=1332292&r1=1332291&r2=1332292&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDocBuilder.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestDocBuilder.java Mon Apr 30 16:59:30 2012
@@ -17,6 +17,9 @@
 package org.apache.solr.handler.dataimport;
 
 import org.apache.solr.common.SolrInputDocument;
+import org.apache.solr.handler.dataimport.config.ConfigNameConstants;
+import org.apache.solr.handler.dataimport.config.DIHConfiguration;
+import org.apache.solr.handler.dataimport.config.Entity;
 
 import org.junit.After;
 import org.junit.Test;
@@ -42,7 +45,8 @@ public class TestDocBuilder extends Abst
   
   @Test
   public void loadClass() throws Exception {
-    Class clz = DocBuilder.loadClass("RegexTransformer", null);
+    @SuppressWarnings("unchecked")
+    Class<Transformer> clz = DocBuilder.loadClass("RegexTransformer", null);
     assertNotNull(clz);
   }
 
@@ -50,13 +54,10 @@ public class TestDocBuilder extends Abst
   public void singleEntityNoRows() {
     DataImporter di = new DataImporter();
     di.loadAndInit(dc_singleEntity);
-    DataConfig cfg = di.getConfig();
-    DataConfig.Entity ent = cfg.document.entities.get(0);
-    MockDataSource.setIterator("select * from x", new ArrayList().iterator());
-    ent.dataSrc = new MockDataSource();
-    ent.isDocRoot = true;
-    DataImporter.RequestParams rp = new DataImporter.RequestParams();
-    rp.command = "full-import";
+    DIHConfiguration cfg = di.getConfig();
+    Entity ent = cfg.getEntities().get(0);
+    MockDataSource.setIterator("select * from x", new ArrayList<Map<String, Object>>().iterator());
+    RequestInfo rp = new RequestInfo(createMap("command", "full-import"), null);
     SolrWriterImpl swi = new SolrWriterImpl();
     di.runCmd(rp, swi);
     assertEquals(Boolean.TRUE, swi.deleteAllCalled);
@@ -72,13 +73,11 @@ public class TestDocBuilder extends Abst
   public void testDeltaImportNoRows_MustNotCommit() {
     DataImporter di = new DataImporter();
     di.loadAndInit(dc_deltaConfig);
-    DataConfig cfg = di.getConfig();
-    DataConfig.Entity ent = cfg.document.entities.get(0);
-    MockDataSource.setIterator("select * from x", new ArrayList().iterator());
-    MockDataSource.setIterator("select id from x", new ArrayList().iterator());
-    ent.dataSrc = new MockDataSource();
-    ent.isDocRoot = true;
-    DataImporter.RequestParams rp = new DataImporter.RequestParams(createMap("command", "delta-import"));
+    DIHConfiguration cfg = di.getConfig();
+    Entity ent = cfg.getEntities().get(0);
+    MockDataSource.setIterator("select * from x", new ArrayList<Map<String, Object>>().iterator());
+    MockDataSource.setIterator("select id from x", new ArrayList<Map<String, Object>>().iterator());
+    RequestInfo rp = new RequestInfo(createMap("command", "delta-import"), null);
     SolrWriterImpl swi = new SolrWriterImpl();
     di.runCmd(rp, swi);
     assertEquals(Boolean.FALSE, swi.deleteAllCalled);
@@ -94,15 +93,12 @@ public class TestDocBuilder extends Abst
   public void singleEntityOneRow() {
     DataImporter di = new DataImporter();
     di.loadAndInit(dc_singleEntity);
-    DataConfig cfg = di.getConfig();
-    DataConfig.Entity ent = cfg.document.entities.get(0);
-    List l = new ArrayList();
+    DIHConfiguration cfg = di.getConfig();
+    Entity ent = cfg.getEntities().get(0);
+    List<Map<String, Object>> l = new ArrayList<Map<String, Object>>();
     l.add(createMap("id", 1, "desc", "one"));
     MockDataSource.setIterator("select * from x", l.iterator());
-    ent.dataSrc = new MockDataSource();
-    ent.isDocRoot = true;
-    DataImporter.RequestParams rp = new DataImporter.RequestParams();
-    rp.command = "full-import";
+    RequestInfo rp = new RequestInfo(createMap("command", "full-import"), null);
     SolrWriterImpl swi = new SolrWriterImpl();
     di.runCmd(rp, swi);
     assertEquals(Boolean.TRUE, swi.deleteAllCalled);
@@ -114,7 +110,7 @@ public class TestDocBuilder extends Abst
     assertEquals(1, di.getDocBuilder().importStatistics.rowsCount.get());
 
     for (int i = 0; i < l.size(); i++) {
-      Map<String, Object> map = (Map<String, Object>) l.get(i);
+      Map<String, Object> map = l.get(i);
       SolrInputDocument doc = swi.docs.get(i);
       for (Map.Entry<String, Object> entry : map.entrySet()) {
         assertEquals(entry.getValue(), doc.getFieldValue(entry.getKey()));
@@ -126,14 +122,12 @@ public class TestDocBuilder extends Abst
   public void testImportCommand() {
     DataImporter di = new DataImporter();
     di.loadAndInit(dc_singleEntity);
-    DataConfig cfg = di.getConfig();
-    DataConfig.Entity ent = cfg.document.entities.get(0);
-    List l = new ArrayList();
+    DIHConfiguration cfg = di.getConfig();
+    Entity ent = cfg.getEntities().get(0);
+    List<Map<String, Object>> l = new ArrayList<Map<String, Object>>();
     l.add(createMap("id", 1, "desc", "one"));
     MockDataSource.setIterator("select * from x", l.iterator());
-    ent.dataSrc = new MockDataSource();
-    ent.isDocRoot = true;
-    DataImporter.RequestParams rp = new DataImporter.RequestParams(createMap("command", "import"));
+    RequestInfo rp = new RequestInfo(createMap("command", "import"), null);
     SolrWriterImpl swi = new SolrWriterImpl();
     di.runCmd(rp, swi);
     assertEquals(Boolean.FALSE, swi.deleteAllCalled);
@@ -157,18 +151,15 @@ public class TestDocBuilder extends Abst
   public void singleEntityMultipleRows() {
     DataImporter di = new DataImporter();
     di.loadAndInit(dc_singleEntity);
-    DataConfig cfg = di.getConfig();
-    DataConfig.Entity ent = cfg.document.entities.get(0);
-    ent.isDocRoot = true;
-    DataImporter.RequestParams rp = new DataImporter.RequestParams();
-    rp.command = "full-import";
-    List l = new ArrayList();
+    DIHConfiguration cfg = di.getConfig();
+    Entity ent = cfg.getEntities().get(0);
+    RequestInfo rp = new RequestInfo(createMap("command", "full-import"), null);
+    List<Map<String, Object>> l = new ArrayList<Map<String, Object>>();
     l.add(createMap("id", 1, "desc", "one"));
     l.add(createMap("id", 2, "desc", "two"));
     l.add(createMap("id", 3, "desc", "three"));
 
     MockDataSource.setIterator("select * from x", l.iterator());
-    ent.dataSrc = new MockDataSource();
     SolrWriterImpl swi = new SolrWriterImpl();
     di.runCmd(rp, swi);
     assertEquals(Boolean.TRUE, swi.deleteAllCalled);

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestEntityProcessorBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestEntityProcessorBase.java?rev=1332292&r1=1332291&r2=1332292&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestEntityProcessorBase.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestEntityProcessorBase.java Mon Apr 30 16:59:30 2012
@@ -47,7 +47,7 @@ public class TestEntityProcessorBase ext
     Map<String, Object> src = new HashMap<String, Object>();
     src.put("A", "NA");
     src.put("B", "NA");
-    EntityProcessorWrapper sep = new EntityProcessorWrapper(new SqlEntityProcessor(), null);
+    EntityProcessorWrapper sep = new EntityProcessorWrapper(new SqlEntityProcessor(), null, null);
     sep.init(context);
     Map<String, Object> res = sep.applyTransformer(src);
     assertNotNull(res.get("T1"));

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFieldReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFieldReader.java?rev=1332292&r1=1332291&r2=1332292&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFieldReader.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestFieldReader.java Mon Apr 30 16:59:30 2012
@@ -36,7 +36,7 @@ public class TestFieldReader extends Abs
     DataImporter di = new DataImporter();
     di.loadAndInit(config);
     TestDocBuilder.SolrWriterImpl sw = new TestDocBuilder.SolrWriterImpl();
-    DataImporter.RequestParams rp = new DataImporter.RequestParams(createMap("command", "full-import"));
+    RequestInfo rp = new RequestInfo(createMap("command", "full-import"), null);
     List<Map<String, Object>> l = new ArrayList<Map<String, Object>>();
     l.add(createMap("xml", xml));
     MockDataSource.setIterator("select * from a", l.iterator());

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestPlainTextEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestPlainTextEntityProcessor.java?rev=1332292&r1=1332291&r2=1332292&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestPlainTextEntityProcessor.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestPlainTextEntityProcessor.java Mon Apr 30 16:59:30 2012
@@ -34,7 +34,7 @@ public class TestPlainTextEntityProcesso
     DataImporter di = new DataImporter();
     di.loadAndInit(DATA_CONFIG);
     TestDocBuilder.SolrWriterImpl sw = new TestDocBuilder.SolrWriterImpl();
-    DataImporter.RequestParams rp = new DataImporter.RequestParams(createMap("command", "full-import"));
+    RequestInfo rp = new RequestInfo(createMap("command", "full-import"), null);
     di.runCmd(rp, sw);
     assertEquals(DS.s, sw.docs.get(0).getFieldValue("x"));
   }

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestScriptTransformer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestScriptTransformer.java?rev=1332292&r1=1332291&r2=1332292&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestScriptTransformer.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestScriptTransformer.java Mon Apr 30 16:59:30 2012
@@ -16,6 +16,8 @@
  */
 package org.apache.solr.handler.dataimport;
 
+import org.apache.solr.handler.dataimport.config.ConfigNameConstants;
+import org.apache.solr.handler.dataimport.config.DIHConfiguration;
 import org.junit.Test;
 import org.w3c.dom.Document;
 import org.w3c.dom.Element;
@@ -48,7 +50,7 @@ public class TestScriptTransformer exten
       Context context = getContext("f1", script);
       Map<String, Object> map = new HashMap<String, Object>();
       map.put("name", "Scott");
-      EntityProcessorWrapper sep = new EntityProcessorWrapper(new SqlEntityProcessor(), null);
+      EntityProcessorWrapper sep = new EntityProcessorWrapper(new SqlEntityProcessor(), null, null);
       sep.init(context);
       sep.applyTransformer(map);
       assertEquals(map.get("name"), "Hello Scott");
@@ -81,7 +83,7 @@ public class TestScriptTransformer exten
       Context context = getContext("f1", script);
       Map<String, Object> map = new HashMap<String, Object>();
       map.put("name", "Scott");
-      EntityProcessorWrapper sep = new EntityProcessorWrapper(new SqlEntityProcessor(), null);
+      EntityProcessorWrapper sep = new EntityProcessorWrapper(new SqlEntityProcessor(), null, null);
       sep.init(context);
       sep.applyTransformer(map);
       assertEquals(map.get("name"), "Hello Scott");
@@ -98,10 +100,9 @@ public class TestScriptTransformer exten
       DocumentBuilder builder = DocumentBuilderFactory.newInstance()
               .newDocumentBuilder();
       Document document = builder.parse(new InputSource(new StringReader(xml)));
-      DataConfig config = new DataConfig();
-      config.readFromXml((Element) document.getElementsByTagName("dataConfig")
-              .item(0));
-      assertTrue(config.script.text.indexOf("checkNextToken") > -1);
+      DataImporter di = new DataImporter();
+      DIHConfiguration dc = di.readFromXml(document);
+      assertTrue(dc.getScript().getText().indexOf("checkNextToken") > -1);
     } catch (DataImportHandlerException e) {    
       assumeFalse("This JVM does not have Rhino installed.  Test Skipped.", e
           .getMessage().startsWith("Cannot load Script Engine for language"));
@@ -115,15 +116,13 @@ public class TestScriptTransformer exten
       DocumentBuilder builder = DocumentBuilderFactory.newInstance()
               .newDocumentBuilder();
       Document document = builder.parse(new InputSource(new StringReader(xml)));
-      DataConfig config = new DataConfig();
-      config.readFromXml((Element) document.getElementsByTagName("dataConfig")
-              .item(0));
-
-      Context c = getContext("checkNextToken", config.script.text);
+      DataImporter di = new DataImporter();
+      DIHConfiguration dc = di.readFromXml(document);
+      Context c = getContext("checkNextToken", dc.getScript().getText());
 
       Map map = new HashMap();
       map.put("nextToken", "hello");
-      EntityProcessorWrapper sep = new EntityProcessorWrapper(new SqlEntityProcessor(), null);
+      EntityProcessorWrapper sep = new EntityProcessorWrapper(new SqlEntityProcessor(), null, null);
       sep.init(c);
       sep.applyTransformer(map);
       assertEquals("true", map.get("$hasMore"));

Modified: lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSqlEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSqlEntityProcessor.java?rev=1332292&r1=1332291&r2=1332292&view=diff
==============================================================================
--- lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSqlEntityProcessor.java (original)
+++ lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/apache/solr/handler/dataimport/TestSqlEntityProcessor.java Mon Apr 30 16:59:30 2012
@@ -53,7 +53,7 @@ public class TestSqlEntityProcessor exte
 
   @Test
   public void testTranformer() {
-    EntityProcessor sep = new EntityProcessorWrapper( new SqlEntityProcessor(), null);
+    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>();
@@ -76,7 +76,7 @@ public class TestSqlEntityProcessor exte
 
   @Test
   public void testTranformerWithReflection() {
-    EntityProcessor sep = new EntityProcessorWrapper(new SqlEntityProcessor(), null);
+    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>();
@@ -99,7 +99,7 @@ public class TestSqlEntityProcessor exte
 
   @Test
   public void testTranformerList() {
-    EntityProcessor sep = new EntityProcessorWrapper(new SqlEntityProcessor(),null);
+    EntityProcessor sep = new EntityProcessorWrapper(new SqlEntityProcessor(),null, null);
     List<Map<String, Object>> rows = getRows(2);
     VariableResolverImpl vr = new VariableResolverImpl();