You are viewing a plain text version of this content. The canonical link for it is here.
Posted to solr-commits@lucene.apache.org by sh...@apache.org on 2008/07/30 21:36:00 UTC

svn commit: r681182 [3/6] - in /lucene/solr/trunk: ./ contrib/ contrib/dataimporthandler/ contrib/dataimporthandler/src/ contrib/dataimporthandler/src/main/ contrib/dataimporthandler/src/main/java/ contrib/dataimporthandler/src/main/java/org/ contrib/d...

Added: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/JdbcDataSource.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/JdbcDataSource.java?rev=681182&view=auto
==============================================================================
--- lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/JdbcDataSource.java (added)
+++ lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/JdbcDataSource.java Wed Jul 30 12:35:58 2008
@@ -0,0 +1,325 @@
+/**
+ * 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 java.sql.*;
+import java.util.*;
+import java.util.concurrent.Callable;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * <p>
+ * A DataSource implementation which can fetch data using JDBC.
+ * </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 may change in the future.</b>
+ *
+ * @version $Id$
+ * @since solr 1.3
+ */
+public class JdbcDataSource extends
+        DataSource<Iterator<Map<String, Object>>> {
+  private static final Logger LOG = Logger.getLogger(JdbcDataSource.class
+          .getName());
+
+  private Callable<Connection> factory;
+
+  private long connLastUsed = System.currentTimeMillis();
+
+  private Connection conn;
+
+  private Map<String, Integer> fieldNameVsType = new HashMap<String, Integer>();
+
+  private boolean convertType = false;
+
+  private int batchSize = FETCH_SIZE;
+
+  public void init(Context context, Properties initProps) {
+    Object o = initProps.get(CONVERT_TYPE);
+    if (o != null)
+      convertType = Boolean.parseBoolean(o.toString());
+
+    createConnectionFactory(context, initProps);
+    try {
+      conn = factory.call();
+    } catch (Exception e) {
+      throw new DataImportHandlerException(DataImportHandlerException.SEVERE,
+              "Unable to create database connection", e);
+    }
+
+    String bsz = initProps.getProperty("batchSize");
+    if (bsz != null) {
+      try {
+        batchSize = Integer.parseInt(bsz);
+        if (batchSize == -1)
+          batchSize = Integer.MIN_VALUE;
+      } catch (NumberFormatException e) {
+        LOG.log(Level.WARNING, "Invalid batch size: " + bsz);
+      }
+    }
+
+    for (Map<String, String> map : context.getAllEntityFields()) {
+      String n = map.get(DataImporter.COLUMN);
+      String t = map.get(DataImporter.TYPE);
+      if ("sint".equals(t) || "integer".equals(t))
+        fieldNameVsType.put(n, Types.INTEGER);
+      else if ("slong".equals(t) || "long".equals(t))
+        fieldNameVsType.put(n, Types.BIGINT);
+      else if ("float".equals(t) || "sfloat".equals(t))
+        fieldNameVsType.put(n, Types.FLOAT);
+      else if ("double".equals(t) || "sdouble".equals(t))
+        fieldNameVsType.put(n, Types.DOUBLE);
+      else if ("date".equals(t))
+        fieldNameVsType.put(n, Types.DATE);
+      else if ("boolean".equals(t))
+        fieldNameVsType.put(n, Types.BOOLEAN);
+      else
+        fieldNameVsType.put(n, Types.VARCHAR);
+    }
+  }
+
+  private void createConnectionFactory(final Context context,
+                                       final Properties initProps) {
+
+    final String url = initProps.getProperty(URL);
+    String driver = initProps.getProperty(DRIVER);
+
+    if (url == null)
+      throw new DataImportHandlerException(DataImportHandlerException.SEVERE,
+              "JDBC URL cannot be null");
+
+    try {
+      if (driver != null)
+        Class.forName(driver);
+    } catch (ClassNotFoundException e) {
+      throw new DataImportHandlerException(DataImportHandlerException.SEVERE,
+              "driver could not be loaded");
+    }
+    factory = new Callable<Connection>() {
+      public Connection call() throws Exception {
+        LOG.info("Creating a connection for entity "
+                + context.getEntityAttribute(DataImporter.NAME) + " with URL: "
+                + url);
+        long start = System.currentTimeMillis();
+        Connection c = DriverManager.getConnection(url, initProps);
+        LOG.info("Time taken for getConnection(): "
+                + (System.currentTimeMillis() - start));
+        return c;
+      }
+    };
+  }
+
+  public Iterator<Map<String, Object>> getData(String query) {
+    ResultSetIterator r = new ResultSetIterator(query);
+    return r.getIterator();
+  }
+
+  private void logError(String msg, Exception e) {
+    LOG.log(Level.WARNING, msg, e);
+  }
+
+  private List<String> readFieldNames(ResultSetMetaData metaData)
+          throws SQLException {
+    List<String> colNames = new ArrayList<String>();
+    int count = metaData.getColumnCount();
+    for (int i = 0; i < count; i++) {
+      colNames.add(metaData.getColumnLabel(i + 1));
+    }
+    return colNames;
+  }
+
+  private class ResultSetIterator {
+    ResultSet resultSet;
+
+    Statement stmt = null;
+
+    List<String> colNames;
+
+    Iterator<Map<String, Object>> rSetIterator;
+
+    public ResultSetIterator(String query) {
+
+      try {
+        Connection c = getConnection();
+        stmt = c.createStatement(ResultSet.TYPE_FORWARD_ONLY,
+                ResultSet.CONCUR_READ_ONLY);
+        stmt.setFetchSize(batchSize);
+        LOG.finer("Executing SQL: " + query);
+        long start = System.currentTimeMillis();
+        if (stmt.execute(query)) {
+          resultSet = stmt.getResultSet();
+        }
+        LOG.finest("Time taken for sql :"
+                + (System.currentTimeMillis() - start));
+        colNames = readFieldNames(resultSet.getMetaData());
+      } catch (Exception e) {
+        throw new DataImportHandlerException(DataImportHandlerException.SEVERE,
+                "Unable to execute query: " + query, e);
+      }
+      if (resultSet == null) {
+        rSetIterator = new ArrayList<Map<String, Object>>().iterator();
+        return;
+      }
+
+      rSetIterator = new Iterator<Map<String, Object>>() {
+        public boolean hasNext() {
+          return hasnext();
+        }
+
+        public Map<String, Object> next() {
+          return getARow();
+        }
+
+        public void remove() {/* do nothing */
+        }
+      };
+    }
+
+    private Iterator<Map<String, Object>> getIterator() {
+      return rSetIterator;
+    }
+
+    private Map<String, Object> getARow() {
+      if (resultSet == null)
+        return null;
+      Map<String, Object> result = new HashMap<String, Object>();
+      for (String colName : colNames) {
+        try {
+          if (!convertType) {
+            // Use underlying database's type information
+            result.put(colName, resultSet.getObject(colName));
+            continue;
+          }
+
+          Integer type = fieldNameVsType.get(colName);
+          if (type == null)
+            type = 12;
+          switch (type) {
+            case Types.INTEGER:
+              result.put(colName, resultSet.getInt(colName));
+              break;
+            case Types.FLOAT:
+              result.put(colName, resultSet.getFloat(colName));
+              break;
+            case Types.BIGINT:
+              result.put(colName, resultSet.getLong(colName));
+              break;
+            case Types.DOUBLE:
+              result.put(colName, resultSet.getDouble(colName));
+              break;
+            case Types.DATE:
+              result.put(colName, resultSet.getDate(colName));
+              break;
+            case Types.BOOLEAN:
+              result
+                      .put(colName, resultSet.getBoolean(colName));
+              break;
+            default:
+              result.put(colName, resultSet.getString(colName));
+              break;
+          }
+        } catch (SQLException e) {
+          logError("Error reading data ", e);
+          throw new DataImportHandlerException(
+                  DataImportHandlerException.SEVERE,
+                  "Error reading data from database", e);
+        }
+      }
+      return result;
+    }
+
+    private boolean hasnext() {
+      if (resultSet == null)
+        return false;
+      try {
+        if (resultSet.next()) {
+          return true;
+        } else {
+          close();
+          return false;
+        }
+      } catch (SQLException e) {
+        logError("Error reading data ", e);
+        close();
+        return false;
+      }
+    }
+
+    private void close() {
+      try {
+        if (resultSet != null)
+          resultSet.close();
+        if (stmt != null)
+          stmt.close();
+
+      } catch (Exception e) {
+        logError("Exception while closing result set", e);
+      } finally {
+        resultSet = null;
+        stmt = null;
+      }
+    }
+  }
+
+  private Connection getConnection() throws Exception {
+    long currTime = System.currentTimeMillis();
+    if (currTime - connLastUsed > CONN_TIME_OUT) {
+      synchronized (this) {
+        Connection tmpConn = factory.call();
+        finalize();
+        connLastUsed = System.currentTimeMillis();
+        return conn = tmpConn;
+      }
+
+    } else {
+      connLastUsed = currTime;
+      return conn;
+    }
+  }
+
+  protected void finalize() {
+    try {
+      conn.close();
+    } catch (Exception e) {
+    }
+  }
+
+  public void close() {
+    try {
+      conn.close();
+    } catch (Exception e) {
+    }
+
+  }
+
+  private static final long CONN_TIME_OUT = 10 * 1000; // 10 seconds
+
+  private static final int FETCH_SIZE = 500;
+
+  public static final String URL = "url";
+
+  public static final String DRIVER = "driver";
+
+  public static final String CONVERT_TYPE = "convertType";
+}

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/JdbcDataSource.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/JdbcDataSource.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/MockDataSource.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/MockDataSource.java?rev=681182&view=auto
==============================================================================
--- lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/MockDataSource.java (added)
+++ lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/MockDataSource.java Wed Jul 30 12:35:58 2008
@@ -0,0 +1,59 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.solr.handler.dataimport;
+
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ * <p>
+ * A mock DataSource implementation which can be used for testing.
+ * </p>
+ * <p/>
+ * <b>This API is experimental and may change in the future.</b>
+ *
+ * @version $Id$
+ * @since solr 1.3
+ */
+public class MockDataSource extends
+        DataSource<Iterator<Map<String, Object>>> {
+
+  private static Map<String, Iterator<Map<String, Object>>> cache = new HashMap<String, Iterator<Map<String, Object>>>();
+
+  public static void setIterator(String query,
+                                 Iterator<Map<String, Object>> iter) {
+    cache.put(query, iter);
+  }
+
+  public static void clearCache() {
+    cache.clear();
+  }
+
+  public void init(Context context, Properties initProps) {
+  }
+
+  public Iterator<Map<String, Object>> getData(String query) {
+    return cache.get(query);
+  }
+
+  public void close() {
+    cache.clear();
+
+  }
+}

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/MockDataSource.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/MockDataSource.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/NumberFormatTransformer.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/NumberFormatTransformer.java?rev=681182&view=auto
==============================================================================
--- lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/NumberFormatTransformer.java (added)
+++ lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/NumberFormatTransformer.java Wed Jul 30 12:35:58 2008
@@ -0,0 +1,97 @@
+package org.apache.solr.handler.dataimport;
+
+import java.text.NumberFormat;
+import java.text.ParseException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * <p>
+ * A Transformer instance which can extract numbers out of strings. It uses
+ * <code>java.text.NumberFormat</code> class to parse strings and supports
+ * Number, Integer, Currency and Percent styles as supported by
+ * <code>java.text.NumberFormat</code>
+ * </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 may change in the future.</b>
+ *
+ * @version $Id$
+ * @since solr 1.3
+ */
+public class NumberFormatTransformer extends Transformer {
+
+  @SuppressWarnings("unchecked")
+  public Object transformRow(Map<String, Object> row, Context context) {
+    for (Map<String, String> fld : context.getAllEntityFields()) {
+      String style = fld.get(FORMAT_STYLE);
+      if (style != null) {
+        String column = fld.get(DataImporter.COLUMN);
+        String srcCol = fld.get(RegexTransformer.SRC_COL_NAME);
+        if (srcCol == null)
+          srcCol = column;
+
+        Object val = row.get(srcCol);
+        String styleSmall = style.toLowerCase();
+
+        if (val instanceof List) {
+          List<String> inputs = (List) val;
+          List results = new ArrayList();
+          for (String input : inputs) {
+            try {
+              results.add(process(input, styleSmall));
+            } catch (ParseException e) {
+              throw new DataImportHandlerException(
+                      DataImportHandlerException.SEVERE,
+                      "Failed to apply NumberFormat on column: " + column, e);
+            }
+          }
+          row.put(column, results);
+        } else {
+          if (val == null || val.toString().trim().equals(""))
+            continue;
+          try {
+            row.put(column, process(val.toString(), styleSmall));
+          } catch (ParseException e) {
+            throw new DataImportHandlerException(
+                    DataImportHandlerException.SEVERE,
+                    "Failed to apply NumberFormat on column: " + column, e);
+          }
+        }
+      }
+    }
+    return row;
+  }
+
+  private Number process(String val, String style) throws ParseException {
+    if (INTEGER.equals(style)) {
+      return NumberFormat.getIntegerInstance().parse(val);
+    } else if (NUMBER.equals(style)) {
+      return NumberFormat.getNumberInstance().parse(val);
+    } else if (CURRENCY.equals(style)) {
+      return NumberFormat.getCurrencyInstance().parse(val);
+    } else if (PERCENT.equals(style)) {
+      return NumberFormat.getPercentInstance().parse(val);
+    }
+
+    return null;
+  }
+
+  public static final String FORMAT_STYLE = "formatStyle";
+
+  public static final String LOCALE = "locale";
+
+  public static final String NUMBER = "number";
+
+  public static final String PERCENT = "percent";
+
+  public static final String INTEGER = "integer";
+
+  public static final String CURRENCY = "currency";
+}

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/NumberFormatTransformer.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/NumberFormatTransformer.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/RegexTransformer.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/RegexTransformer.java?rev=681182&view=auto
==============================================================================
--- lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/RegexTransformer.java (added)
+++ lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/RegexTransformer.java Wed Jul 30 12:35:58 2008
@@ -0,0 +1,146 @@
+/**
+ * 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 java.util.*;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * <p>
+ * A Transformer implementation which uses Regular Expressions to extract, split
+ * and replace data in fields.
+ * </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 may change in the future.</b>
+ *
+ * @version $Id$
+ * @since solr 1.3
+ */
+public class RegexTransformer extends Transformer {
+  private static final Logger LOG = Logger.getLogger(RegexTransformer.class
+          .getName());
+
+  @SuppressWarnings("unchecked")
+  public Map<String, Object> transformRow(Map<String, Object> row,
+                                          Context context) {
+    List<Map<String, String>> fields = context.getAllEntityFields();
+    for (Map<String, String> field : fields) {
+      String col = field.get(DataImporter.COLUMN);
+      String reStr = field.get(REGEX);
+      String splitBy = field.get(SPLIT_BY);
+      String replaceWith = field.get(REPLACE_WITH);
+      if (reStr != null || splitBy != null) {
+        String srcColName = field.get(SRC_COL_NAME);
+        if (srcColName == null) {
+          srcColName = col;
+        }
+        Object tmpVal = row.get(srcColName);
+        if (tmpVal == null)
+          continue;
+
+        if (tmpVal instanceof List) {
+          List<String> inputs = (List<String>) tmpVal;
+          List results = new ArrayList();
+          for (String input : inputs) {
+            Object o = process(col, reStr, splitBy, replaceWith, input);
+            if (o != null)
+              results.add(o);
+          }
+          row.put(col, results);
+        } else {
+          String value = tmpVal.toString();
+          Object o = process(col, reStr, splitBy, replaceWith, value);
+          if (o != null)
+            row.put(col, o);
+        }
+      }
+    }
+    return row;
+  }
+
+  private Object process(String col, String reStr, String splitBy,
+                         String replaceWith, String value) {
+    if (splitBy != null) {
+      return readBySplit(splitBy, value);
+    } else if (replaceWith != null) {
+      Pattern p = getPattern(reStr);
+      return p.matcher(value).replaceAll(replaceWith);
+    } else {
+      return readfromRegExp(reStr, value, col);
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  private List<String> readBySplit(String splitBy, String value) {
+    String[] vals = value.split(splitBy);
+    List<String> l = new ArrayList<String>();
+    l.addAll(Arrays.asList(vals));
+    return l;
+  }
+
+  @SuppressWarnings("unchecked")
+  private Object readfromRegExp(String reStr, String value, String columnName) {
+    Pattern regexp = getPattern(reStr);
+    Matcher m = regexp.matcher(value);
+    if (m.find() && m.groupCount() > 0) {
+      if (m.groupCount() > 1) {
+        List l = new ArrayList();
+        for (int i = 1; i <= m.groupCount(); i++) {
+          try {
+            l.add(m.group(i));
+          } catch (Exception e) {
+            LOG.log(Level.WARNING, "Parsing failed for field : " + columnName,
+                    e);
+          }
+        }
+        return l;
+      } else {
+        return m.group(1);
+      }
+    }
+
+    return null;
+  }
+
+  private Pattern getPattern(String reStr) {
+    Pattern result = PATTERN_CACHE.get(reStr);
+    if (result == null) {
+      PATTERN_CACHE.put(reStr, result = Pattern.compile(reStr));
+    }
+    return result;
+  }
+
+  private HashMap<String, Pattern> PATTERN_CACHE = new HashMap<String, Pattern>();
+
+  public static final String REGEX = "regex";
+
+  public static final String REPLACE_WITH = "replaceWith";
+
+  public static final String SPLIT_BY = "splitBy";
+
+  public static final String SRC_COL_NAME = "sourceColName";
+
+}

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/RegexTransformer.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/RegexTransformer.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ScriptTransformer.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ScriptTransformer.java?rev=681182&view=auto
==============================================================================
--- lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ScriptTransformer.java (added)
+++ lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ScriptTransformer.java Wed Jul 30 12:35:58 2008
@@ -0,0 +1,102 @@
+/**
+ * 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 java.lang.reflect.InvocationTargetException;
+import java.lang.reflect.Method;
+import java.util.Map;
+
+/**
+ * <p>
+ * A Transformer instance capable of executing functions written in scripting
+ * languages as a Transformer instance.
+ * </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 may change in the future.</b>
+ *
+ * @version $Id$
+ * @since solr 1.3
+ */
+public class ScriptTransformer extends Transformer {
+  private Object engine;
+
+  private Method invokeFunctionMethod;
+
+  private String functionName;
+
+  public Object transformRow(Map<String, Object> row, Context context) {
+    try {
+      if (engine == null)
+        initEngine(context);
+      if (engine == null)
+        return row;
+      return invokeFunctionMethod.invoke(engine, functionName, new Object[]{
+              row, context});
+    } catch (DataImportHandlerException e) {
+      throw e;
+    } catch (InvocationTargetException e) {
+      throw new DataImportHandlerException(DataImportHandlerException.SEVERE,
+              "Could not invoke method :"
+                      + functionName
+                      + "\n <script>\n"
+                      + context.getVariableResolver().resolve(
+                      DataConfig.IMPORTER_NS + "." + DataConfig.SCRIPT)
+                      + "</script>", e);
+    } catch (Exception e) {
+      throw new DataImportHandlerException(DataImportHandlerException.SEVERE,
+              "Error invoking script for entity "
+                      + context.getEntityAttribute("name"), e);
+    }
+  }
+
+  private void initEngine(Context context) {
+    try {
+      String scriptText = (String) context.getVariableResolver().resolve(
+              DataConfig.IMPORTER_NS + "." + DataConfig.SCRIPT);
+      String scriptLang = (String) context.getVariableResolver().resolve(
+              DataConfig.IMPORTER_NS + "." + DataConfig.SCRIPT_LANG);
+      Object scriptEngineMgr = Class
+              .forName("javax.script.ScriptEngineManager").newInstance();
+      // create a Script engine
+      Method getEngineMethod = scriptEngineMgr.getClass().getMethod(
+              "getEngineByName", String.class);
+      engine = getEngineMethod.invoke(scriptEngineMgr, scriptLang);
+      Method evalMethod = engine.getClass().getMethod("eval", String.class);
+      invokeFunctionMethod = engine.getClass().getMethod("invokeFunction",
+              String.class, Object[].class);
+      evalMethod.invoke(engine, scriptText);
+    } catch (Exception e) {
+      throw new DataImportHandlerException(DataImportHandlerException.SEVERE,
+              "<script> can be used only in java 6 or above", e);
+    }
+  }
+
+  public void setFunctionName(String methodName) {
+    this.functionName = methodName;
+  }
+
+  public String getFunctionName() {
+    return functionName;
+  }
+
+}

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ScriptTransformer.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ScriptTransformer.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/SolrWriter.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/SolrWriter.java?rev=681182&view=auto
==============================================================================
--- lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/SolrWriter.java (added)
+++ lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/SolrWriter.java Wed Jul 30 12:35:58 2008
@@ -0,0 +1,289 @@
+/**
+ * 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.apache.lucene.document.Document;
+import org.apache.solr.update.AddUpdateCommand;
+import org.apache.solr.update.CommitUpdateCommand;
+import org.apache.solr.update.DeleteUpdateCommand;
+import org.apache.solr.update.UpdateHandler;
+
+import java.io.*;
+import java.text.ParseException;
+import java.util.Date;
+import java.util.Properties;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+
+/**
+ * <p>
+ * Writes documents to SOLR as well as provides methods for loading and
+ * persisting last index time.
+ * </p>
+ * <p/>
+ * <b>This API is experimental and may change in the future.</b>
+ *
+ * @version $Id$
+ * @since solr 1.3
+ */
+public abstract class SolrWriter {
+  private static final Logger LOG = Logger
+          .getLogger(SolrWriter.class.getName());
+
+  static final String IMPORTER_PROPERTIES = "dataimport.properties";
+
+  static final String LAST_INDEX_KEY = "last_index_time";
+
+  private final UpdateHandler updater;
+
+  private final String configDir;
+
+  public SolrWriter(UpdateHandler updater, String confDir) {
+    this.updater = updater;
+    configDir = confDir;
+
+  }
+
+  public boolean upload(Document d) {
+    try {
+      AddUpdateCommand command = new AddUpdateCommand();
+      command.doc = d;
+      command.allowDups = false;
+      command.overwritePending = true;
+      command.overwriteCommitted = true;
+      updater.addDoc(command);
+    } catch (IOException e) {
+      LOG.log(Level.SEVERE, "Exception while adding: " + d, e);
+      return false;
+    } catch (Exception e) {
+      LOG.log(Level.WARNING, "Error creating document : " + d);
+      return false;
+    }
+
+    return true;
+  }
+
+  public void deleteDoc(Object id) {
+    try {
+      LOG.info("deleted from document to Solr: " + id);
+      DeleteUpdateCommand delCmd = new DeleteUpdateCommand();
+      delCmd.id = id.toString();
+      delCmd.fromPending = true;
+      delCmd.fromCommitted = true;
+      updater.delete(delCmd);
+    } catch (IOException e) {
+      LOG.log(Level.SEVERE, "Exception while deleteing: " + id, e);
+    }
+  }
+
+  Date getStartTime() {
+    Properties props = readIndexerProperties();
+    String result = props.getProperty(SolrWriter.LAST_INDEX_KEY);
+
+    try {
+      if (result != null)
+        return DataImporter.DATE_TIME_FORMAT.parse(result);
+    } catch (ParseException e) {
+      throw new DataImportHandlerException(DataImportHandlerException.WARN,
+              "Unable to read last indexed time from: "
+                      + SolrWriter.IMPORTER_PROPERTIES, e);
+    }
+    return null;
+  }
+
+  private void persistStartTime(Date date) {
+    OutputStream propOutput = null;
+
+    Properties props = readIndexerProperties();
+
+    try {
+      props.put(SolrWriter.LAST_INDEX_KEY, DataImporter.DATE_TIME_FORMAT
+              .format(date));
+      String filePath = configDir;
+      if (configDir != null && !configDir.endsWith(File.separator))
+        filePath += File.separator;
+      filePath += SolrWriter.IMPORTER_PROPERTIES;
+      propOutput = new FileOutputStream(filePath);
+      props.store(propOutput, null);
+      LOG.info("Wrote last indexed time to " + SolrWriter.IMPORTER_PROPERTIES);
+    } catch (FileNotFoundException e) {
+      throw new DataImportHandlerException(DataImportHandlerException.SEVERE,
+              "Unable to persist Index Start Time", e);
+    } catch (IOException e) {
+      throw new DataImportHandlerException(DataImportHandlerException.SEVERE,
+              "Unable to persist Index Start Time", e);
+    } finally {
+      try {
+        if (propOutput != null)
+          propOutput.close();
+      } catch (IOException e) {
+        propOutput = null;
+      }
+    }
+  }
+
+  private Properties readIndexerProperties() {
+    Properties props = new Properties();
+    InputStream propInput = null;
+
+    try {
+      propInput = new FileInputStream(configDir
+              + SolrWriter.IMPORTER_PROPERTIES);
+      props.load(propInput);
+      LOG.info("Read " + SolrWriter.IMPORTER_PROPERTIES);
+    } catch (Exception e) {
+      LOG.log(Level.WARNING, "Unable to read: "
+              + SolrWriter.IMPORTER_PROPERTIES);
+    } finally {
+      try {
+        if (propInput != null)
+          propInput.close();
+      } catch (IOException e) {
+        propInput = null;
+      }
+    }
+
+    return props;
+  }
+
+  public void deleteByQuery(String query) {
+    try {
+      LOG.info("Deleting documents from Solr with query: " + query);
+      DeleteUpdateCommand delCmd = new DeleteUpdateCommand();
+      delCmd.query = query;
+      delCmd.fromCommitted = true;
+      delCmd.fromPending = true;
+      updater.deleteByQuery(delCmd);
+    } catch (IOException e) {
+      LOG.log(Level.SEVERE, "Exception while deleting by query: " + query, e);
+    }
+  }
+
+  public void commit(boolean optimize) {
+    try {
+      CommitUpdateCommand commit = new CommitUpdateCommand(optimize);
+      updater.commit(commit);
+    } catch (Exception e) {
+      LOG.log(Level.SEVERE, "Exception while solr commit.", e);
+    }
+  }
+
+  public void doDeleteAll() {
+    try {
+      DeleteUpdateCommand deleteCommand = new DeleteUpdateCommand();
+      deleteCommand.query = "*:*";
+      deleteCommand.fromCommitted = true;
+      deleteCommand.fromPending = true;
+      updater.deleteByQuery(deleteCommand);
+    } catch (IOException e) {
+      throw new DataImportHandlerException(DataImportHandlerException.SEVERE,
+              "Exception in full dump while deleting all documents.", e);
+    }
+  }
+
+  static String getResourceAsString(InputStream in) throws IOException {
+    ByteArrayOutputStream baos = new ByteArrayOutputStream(1024);
+    byte[] buf = new byte[1024];
+    int sz = 0;
+    try {
+      while (true) {
+        sz = in.read(buf);
+        baos.write(buf, 0, sz);
+        if (sz < buf.length)
+          break;
+      }
+    } finally {
+      try {
+        in.close();
+      } catch (Exception e) {
+
+      }
+    }
+    return new String(baos.toByteArray());
+  }
+
+  static String getDocCount() {
+    if (DocBuilder.INSTANCE.get() != null) {
+      return ""
+              + (DocBuilder.INSTANCE.get().importStatistics.docCount.get() + 1);
+    } else {
+      return "";
+    }
+  }
+
+  public Date loadIndexStartTime() {
+    return this.getStartTime();
+  }
+
+  public Class loadClass(String name) throws ClassNotFoundException {
+    return Class.forName(name);
+  }
+
+  /**
+   * <p>
+   * Stores the last indexed time into the <code>IMPORTER_PROPERTIES</code>
+   * file. If any properties are already defined in the file, then they are
+   * preserved.
+   * </p>
+   *
+   * @param date
+   */
+  public void persistIndexStartTime(Date date) {
+    this.persistStartTime(date);
+  }
+
+  public abstract SolrDoc getSolrDocInstance();
+
+  /**
+   * <p>
+   * Write the document to the index
+   * </p>
+   *
+   * @param d . The Document warapper object
+   * @return
+   */
+  public abstract boolean upload(SolrDoc d);
+
+  /**
+   * This method is used for verbose debugging
+   *
+   * @param event The event name start.entity ,end.entity ,transformer.row
+   * @param name  Name of the entity/transformer
+   * @param row   The actual data . Can be a Map<String,object> or a List<Map<String,object>>
+   */
+  public abstract void log(int event, String name, Object row);
+
+  /**
+   * The purpose of this interface to provide pluggable implementations for Solr
+   * 1.2 & 1.3 The implementation can choose to wrap appropriate Objects based
+   * on the version
+   */
+  public static interface SolrDoc {
+
+    public void addField(String name, Object value, float boost);
+
+    public Object getField(String field);
+
+    public void setDocumentBoost(float boost);
+  }
+
+  public static final int START_ENTITY = 1, END_ENTITY = 2,
+          TRANSFORMED_ROW = 3, ENTITY_META = 4, PRE_TRANSFORMER_ROW = 5,
+          START_DOC = 6, END_DOC = 7, ENTITY_OUT = 8, ROW_END = 9,
+          TRANSFORMER_EXCEPTION = 10, ENTITY_EXCEPTION = 11, DISABLE_LOGGING = 12,
+          ENABLE_LOGGING = 13;
+}

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/SolrWriter.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/SolrWriter.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/SqlEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/SqlEntityProcessor.java?rev=681182&view=auto
==============================================================================
--- lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/SqlEntityProcessor.java (added)
+++ lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/SqlEntityProcessor.java Wed Jul 30 12:35:58 2008
@@ -0,0 +1,171 @@
+/**
+ * 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 java.util.Iterator;
+import java.util.Map;
+import java.util.logging.Level;
+import java.util.logging.Logger;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * <p>
+ * An EntityProcessor instance which provides support for reading from
+ * databases. It is used in conjunction with JdbcDataSource. This is the default
+ * EntityProcessor if none is specified explicitly in 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 may change in the future.</b>
+ *
+ * @version $Id$
+ * @since solr 1.3
+ */
+public class SqlEntityProcessor extends EntityProcessorBase {
+  private static final Logger LOG = Logger.getLogger(SqlEntityProcessor.class
+          .getName());
+
+  protected DataSource<Iterator<Map<String, Object>>> dataSource;
+
+  @SuppressWarnings("unchecked")
+  public void init(Context context) {
+    super.init(context);
+    dataSource = context.getDataSource();
+  }
+
+  protected void initQuery(String q) {
+    try {
+      DataImporter.QUERY_COUNT.get().incrementAndGet();
+      rowIterator = dataSource.getData(q);
+      this.query = q;
+    } catch (DataImportHandlerException e) {
+      throw e;
+    } catch (Exception e) {
+      LOG.log(Level.SEVERE, "The query failed '" + q + "'", e);
+      throw new DataImportHandlerException(DataImportHandlerException.SEVERE, e);
+    }
+  }
+
+  public Map<String, Object> nextRow() {
+    if (rowcache != null)
+      return getFromRowCache();
+    if (rowIterator == null) {
+      String q = getQuery();
+      initQuery(resolver.replaceTokens(q));
+    }
+    while (true) {
+      Map<String, Object> r = getNext();
+      if (r == null)
+        return null;
+      r = applyTransformer(r);
+      if (r != null)
+        return r;
+    }
+
+  }
+
+  public Map<String, Object> nextModifiedRowKey() {
+    if (rowIterator == null) {
+      String deltaQuery = context.getEntityAttribute(DELTA_QUERY);
+      if (deltaQuery == null)
+        return null;
+      initQuery(resolver.replaceTokens(deltaQuery));
+    }
+    return getNext();
+  }
+
+  public Map<String, Object> nextDeletedRowKey() {
+    if (rowIterator == null) {
+      String deletedPkQuery = context.getEntityAttribute(DEL_PK_QUERY);
+      if (deletedPkQuery == null)
+        return null;
+      initQuery(resolver.replaceTokens(deletedPkQuery));
+    }
+    return getNext();
+  }
+
+  public Map<String, Object> nextModifiedParentRowKey() {
+    if (rowIterator == null) {
+      String parentDeltaQuery = context.getEntityAttribute(PARENT_DELTA_QUERY);
+      if (parentDeltaQuery == null)
+        return null;
+      LOG.info("Running parentDeltaQuery for Entity: "
+              + context.getEntityAttribute("name"));
+      initQuery(resolver.replaceTokens(parentDeltaQuery));
+    }
+    return getNext();
+  }
+
+  public String getQuery() {
+    String queryString = context.getEntityAttribute(QUERY);
+    if (context.currentProcess() == Context.FULL_DUMP
+            || !context.isRootEntity()) {
+      return queryString;
+    }
+    return getDeltaImportQuery(queryString);
+  }
+
+  public String getDeltaImportQuery(String queryString) {
+    StringBuffer sb = new StringBuffer(queryString);
+    if (SELECT_WHERE_PATTERN.matcher(queryString).find()) {
+      sb.append(" and ");
+    } else {
+      sb.append(" where ");
+    }
+    boolean first = true;
+    String[] primaryKeys = context.getEntityAttribute("pk").split(",");
+    for (String primaryKey : primaryKeys) {
+      if (!first) {
+        sb.append(" and ");
+      }
+      first = false;
+      Object val = resolver.resolve("dataimporter.delta." + primaryKey);
+      if (val == null) {
+        Matcher m = DOT_PATTERN.matcher(primaryKey);
+        if (m.find()) {
+          val = resolver.resolve("dataimporter.delta." + m.group(1));
+        }
+      }
+      sb.append(primaryKey).append(" = ");
+      if (val instanceof Number) {
+        sb.append(val.toString());
+      } else {
+        sb.append("'").append(val.toString()).append("'");
+      }
+    }
+    return sb.toString();
+  }
+
+  private static Pattern SELECT_WHERE_PATTERN = Pattern.compile(
+          "^\\s*(select\\b.*?\\b)(where).*", Pattern.CASE_INSENSITIVE);
+
+  public static final String QUERY = "query";
+
+  public static final String DELTA_QUERY = "deltaQuery";
+
+  public static final String PARENT_DELTA_QUERY = "parentDeltaQuery";
+
+  public static final String DEL_PK_QUERY = "deletedPkQuery";
+
+  public static final Pattern DOT_PATTERN = Pattern.compile(".*?\\.(.*)$");
+}

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/SqlEntityProcessor.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/SqlEntityProcessor.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/TemplateString.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/TemplateString.java?rev=681182&view=auto
==============================================================================
--- lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/TemplateString.java (added)
+++ lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/TemplateString.java Wed Jul 30 12:35:58 2008
@@ -0,0 +1,115 @@
+/**
+ * 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 java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+/**
+ * <p>
+ * Provides functionality for replacing variables in a templatized string. It
+ * can also be used to get the place-holders (variables) in a templatized
+ * string.
+ * </p>
+ * <p/>
+ * <b>This API is experimental and may change in the future.</b>
+ *
+ * @version $Id$
+ * @since solr 1.3
+ */
+public class TemplateString {
+  private List<String> variables = new ArrayList<String>();
+
+  private List<String> pcs = new ArrayList<String>();
+
+  private Map<String, TemplateString> cache;
+
+  public TemplateString() {
+    cache = new HashMap<String, TemplateString>();
+  }
+
+  private TemplateString(String s) {
+    Matcher m = WORD_PATTERN.matcher(s);
+    int idx = 0;
+    while (m.find()) {
+      String aparam = s.substring(m.start() + 2, m.end() - 1);
+      variables.add(aparam);
+      pcs.add(s.substring(idx, m.start()));
+      idx = m.end();
+    }
+    pcs.add(s.substring(idx));
+  }
+
+  /**
+   * Returns a string with all variables replaced by the known values. An
+   * unknown variable is replaced by an empty string.
+   *
+   * @param string
+   * @param resolver
+   * @return
+   */
+  public String replaceTokens(String string, VariableResolver resolver) {
+    TemplateString ts = cache.get(string);
+    if (ts == null) {
+      ts = new TemplateString(string);
+      cache.put(string, ts);
+    }
+    return ts.fillTokens(resolver);
+  }
+
+  private String fillTokens(VariableResolver resolver) {
+    String[] s = new String[variables.size()];
+    for (int i = 0; i < variables.size(); i++) {
+      Object val = resolver.resolve(variables.get(i));
+      s[i] = val == null ? "" : getObjectAsString(val);
+    }
+
+    StringBuffer sb = new StringBuffer();
+    for (int i = 0; i < pcs.size(); i++) {
+      sb.append(pcs.get(i));
+      if (i < s.length) {
+        sb.append(s[i]);
+      }
+    }
+
+    return sb.toString();
+  }
+
+  private String getObjectAsString(Object val) {
+    if (val instanceof java.sql.Date) {
+      java.sql.Date d = (java.sql.Date) val;
+      return DataImporter.DATE_TIME_FORMAT.format(d);
+    }
+    return val.toString();
+  }
+
+  /**
+   * Returns the variables in the given string.
+   *
+   * @param the templatized string
+   * @return the list of variables (strings) in the given templatized string.
+   */
+  public static List<String> getVariables(String s) {
+    return new TemplateString(s).variables;
+  }
+
+  static final Pattern WORD_PATTERN = Pattern.compile("(\\$\\{.*?\\})");
+}

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/TemplateString.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/TemplateString.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/TemplateTransformer.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/TemplateTransformer.java?rev=681182&view=auto
==============================================================================
--- lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/TemplateTransformer.java (added)
+++ lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/TemplateTransformer.java Wed Jul 30 12:35:58 2008
@@ -0,0 +1,108 @@
+/**
+ * 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 java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.logging.Logger;
+
+/**
+ * <p>
+ * A Transformer which can put values into a column by resolving an expression
+ * containing other columns
+ * </p>
+ * <p/>
+ * <p>
+ * For example:<br />
+ * &lt;field column="name" template="${e.lastName}, ${e.firstName}
+ * ${e.middleName}" /&gt; will produce the name by combining values from
+ * lastName, firstName and middleName fields as given in the template attribute.
+ * </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 may change in the future.</b>
+ *
+ * @version $Id$
+ * @since solr 1.3
+ */
+public class TemplateTransformer extends Transformer {
+
+  private static final Logger LOG = Logger.getLogger(TemplateTransformer.class
+          .getName());
+
+  @SuppressWarnings("unchecked")
+  public Object transformRow(Map<String, Object> row, Context context) {
+
+    String entityName = context.getEntityAttribute(DataImporter.NAME);
+
+    VariableResolverImpl resolver = (VariableResolverImpl) context
+            .getVariableResolver();
+    Map<String, Object> resolverMap = (Map<String, Object>) resolver
+            .resolve(entityName);
+
+    // Clone resolver map because the resolver map contains common fields or any
+    // others
+    // that the entity processor chooses to keep.
+    Map<String, Object> resolverMapCopy = new HashMap<String, Object>();
+    if (resolverMap != null) {
+      for (Map.Entry<String, Object> entry : resolverMap.entrySet())
+        resolverMapCopy.put(entry.getKey(), entry.getValue());
+    }
+    // Add current row to the copy of resolver map
+    for (Map.Entry<String, Object> entry : row.entrySet())
+      resolverMapCopy.put(entry.getKey(), entry.getValue());
+    // Add this copy to the namespace of the current entity in the resolver
+    resolver.addNamespace(entityName, resolverMapCopy);
+
+    for (Map<String, String> map : context.getAllEntityFields()) {
+      String expr = map.get(TEMPLATE);
+      if (expr == null)
+        continue;
+
+      String column = map.get(DataImporter.COLUMN);
+
+      // Verify if all variables can be resolved or not
+      boolean resolvable = true;
+      List<String> variables = TemplateString.getVariables(expr);
+      for (String v : variables) {
+        if (resolver.resolve(v) == null) {
+          LOG.warning("Unable to resolve variable: " + v
+                  + " while parsing expression: " + expr);
+          resolvable = false;
+        }
+      }
+
+      if (!resolvable)
+        continue;
+
+      row.put(column, resolver.replaceTokens(expr));
+    }
+
+    // Restore the original resolver map
+    resolver.addNamespace(entityName, resolverMap);
+
+    return row;
+  }
+
+  public static final String TEMPLATE = "template";
+}

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/TemplateTransformer.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/TemplateTransformer.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/Transformer.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/Transformer.java?rev=681182&view=auto
==============================================================================
--- lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/Transformer.java (added)
+++ lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/Transformer.java Wed Jul 30 12:35:58 2008
@@ -0,0 +1,52 @@
+/**
+ * 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 java.util.Map;
+
+/**
+ * <p>
+ * Use this API to implement a custom transformer for any given entity
+ * </p>
+ * <p/>
+ * <p>
+ * Implementations of this interface must provide a public no-args constructor.
+ * </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 may change in the future.</b>
+ *
+ * @version $Id$
+ * @since solr 1.3
+ */
+public abstract class Transformer {
+  /**
+   * The input is a row of data and the output has to be a new row.
+   *
+   * @param context The current context
+   * @param row     A row of data
+   * @return The changed data. It must be a Map<String, Object> if it returns
+   *         only one row or if there are multiple rows to be returned it must
+   *         be a List<Map<String, Object>>
+   */
+  public abstract Object transformRow(Map<String, Object> row, Context context);
+}

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/Transformer.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/Transformer.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/VariableResolver.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/VariableResolver.java?rev=681182&view=auto
==============================================================================
--- lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/VariableResolver.java (added)
+++ lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/VariableResolver.java Wed Jul 30 12:35:58 2008
@@ -0,0 +1,53 @@
+/**
+ * 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;
+
+/**
+ * <p>
+ * This class is more or less like a Map. But has more intelligence to resolve
+ * namespaces. Namespaces are delimited with '.' (period)
+ * </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 may change in the future.</b>
+ *
+ * @version $Id$
+ * @since solr 1.3
+ */
+public abstract class VariableResolver {
+
+  /**
+   * Resolves a given value with a name
+   *
+   * @param name
+   * @return
+   */
+  public abstract Object resolve(String name);
+
+  /**
+   * Given a String with place holders, replace them with the value tokens.
+   *
+   * @param template
+   * @return the string with the placeholders replaced with their values
+   */
+  public abstract String replaceTokens(String template);
+}

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/VariableResolver.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/VariableResolver.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/VariableResolverImpl.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/VariableResolverImpl.java?rev=681182&view=auto
==============================================================================
--- lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/VariableResolverImpl.java (added)
+++ lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/VariableResolverImpl.java Wed Jul 30 12:35:58 2008
@@ -0,0 +1,120 @@
+/**
+ * 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 java.util.HashMap;
+import java.util.Map;
+import java.util.regex.Pattern;
+
+/**
+ * <p>
+ * The default implementation of VariableResolver interface
+ * </p>
+ * <p/>
+ * <b>This API is experimental and may change in the future.</b>
+ *
+ * @version $Id$
+ * @see VariableResolver
+ * @since solr 1.3
+ */
+public class VariableResolverImpl extends VariableResolver {
+  private Map<String, Object> container = new HashMap<String, Object>();
+
+  private static final TemplateString TEMPLATE_STRING = new TemplateString();
+
+  public VariableResolverImpl() {
+  }
+
+  @SuppressWarnings("unchecked")
+  public VariableResolverImpl addNamespace(String name, Map<String, Object> map) {
+    if (name != null) {
+      String[] parts = DOT_SPLIT.split(name, 0);
+      Map ns = container;
+      for (int i = 0; i < parts.length; i++) {
+        if (i == parts.length - 1) {
+          ns.put(parts[i], map);
+        }
+        if (ns.get(parts[i]) == null) {
+          ns.put(parts[i], new HashMap());
+          ns = (Map) ns.get(parts[i]);
+        } else {
+          if (ns.get(parts[i]) instanceof Map) {
+            ns = (Map) ns.get(parts[i]);
+          } else {
+            ns.put(parts[i], new HashMap());
+            ns = (Map) ns.get(parts[i]);
+          }
+        }
+      }
+
+    } else {
+      container.putAll(map);
+    }
+    return this;
+
+  }
+
+  public void removeNamespace(String name) {
+    if (name != null)
+      container.remove(name);
+  }
+
+  public String replaceTokens(String template) {
+    return TEMPLATE_STRING.replaceTokens(template, this);
+  }
+
+  @SuppressWarnings("unchecked")
+  public Object resolve(String name) {
+    if (name == null)
+      return container;
+    if ("".equals(name))
+      return null;
+    String[] parts = DOT_SPLIT.split(name, 0);
+    Map<String, Object> namespace = container;
+    for (int i = 0; i < parts.length; i++) {
+      String thePart = parts[i];
+      if (i == parts.length - 1) {
+        return namespace.get(thePart);
+      }
+      Object temp = namespace.get(thePart);
+      if (temp == null) {
+        return namespace.get(mergeAll(parts, i));
+      } else {
+        if (temp instanceof Map) {
+          namespace = (Map) temp;
+        } else {
+          return null;
+        }
+      }
+    }
+    return null;
+  }
+
+  private String mergeAll(String[] parts, int i) {
+    if (i == parts.length - 1)
+      return parts[parts.length - 1];
+    StringBuffer sb = new StringBuffer();
+    for (int j = i; j < parts.length; j++) {
+      sb.append(parts[j]);
+      if (j < parts.length - 1)
+        sb.append(".");
+    }
+    return sb.toString();
+  }
+
+  static final Pattern DOT_SPLIT = Pattern.compile("\\.");
+}

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/VariableResolverImpl.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/VariableResolverImpl.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL

Added: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java?rev=681182&view=auto
==============================================================================
--- lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java (added)
+++ lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java Wed Jul 30 12:35:58 2008
@@ -0,0 +1,329 @@
+/**
+ * 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 javax.xml.transform.Source;
+import javax.xml.transform.TransformerException;
+import javax.xml.transform.TransformerFactory;
+import javax.xml.transform.stream.StreamResult;
+import javax.xml.transform.stream.StreamSource;
+import java.io.CharArrayReader;
+import java.io.CharArrayWriter;
+import java.io.Reader;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.logging.Logger;
+
+/**
+ * <p>
+ * An implementation of EntityProcessor which uses a streaming xpath parser to
+ * extract values out of XML documents. It is typically used in conjunction with
+ * HttpDataSource or FileDataSource.
+ * </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 may change in the future.</b>
+ *
+ * @version $Id$
+ * @see XPathRecordReader
+ * @since solr 1.3
+ */
+public class XPathEntityProcessor extends EntityProcessorBase {
+  private static final Logger LOG = Logger.getLogger(XPathEntityProcessor.class
+          .getName());
+
+  protected List<String> placeHolderVariables;
+
+  protected List<String> commonFields;
+
+  private String pk;
+
+  private XPathRecordReader xpathReader;
+
+  protected DataSource<Reader> dataSource;
+
+  protected javax.xml.transform.Transformer xslTransformer;
+
+  @SuppressWarnings("unchecked")
+  public void init(Context context) {
+    super.init(context);
+    if (xpathReader == null)
+      initXpathReader();
+    pk = context.getEntityAttribute("pk");
+    dataSource = context.getDataSource();
+
+  }
+
+  private void initXpathReader() {
+    boolean useSolrAddXml = Boolean.parseBoolean(context
+            .getEntityAttribute(USE_SOLR_ADD_SCHEMA));
+    String xslt = context.getEntityAttribute(XSL);
+    if (xslt != null) {
+      xslt = resolver.replaceTokens(xslt);
+      try {
+        Source xsltSource = new StreamSource(xslt);
+        // create an instance of TransformerFactory
+        TransformerFactory transFact = TransformerFactory.newInstance();
+        xslTransformer = transFact.newTransformer(xsltSource);
+        LOG
+                .info("Using xslTransformer: "
+                        + xslTransformer.getClass().getName());
+      } catch (Exception e) {
+        throw new DataImportHandlerException(DataImportHandlerException.SEVERE,
+                "Error initializing XSL ", e);
+      }
+    }
+
+    if (useSolrAddXml) {
+      // Support solr add documents
+      xpathReader = new XPathRecordReader("/add/doc");
+      xpathReader.addField("name", "/add/doc/field/@name", true);
+      xpathReader.addField("value", "/add/doc/field", true);
+    } else {
+      String forEachXpath = context.getEntityAttribute(FOR_EACH);
+      if (forEachXpath == null)
+        throw new DataImportHandlerException(DataImportHandlerException.SEVERE,
+                "Entity : " + context.getEntityAttribute("name")
+                        + " must have a 'forEach' attribute");
+
+      try {
+        xpathReader = new XPathRecordReader(forEachXpath);
+        for (Map<String, String> field : context.getAllEntityFields()) {
+          if (field.get(XPATH) == null)
+            continue;
+          xpathReader.addField(field.get(DataImporter.COLUMN),
+                  field.get(XPATH), Boolean.parseBoolean(field
+                  .get(DataImporter.MULTI_VALUED)));
+        }
+      } catch (RuntimeException e) {
+        throw new DataImportHandlerException(DataImportHandlerException.SEVERE,
+                "Exception while reading xpaths for fields", e);
+      }
+    }
+
+    List<String> l = TemplateString.getVariables(context
+            .getEntityAttribute(URL));
+    for (String s : l) {
+      if (s.startsWith(entityName + ".")) {
+        if (placeHolderVariables == null)
+          placeHolderVariables = new ArrayList<String>();
+        placeHolderVariables.add(s.substring(entityName.length() + 1));
+      }
+    }
+    for (Map<String, String> fld : context.getAllEntityFields()) {
+      if (fld.get(COMMON_FIELD) != null && "true".equals(fld.get(COMMON_FIELD))) {
+        if (commonFields == null)
+          commonFields = new ArrayList<String>();
+        commonFields.add(fld.get(DataImporter.COLUMN));
+      }
+    }
+
+  }
+
+  public Map<String, Object> nextRow() {
+    Map<String, Object> result;
+
+    if (!context.isRootEntity())
+      return fetchNextRow();
+
+    while (true) {
+      result = fetchNextRow();
+
+      if (result == null)
+        return null;
+
+      if (pk == null || result.get(pk) != null)
+        return result;
+    }
+  }
+
+  @SuppressWarnings("unchecked")
+  private Map<String, Object> fetchNextRow() {
+    Map<String, Object> r = null;
+    while (true) {
+      if (rowcache != null)
+        return getFromRowCache();
+      if (rowIterator == null)
+        initQuery(resolver.replaceTokens(context.getEntityAttribute(URL)));
+      r = getNext();
+      if (r == null) {
+        Object hasMore = getSessionAttribute(HAS_MORE);
+        if ("true".equals(hasMore) || Boolean.TRUE.equals(hasMore)) {
+          String url = (String) getSessionAttribute(NEXT_URL);
+          if (url == null)
+            url = context.getEntityAttribute(URL);
+          Map namespace = (Map) getSessionAttribute(entityName);
+          if (namespace != null)
+            resolver.addNamespace(entityName, namespace);
+          clearSession();
+          initQuery(resolver.replaceTokens(url));
+          r = getNext();
+          if (r == null)
+            return null;
+        } else {
+          return null;
+        }
+      }
+      r = applyTransformer(r);
+      if (r != null)
+        return readUsefulVars(r);
+    }
+  }
+
+  private void initQuery(String s) {
+    Reader data = null;
+    try {
+      final List<Map<String, Object>> rows = new ArrayList<Map<String, Object>>();
+      data = dataSource.getData(s);
+      if (xslTransformer != null) {
+        try {
+          SimpleCharArrayReader caw = new SimpleCharArrayReader();
+          xslTransformer.transform(new StreamSource(data),
+                  new StreamResult(caw));
+          data = caw.getReader();
+        } catch (TransformerException e) {
+          throw new DataImportHandlerException(
+                  DataImportHandlerException.SEVERE,
+                  "Exception in applying XSL Transformeation", e);
+        }
+      }
+      final List<Map<String, Object>> solrDocs = new ArrayList<Map<String, Object>>();
+      final boolean useSolrAddXml = Boolean.parseBoolean(context
+              .getEntityAttribute(USE_SOLR_ADD_SCHEMA));
+      xpathReader.streamRecords(data, new XPathRecordReader.Handler() {
+        @SuppressWarnings("unchecked")
+        public void handle(Map<String, Object> record, String xpath) {
+          if (useSolrAddXml) {
+            List<String> names = (List<String>) record.get("name");
+            List<String> values = (List<String>) record.get("value");
+
+            Map<String, Object> row = new HashMap<String, Object>();
+
+            for (int i = 0; i < names.size(); i++) {
+              if (row.containsKey(names.get(i))) {
+                Object existing = row.get(names.get(i));
+                if (existing instanceof List) {
+                  List list = (List) existing;
+                  list.add(values.get(i));
+                } else {
+                  List list = new ArrayList();
+                  list.add(existing);
+                  list.add(values.get(i));
+                  row.put(names.get(i), list);
+                }
+              } else {
+                row.put(names.get(i), values.get(i));
+              }
+            }
+
+            solrDocs.add(row);
+          } else {
+            record.put(XPATH_FIELD_NAME, xpath);
+            rows.add(record);
+          }
+        }
+      });
+
+      if (useSolrAddXml) {
+        rowIterator = solrDocs.iterator();
+      } else {
+        rowIterator = rows.iterator();
+      }
+    } finally {
+      try {
+        data.close();
+      } catch (Exception e) { /* Ignore */
+      }
+    }
+  }
+
+  private static class SimpleCharArrayReader extends CharArrayWriter {
+    public Reader getReader() {
+      return new CharArrayReader(super.buf, 0, super.count);
+    }
+
+  }
+
+  @SuppressWarnings("unchecked")
+  private Map<String, Object> readUsefulVars(Map<String, Object> r) {
+    Object val = r.get(HAS_MORE);
+    if (val != null)
+      setSessionAttribute(HAS_MORE, val);
+    val = r.get(NEXT_URL);
+    if (val != null)
+      setSessionAttribute(NEXT_URL, val);
+    if (placeHolderVariables != null) {
+      Map namespace = getNameSpace();
+      for (String s : placeHolderVariables) {
+        val = r.get(s);
+        if (val != null)
+          namespace.put(s, val);
+      }
+    }
+    if (commonFields != null) {
+      for (String s : commonFields) {
+        Object commonVal = r.get(s);
+        if (commonVal != null) {
+          setSessionAttribute(s, commonVal);
+          getNameSpace().put(s, commonVal);
+        } else {
+          commonVal = getSessionAttribute(s);
+          if (commonVal != null)
+            r.put(s, commonVal);
+        }
+      }
+    }
+    return r;
+
+  }
+
+  @SuppressWarnings("unchecked")
+  private Map getNameSpace() {
+    Map namespace = (Map) getSessionAttribute(entityName);
+    if (namespace == null) {
+      namespace = new HashMap();
+      setSessionAttribute(entityName, namespace);
+    }
+    return namespace;
+  }
+
+  public static final String URL = "url";
+
+  public static final String HAS_MORE = "$hasMore";
+
+  public static final String NEXT_URL = "$nextUrl";
+
+  public static final String XPATH_FIELD_NAME = "$forEach";
+
+  public static final String FOR_EACH = "forEach";
+
+  public static final String XPATH = "xpath";
+
+  public static final String COMMON_FIELD = "commonField";
+
+  public static final String USE_SOLR_ADD_SCHEMA = "useSolrAddSchema";
+
+  public static final String XSL = "xsl";
+
+}

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java
------------------------------------------------------------------------------
    svn:keywords = Date Author Id Revision HeadURL