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/11/12 10:51:13 UTC

svn commit: r713335 - in /lucene/solr/trunk/contrib/dataimporthandler: ./ src/main/java/org/apache/solr/handler/dataimport/ src/test/java/org/apache/solr/handler/dataimport/

Author: shalin
Date: Wed Nov 12 01:51:12 2008
New Revision: 713335

URL: http://svn.apache.org/viewvc?rev=713335&view=rev
Log:
SOLR-842 -- Better error handling in DataImportHandler with options to abort, skip and continue imports

Added:
    lucene/solr/trunk/contrib/dataimporthandler/src/test/java/org/apache/solr/handler/dataimport/TestErrorHandling.java   (with props)
Modified:
    lucene/solr/trunk/contrib/dataimporthandler/CHANGES.txt
    lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImportHandlerException.java
    lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/EntityProcessorBase.java
    lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java

Modified: lucene/solr/trunk/contrib/dataimporthandler/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/contrib/dataimporthandler/CHANGES.txt?rev=713335&r1=713334&r2=713335&view=diff
==============================================================================
--- lucene/solr/trunk/contrib/dataimporthandler/CHANGES.txt (original)
+++ lucene/solr/trunk/contrib/dataimporthandler/CHANGES.txt Wed Nov 12 01:51:12 2008
@@ -25,6 +25,9 @@
               instead of DataImportHandler manipulating the SQL itself.
               (Noble Paul via shalin)
 
+3. SOLR-842:  Better error handling in DataImportHandler with options to abort, skip and continue imports.
+              (Noble Paul, shalin)
+
 Optimizations
 ----------------------
 

Modified: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImportHandlerException.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImportHandlerException.java?rev=713335&r1=713334&r2=713335&view=diff
==============================================================================
--- lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImportHandlerException.java (original)
+++ lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/DataImportHandlerException.java Wed Nov 12 01:51:12 2008
@@ -18,9 +18,7 @@
 package org.apache.solr.handler.dataimport;
 
 /**
- * <p>
- * Exception class for all DataImportHandler exceptions
- * </p>
+ * <p> Exception class for all DataImportHandler exceptions </p>
  * <p/>
  * <b>This API is experimental and subject to change</b>
  * <p/>
@@ -59,5 +57,22 @@
     return errCode;
   }
 
+  public static void wrapAndThrow(int err, Exception e) {
+    if (e instanceof DataImportHandlerException) {
+      throw (DataImportHandlerException) e;
+    } else {
+      throw new DataImportHandlerException(err, e);
+    }
+  }
+
+  public static void wrapAndThrow(int err, Exception e, String msg) {
+    if (e instanceof DataImportHandlerException) {
+      throw (DataImportHandlerException) e;
+    } else {
+      throw new DataImportHandlerException(err, msg, e);
+    }
+  }
+
+
   public static final String MSG = " Processing Document # ";
 }

Modified: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/EntityProcessorBase.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/EntityProcessorBase.java?rev=713335&r1=713334&r2=713335&view=diff
==============================================================================
--- lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/EntityProcessorBase.java (original)
+++ lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/EntityProcessorBase.java Wed Nov 12 01:51:12 2008
@@ -16,20 +16,16 @@
  */
 package org.apache.solr.handler.dataimport;
 
-import java.lang.reflect.Method;
-import java.util.*;
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.*;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
+import java.lang.reflect.Method;
+import java.util.*;
+
 /**
- * <p>
- * Base class for all implementations of EntityProcessor
- * </p>
- * <p/>
- * <p>
- * Most implementations of EntityProcessor extend this base class which provides
- * common functionality.
- * </p>
+ * <p> Base class for all implementations of EntityProcessor </p> <p/> <p> Most implementations of EntityProcessor
+ * extend this base class which provides common functionality. </p>
  * <p/>
  * <b>This API is experimental and subject to change</b>
  *
@@ -39,6 +35,8 @@
 public class EntityProcessorBase extends EntityProcessor {
   private static final Logger log = LoggerFactory.getLogger(EntityProcessorBase.class);
 
+  protected boolean isFirstInit = true;
+
   protected String entityName;
 
   protected Context context;
@@ -56,14 +54,21 @@
   @SuppressWarnings("unchecked")
   private Map session;
 
+  protected String onError = ABORT;
+
   public void init(Context context) {
     rowIterator = null;
     rowcache = null;
     this.context = context;
-    entityName = context.getEntityAttribute("name");
+    if (isFirstInit) {
+      entityName = context.getEntityAttribute("name");
+      String s = context.getEntityAttribute(ON_ERROR);
+      if (s != null) onError = s;
+    }
     resolver = (VariableResolverImpl) context.getVariableResolver();
     query = null;
     session = null;
+    isFirstInit = false;
 
   }
 
@@ -101,15 +106,15 @@
             String msg = "Transformer :"
                     + trans
                     + "does not implement Transformer interface or does not have a transformRow(Map m)method";
-            log.error( msg);
+            log.error(msg);
             throw new DataImportHandlerException(
-                    DataImportHandlerException.SEVERE, msg);
+                    SEVERE, msg);
           }
           transformers.add(new ReflectionTransformer(meth, clazz, trans));
         }
       } catch (Exception e) {
-        log.error( "Unable to load Transformer: " + aTransArr, e);
-        throw new DataImportHandlerException(DataImportHandlerException.SEVERE,
+        log.error("Unable to load Transformer: " + aTransArr, e);
+        throw new DataImportHandlerException(SEVERE,
                 e);
       }
     }
@@ -138,8 +143,8 @@
       try {
         return meth.invoke(o, aRow);
       } catch (Exception e) {
-        log.warn("method invocation failed on transformer : "+ trans, e);
-        throw new DataImportHandlerException(DataImportHandlerException.WARN, e);
+        log.warn("method invocation failed on transformer : " + trans, e);
+        throw new DataImportHandlerException(WARN, e);
       }
     }
   }
@@ -189,15 +194,17 @@
           } else if (o instanceof List) {
             rows = (List) o;
           } else {
-            log.error( "Transformer must return Map<String, Object> or a List<Map<String, Object>>");
+            log.error("Transformer must return Map<String, Object> or a List<Map<String, Object>>");
           }
         }
-
-      } catch (DataImportHandlerException e) {
-        throw e;
       } catch (Exception e) {
-        log.warn( "transformer threw error", e);
-        throw new DataImportHandlerException(DataImportHandlerException.WARN, e);
+        log.warn("transformer threw error", e);
+        if (ABORT.equals(onError)) {
+          wrapAndThrow(SEVERE, e);
+        } else if (SKIP.equals(onError)) {
+          wrapAndThrow(DataImportHandlerException.SKIP, e);
+        }
+        // onError = continue
       }
     }
     if (rows == null) {
@@ -222,14 +229,13 @@
         return null;
       if (rowIterator.hasNext())
         return rowIterator.next();
-      rowIterator = null;
       query = null;
       return null;
     } catch (Exception e) {
-      log.error( "getNext() failed for query '" + query + "'", e);
-      rowIterator = null;
+      log.error("getNext() failed for query '" + query + "'", e);
       query = null;
-      throw new DataImportHandlerException(DataImportHandlerException.WARN, e);
+      wrapAndThrow(DataImportHandlerException.WARN, e);
+      return null;
     }
   }
 
@@ -259,13 +265,11 @@
   }
 
   /**
-   * For a simple implementation, this is the only method that the sub-class
-   * should implement. This is intended to stream rows one-by-one. Return null
-   * to signal end of rows
+   * For a simple implementation, this is the only method that the sub-class should implement. This is intended to
+   * stream rows one-by-one. Return null to signal end of rows
    *
-   * @return a row where the key is the name of the field and value can be any
-   *         Object or a Collection of objects. Return null to signal end of
-   *         rows
+   * @return a row where the key is the name of the field and value can be any Object or a Collection of objects. Return
+   *         null to signal end of rows
    */
   public Map<String, Object> nextRow() {
     return null;// do not do anything
@@ -324,11 +328,11 @@
   }
 
   /**
-   * If the where clause is present the cache is sql Vs Map of key Vs List of
-   * Rows. Only used by cache implementations.
+   * If the where clause is present the cache is sql Vs Map of key Vs List of Rows. Only used by cache implementations.
    *
    * @param query the query string for which cached data is to be returned
-   * @return the cached row corresponding to the given query after all variables have been resolved 
+   *
+   * @return the cached row corresponding to the given query after all variables have been resolved
    */
   protected Map<String, Object> getIdCacheData(String query) {
     Map<Object, List<Map<String, Object>>> rowIdVsRows = cacheWithWhereClause
@@ -367,12 +371,8 @@
   }
 
   /**
-   * <p>
-   * Get all the rows from the the datasource for the given query. Only used by
-   * cache implementations.
-   * </p>
-   * This <b>must</b> be implemented by sub-classes which intend to provide a
-   * cached implementation
+   * <p> Get all the rows from the the datasource for the given query. Only used by cache implementations. </p> This
+   * <b>must</b> be implemented by sub-classes which intend to provide a cached implementation
    *
    * @return the list of all rows fetched from the datasource.
    */
@@ -381,10 +381,10 @@
   }
 
   /**
-   * If where clause is not present the cache is a Map of query vs List of Rows.
-   * Only used by cache implementations.
+   * If where clause is not present the cache is a Map of query vs List of Rows. Only used by cache implementations.
    *
    * @param query string for which cached row is to be returned
+   *
    * @return the cached row corresponding to the given query
    */
   protected Map<String, Object> getSimpleCacheData(String query) {
@@ -415,5 +415,13 @@
 
   public static final String TRANSFORM_ROW = "transformRow";
 
+  public static final String ON_ERROR = "onError";
+
+  public static final String ABORT = "abort";
+
+  public static final String CONTINUE = "continue";
+
+  public static final String SKIP = "skip";
+
   public static final String SKIP_DOC = "$skipDoc";
 }

Modified: 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=713335&r1=713334&r2=713335&view=diff
==============================================================================
--- lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java (original)
+++ lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/XPathEntityProcessor.java Wed Nov 12 01:51:12 2008
@@ -16,6 +16,11 @@
  */
 package org.apache.solr.handler.dataimport;
 
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.SEVERE;
+import static org.apache.solr.handler.dataimport.DataImportHandlerException.wrapAndThrow;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 import javax.xml.transform.Source;
 import javax.xml.transform.TransformerException;
 import javax.xml.transform.TransformerFactory;
@@ -29,21 +34,13 @@
 import java.util.concurrent.BlockingQueue;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
+import java.util.concurrent.atomic.AtomicReference;
 
 /**
- * <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/> 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>
  *
@@ -68,8 +65,8 @@
 
   protected boolean useSolrAddXml = false;
 
-  protected boolean streamRows   =  false;
-  
+  protected boolean streamRows = false;
+
   private int batchSz = 1000;
 
   @SuppressWarnings("unchecked")
@@ -79,6 +76,7 @@
       initXpathReader();
     pk = context.getEntityAttribute("pk");
     dataSource = context.getDataSource();
+    rowIterator = null;
 
   }
 
@@ -86,8 +84,8 @@
     useSolrAddXml = Boolean.parseBoolean(context
             .getEntityAttribute(USE_SOLR_ADD_SCHEMA));
     streamRows = Boolean.parseBoolean(context
-        .getEntityAttribute(STREAM));
-    if(context.getEntityAttribute("batchSize") != null){
+            .getEntityAttribute(STREAM));
+    if (context.getEntityAttribute("batchSize") != null) {
       batchSz = Integer.parseInt(context.getEntityAttribute("batchSize"));
     }
     String xslt = context.getEntityAttribute(XSL);
@@ -102,7 +100,7 @@
                 .info("Using xslTransformer: "
                         + xslTransformer.getClass().getName());
       } catch (Exception e) {
-        throw new DataImportHandlerException(DataImportHandlerException.SEVERE,
+        throw new DataImportHandlerException(SEVERE,
                 "Error initializing XSL ", e);
       }
     }
@@ -115,7 +113,7 @@
     } else {
       String forEachXpath = context.getEntityAttribute(FOR_EACH);
       if (forEachXpath == null)
-        throw new DataImportHandlerException(DataImportHandlerException.SEVERE,
+        throw new DataImportHandlerException(SEVERE,
                 "Entity : " + context.getEntityAttribute("name")
                         + " must have a 'forEach' attribute");
 
@@ -126,10 +124,10 @@
             continue;
           xpathReader.addField(field.get(DataImporter.COLUMN),
                   field.get(XPATH), Boolean.parseBoolean(field
-                  .get(DataImporter.MULTI_VALUED)));
+                          .get(DataImporter.MULTI_VALUED)));
         }
       } catch (RuntimeException e) {
-        throw new DataImportHandlerException(DataImportHandlerException.SEVERE,
+        throw new DataImportHandlerException(SEVERE,
                 "Exception while reading xpaths for fields", e);
       }
     }
@@ -168,6 +166,7 @@
       if (pk == null || result.get(pk) != null)
         return result;
     }
+
   }
 
   @SuppressWarnings("unchecked")
@@ -204,74 +203,108 @@
   }
 
   private void initQuery(String s) {
-      Reader data = null;
+    Reader data = null;
+    try {
+      final List<Map<String, Object>> rows = new ArrayList<Map<String, Object>>();
       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);
+      } catch (Exception e) {
+        if (ABORT.equals(onError)) {
+          wrapAndThrow(SEVERE, e);
+        } else if (SKIP.equals(onError)) {
+          wrapAndThrow(DataImportHandlerException.SKIP, e);
+        } else {
+          LOG.warn("Failed for url : " + s, e);
+          rowIterator = Collections.EMPTY_LIST.iterator();
+          return;
+        }
+      }
+      if (xslTransformer != null) {
+        try {
+          SimpleCharArrayReader caw = new SimpleCharArrayReader();
+          xslTransformer.transform(new StreamSource(data),
+                  new StreamResult(caw));
+          data = caw.getReader();
+        } catch (TransformerException e) {
+          if (ABORT.equals(onError)) {
+            wrapAndThrow(SEVERE, e, "Exception in applying XSL Transformeation");
+          } else if (SKIP.equals(onError)) {
+            wrapAndThrow(DataImportHandlerException.SKIP, e);
+          } else {
+            LOG.warn("Failed for url : " + s, e);
+            rowIterator = Collections.EMPTY_LIST.iterator();
+            return;
           }
         }
-        if(streamRows ){
-          rowIterator = getRowIterator(data);
-        } else {
+      }
+      if (streamRows) {
+        rowIterator = getRowIterator(data, s);
+      } else {
+        try {
           xpathReader.streamRecords(data, new XPathRecordReader.Handler() {
             @SuppressWarnings("unchecked")
             public void handle(Map<String, Object> record, String xpath) {
               rows.add(readRow(record, xpath));
             }
           });
-          rowIterator = rows.iterator();
-        }
-      } finally {
-        if (!streamRows) {
-          closeIt(data);
+        } catch (Exception e) {
+          String msg = "Parsing failed for xml, url:" + s + "rows processed :" + rows.size();
+          if (rows.size() > 0) msg += "last row : " + rows.get(rows.size() - 1);
+          if (ABORT.equals(onError)) {
+            wrapAndThrow(SEVERE, e, msg);
+          } else if (SKIP.equals(onError)) {
+            LOG.warn(msg, e);
+            Map<String, Object> map = new HashMap<String, Object>();
+            map.put(SKIP_DOC, Boolean.TRUE);
+            rows.add(map);
+          } else if (CONTINUE.equals(onError)) {
+            LOG.warn(msg, e);
+          }
         }
-
+        rowIterator = rows.iterator();
+      }
+    } finally {
+      if (!streamRows) {
+        closeIt(data);
       }
+
     }
+  }
 
-    private void closeIt(Reader data) {
-      try {
-        data.close();
-      } catch (Exception e) { /* Ignore */
-      }
+  private void closeIt(Reader data) {
+    try {
+      data.close();
+    } catch (Exception e) { /* Ignore */
     }
+  }
+
   private Map<String, Object> readRow(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));
-         }
-       }
-       return row;
-     } else {
-       record.put(XPATH_FIELD_NAME, xpath);
-       return  record;
-     }
-   }
+    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));
+        }
+      }
+      return row;
+    } else {
+      record.put(XPATH_FIELD_NAME, xpath);
+      return record;
+    }
+  }
 
 
   private static class SimpleCharArrayReader extends CharArrayWriter {
@@ -313,7 +346,10 @@
     return r;
 
   }
-  private Iterator<Map<String ,Object>> getRowIterator(final Reader data){
+
+  private Iterator<Map<String, Object>> getRowIterator(final Reader data, final String s) {
+    //nothing atomic about it. I just needed a StongReference
+    final AtomicReference<Exception> exp = new AtomicReference<Exception>();
     final BlockingQueue<Map<String, Object>> blockingQueue = new ArrayBlockingQueue<Map<String, Object>>(batchSz);
     final AtomicBoolean isEnd = new AtomicBoolean(false);
     new Thread() {
@@ -322,7 +358,7 @@
           xpathReader.streamRecords(data, new XPathRecordReader.Handler() {
             @SuppressWarnings("unchecked")
             public void handle(Map<String, Object> record, String xpath) {
-              if(isEnd.get()) return ;
+              if (isEnd.get()) return;
               try {
                 blockingQueue.offer(readRow(record, xpath), 10, TimeUnit.SECONDS);
               } catch (Exception e) {
@@ -330,32 +366,52 @@
               }
             }
           });
+        } catch (Exception e) {
+          exp.set(e);
         } finally {
           closeIt(data);
           try {
             blockingQueue.offer(Collections.EMPTY_MAP, 10, TimeUnit.SECONDS);
-          } catch (Exception e) { }
+          } catch (Exception e) {
+          }
         }
       }
     }.start();
 
     return new Iterator<Map<String, Object>>() {
+      private Map<String, Object> lastRow;
+      int count = 0;
+
       public boolean hasNext() {
         return !isEnd.get();
       }
+
       public Map<String, Object> next() {
         try {
           Map<String, Object> row = blockingQueue.poll(10, TimeUnit.SECONDS);
           if (row == null || row == Collections.EMPTY_MAP) {
             isEnd.set(true);
+            if (exp.get() != null) {
+              String msg = "Parsing failed for xml, url:" + s + "rows processed in this xml:" + count;
+              if (lastRow != null) msg += "last row in this xml: " + lastRow;
+              if (ABORT.equals(onError)) {
+                wrapAndThrow(SEVERE, exp.get(), msg);
+              } else if (SKIP.equals(onError)) {
+                wrapAndThrow(DataImportHandlerException.SKIP, exp.get());
+              } else {
+                LOG.warn(msg, exp.get());
+              }
+            }
             return null;
           }
-          return row;
+          count++;
+          return lastRow = row;
         } catch (InterruptedException e) {
           isEnd.set(true);
           return null;
         }
       }
+
       public void remove() {
         /*no op*/
       }

Added: lucene/solr/trunk/contrib/dataimporthandler/src/test/java/org/apache/solr/handler/dataimport/TestErrorHandling.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/contrib/dataimporthandler/src/test/java/org/apache/solr/handler/dataimport/TestErrorHandling.java?rev=713335&view=auto
==============================================================================
--- lucene/solr/trunk/contrib/dataimporthandler/src/test/java/org/apache/solr/handler/dataimport/TestErrorHandling.java (added)
+++ lucene/solr/trunk/contrib/dataimporthandler/src/test/java/org/apache/solr/handler/dataimport/TestErrorHandling.java Wed Nov 12 01:51:12 2008
@@ -0,0 +1,159 @@
+package org.apache.solr.handler.dataimport;
+
+import java.io.Reader;
+import java.io.StringReader;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+/**
+ * Tests exception handling during imports in DataImportHandler
+ *
+ * @version $Id$
+ * @since solr 1.4
+ */
+public class TestErrorHandling extends AbstractDataImportHandlerTest {
+
+  public void testMalformedStreamingXml() throws Exception {
+    StringDataSource.xml = malformedXml;
+    super.runFullImport(dataConfigWithStreaming);
+    assertQ(req("id:1"), "//*[@numFound='1']");
+    assertQ(req("id:2"), "//*[@numFound='1']");
+  }
+
+  public void testMalformedNonStreamingXml() throws Exception {
+    StringDataSource.xml = malformedXml;
+    super.runFullImport(dataConfigWithoutStreaming);
+    assertQ(req("id:1"), "//*[@numFound='1']");
+    assertQ(req("id:2"), "//*[@numFound='1']");
+  }
+
+  public void testAbortOnError() throws Exception {
+    StringDataSource.xml = malformedXml;
+    super.runFullImport(dataConfigAbortOnError);
+    assertQ(req("*:*"), "//*[@numFound='0']");
+  }
+
+  public void testTransformerErrorContinue() throws Exception {
+    StringDataSource.xml = wellformedXml;
+    List<Map<String, Object>> rows = new ArrayList<Map<String, Object>>();
+    rows.add(createMap("id", "3", "desc", "exception-transformer"));
+    MockDataSource.setIterator("select * from foo", rows.iterator());
+    super.runFullImport(dataConfigWithTransformer);
+    assertQ(req("*:*"), "//*[@numFound='3']");
+  }
+
+  @Override
+  public String getSchemaFile() {
+    return "dataimport-schema.xml";
+  }
+
+  @Override
+  public String getSolrConfigFile() {
+    return "dataimport-solrconfig.xml";
+  }
+
+  @Override
+  public void setUp() throws Exception {
+    super.setUp();
+  }
+
+  @Override
+  public void tearDown() throws Exception {
+    super.tearDown();
+  }
+
+  public static class StringDataSource extends DataSource<Reader> {
+    public static String xml = "";
+
+    public void init(Context context, Properties initProps) {
+    }
+
+    public Reader getData(String query) {
+      return new StringReader(xml);
+    }
+
+    public void close() {
+
+    }
+  }
+
+  public static class ExceptionTransformer extends Transformer {
+    public Object transformRow(Map<String, Object> row, Context context) {
+      throw new RuntimeException("Test exception");
+    }
+  }
+
+  private String dataConfigWithStreaming = "<dataConfig>\n" +
+          "        <dataSource name=\"str\" type=\"TestErrorHandling$StringDataSource\" />" +
+          "    <document>\n" +
+          "        <entity name=\"node\" dataSource=\"str\" processor=\"XPathEntityProcessor\" url=\"test\" stream=\"true\" forEach=\"/root/node\" onError=\"skip\">\n" +
+          "            <field column=\"id\" xpath=\"/root/node/id\" />\n" +
+          "            <field column=\"desc\" xpath=\"/root/node/desc\" />\n" +
+          "        </entity>\n" +
+          "    </document>\n" +
+          "</dataConfig>";
+
+  private String dataConfigWithoutStreaming = "<dataConfig>\n" +
+          "        <dataSource name=\"str\" type=\"TestErrorHandling$StringDataSource\" />" +
+          "    <document>\n" +
+          "        <entity name=\"node\" dataSource=\"str\" processor=\"XPathEntityProcessor\" url=\"test\" forEach=\"/root/node\" onError=\"skip\">\n" +
+          "            <field column=\"id\" xpath=\"/root/node/id\" />\n" +
+          "            <field column=\"desc\" xpath=\"/root/node/desc\" />\n" +
+          "        </entity>\n" +
+          "    </document>\n" +
+          "</dataConfig>";
+
+  private String dataConfigAbortOnError = "<dataConfig>\n" +
+          "        <dataSource name=\"str\" type=\"TestErrorHandling$StringDataSource\" />" +
+          "    <document>\n" +
+          "        <entity name=\"node\" dataSource=\"str\" processor=\"XPathEntityProcessor\" url=\"test\" forEach=\"/root/node\" onError=\"abort\">\n" +
+          "            <field column=\"id\" xpath=\"/root/node/id\" />\n" +
+          "            <field column=\"desc\" xpath=\"/root/node/desc\" />\n" +
+          "        </entity>\n" +
+          "    </document>\n" +
+          "</dataConfig>";
+
+  private String dataConfigWithTransformer = "<dataConfig>\n" +
+          "        <dataSource name=\"str\" type=\"TestErrorHandling$StringDataSource\" />" +
+          "    <document>\n" +
+          "        <entity name=\"node\" dataSource=\"str\" processor=\"XPathEntityProcessor\" url=\"test\" forEach=\"/root/node\">\n" +
+          "            <field column=\"id\" xpath=\"/root/node/id\" />\n" +
+          "            <field column=\"desc\" xpath=\"/root/node/desc\" />\n" +
+          "            <entity name=\"child\" query=\"select * from foo\" transformer=\"TestErrorHandling$ExceptionTransformer\" onError=\"continue\">\n" +
+          "            </entity>" +
+          "        </entity>\n" +
+          "    </document>\n" +
+          "</dataConfig>";
+
+  private String malformedXml = "<root>\n" +
+          "    <node>\n" +
+          "        <id>1</id>\n" +
+          "        <desc>test1</desc>\n" +
+          "    </node>\n" +
+          "    <node>\n" +
+          "        <id>2</id>\n" +
+          "        <desc>test2</desc>\n" +
+          "    </node>\n" +
+          "    <node>\n" +
+          "        <id/>3</id>\n" +
+          "        <desc>test3</desc>\n" +
+          "    </node>\n" +
+          "</root>";
+
+  private String wellformedXml = "<root>\n" +
+          "    <node>\n" +
+          "        <id>1</id>\n" +
+          "        <desc>test1</desc>\n" +
+          "    </node>\n" +
+          "    <node>\n" +
+          "        <id>2</id>\n" +
+          "        <desc>test2</desc>\n" +
+          "    </node>\n" +
+          "    <node>\n" +
+          "        <id>3</id>\n" +
+          "        <desc>test3</desc>\n" +
+          "    </node>\n" +
+          "</root>";
+}

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

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