diff --git a/build.xml b/build.xml
index 7dfd15e..4854861 100644
--- a/build.xml
+++ b/build.xml
@@ -154,6 +154,7 @@
       <packageset dir="${plugins.dir}/index-anchor/src/java"/>
       <packageset dir="${plugins.dir}/index-basic/src/java"/>
       <packageset dir="${plugins.dir}/index-more/src/java"/>
+      <packageset dir="${plugins.dir}/indexer-solr/src/java"/>
       <packageset dir="${plugins.dir}/language-identifier/src/java"/>
       <packageset dir="${plugins.dir}/lib-http/src/java"/>
       <packageset dir="${plugins.dir}/lib-regex-filter/src/java"/>
@@ -201,6 +202,7 @@
    <group title="Scoring Plugins" packages="${plugins.scoring}" />
    <group title="Parse Plugins" packages="${plugins.parse}" />
    <group title="Indexing Filter Plugins" packages="${plugins.index}" />
+   <group title="Indexer Plugins" packages="${plugins.indexer}"/>
    <group title="Misc. Plugins" packages="${plugins.misc}" />
   </javadoc>
   <jar jarfile="${maven-javadoc-jar}">
@@ -566,6 +568,7 @@
       <packageset dir="${plugins.dir}/index-anchor/src/java"/>
       <packageset dir="${plugins.dir}/index-basic/src/java"/>
       <packageset dir="${plugins.dir}/index-more/src/java"/>
+      <packageset dir="${plugins.dir}/indexer-solr/src/java"/>
       <packageset dir="${plugins.dir}/language-identifier/src/java"/>
       <packageset dir="${plugins.dir}/lib-http/src/java"/>
       <packageset dir="${plugins.dir}/lib-regex-filter/src/java"/>
@@ -613,6 +616,7 @@
    <group title="Scoring Plugins" packages="${plugins.scoring}" />
    <group title="Parse Plugins" packages="${plugins.parse}" />
    <group title="Indexing Filter Plugins" packages="${plugins.index}" />
+   <group title="Indexer Plugins" packages="${plugins.indexer}"/>
    <group title="Misc. Plugins" packages="${plugins.misc}" />
   </javadoc>
   <!-- Copy the plugin.dtd file to the plugin doc-files dir -->
diff --git a/conf/log4j.properties b/conf/log4j.properties
index 710a095..4103d50 100644
--- a/conf/log4j.properties
+++ b/conf/log4j.properties
@@ -32,12 +32,9 @@ log4j.logger.org.apache.nutch.crawl.DbUpdaterJob=INFO,cmdstdout
 log4j.logger.org.apache.nutch.host.HostDbUpdateJob=INFO,cmdstdout
 log4j.logger.org.apache.nutch.fetcher.FetcherJob=INFO,cmdstdout
 log4j.logger.org.apache.nutch.parse.ParserJob=INFO,cmdstdout
-log4j.logger.org.apache.nutch.indexer.IndexerJob=INFO,cmdstdout
-log4j.logger.org.apache.nutch.indexer.solr.SolrIndexerJob=INFO,cmdstdout
-log4j.logger.org.apache.nutch.indexer.solr.SolrWriter=INFO,cmdstdout
+log4j.logger.org.apache.nutch.indexer.Index\u0130NGJob=INFO,cmdstdout
 log4j.logger.org.apache.nutch.indexer.DeleteDuplicates=INFO,cmdstdout
-log4j.logger.org.apache.nutch.indexer.solr.SolrClean=INFO,cmdstdout
-log4j.logger.org.apache.nutch.indexer.IndexCleanerJob=INFO,cmdstdout
+log4j.logger.org.apache.nutch.indexer.CleaningJob=INFO,cmdstdout
 log4j.logger.org.apache.nutch.crawl.WebTableReader=INFO,cmdstdout
 log4j.logger.org.apache.nutch.host.HostDbReader=INFO,cmdstdout
 log4j.logger.org.apache.nutch.parse.ParserChecker=INFO,cmdstdout
diff --git a/conf/nutch-default.xml b/conf/nutch-default.xml
index d28e588..5712a06 100644
--- a/conf/nutch-default.xml
+++ b/conf/nutch-default.xml
@@ -840,7 +840,7 @@
 
 <property>
   <name>plugin.includes</name>
- <value>protocol-http|urlfilter-regex|parse-(html|tika)|index-(basic|anchor)|urlnormalizer-(pass|regex|basic)|scoring-opic</value>
+ <value>protocol-http|urlfilter-regex|parse-(html|tika)|index-(basic|anchor)|indexer-solr|urlnormalizer-(pass|regex|basic)|scoring-opic</value>
  <description>Regular expression naming plugin directory names to
   include.  Any plugin not matching this expression is excluded.
   In any case you need at least include the nutch-extensionpoints plugin. By
diff --git a/default.properties b/default.properties
index 843d85f..1782839 100644
--- a/default.properties
+++ b/default.properties
@@ -142,6 +142,11 @@ plugins.index=\
    org.apache.nutch.indexer.subcollection*:\
    org.apache.nutch.indexer.tld*
 
+# Indexing Backend Plugins
+#
+plugins.indexer=\
+   org.apache.nutch.indexwriter.solr*
+
 #
 # Misc. Plugins
 #
diff --git a/src/bin/nutch b/src/bin/nutch
index 6ce4ae8..36c66e9 100755
--- a/src/bin/nutch
+++ b/src/bin/nutch
@@ -207,11 +207,17 @@ CLASS=org.apache.nutch.host.HostDbReader
 elif [ "$COMMAND" = "elasticindex" ] ; then
 CLASS=org.apache.nutch.indexer.elastic.ElasticIndexerJob
 elif [ "$COMMAND" = "solrindex" ] ; then
-CLASS=org.apache.nutch.indexer.solr.SolrIndexerJob
+CLASS="org.apache.nutch.indexer.IndexingJob -D solr.server.url=$1"
+shift
+elif [ "$COMMAND" = "index" ] ; then
+CLASS=org.apache.nutch.indexer.IndexingJob
 elif [ "$COMMAND" = "solrdedup" ] ; then
 CLASS=org.apache.nutch.indexer.solr.SolrDeleteDuplicates
 elif [ "$COMMAND" = "solrclean" ] ; then
-CLASS=org.apache.nutch.indexer.solr.SolrClean
+  CLASS="org.apache.nutch.indexer.CleaningJob -D solr.server.url=$2 $1"
+  shift; shift
+elif [ "$COMMAND" = "clean" ] ; then
+  CLASS=org.apache.nutch.indexer.CleaningJob
 elif [ "$COMMAND" = "parsechecker" ] ; then
   CLASS=org.apache.nutch.parse.ParserChecker
 elif [ "$COMMAND" = "indexchecker" ] ; then
diff --git a/src/java/org/apache/nutch/api/impl/RAMJobManager.java b/src/java/org/apache/nutch/api/impl/RAMJobManager.java
index 6ced1a4..6b396a5 100644
--- a/src/java/org/apache/nutch/api/impl/RAMJobManager.java
+++ b/src/java/org/apache/nutch/api/impl/RAMJobManager.java
@@ -33,13 +33,12 @@ import org.apache.nutch.api.JobManager;
 import org.apache.nutch.api.JobStatus;
 import org.apache.nutch.api.JobStatus.State;
 import org.apache.nutch.api.NutchApp;
-import org.apache.nutch.crawl.Crawler;
 import org.apache.nutch.crawl.DbUpdaterJob;
 import org.apache.nutch.crawl.GeneratorJob;
 import org.apache.nutch.crawl.InjectorJob;
 import org.apache.nutch.crawl.WebTableReader;
 import org.apache.nutch.fetcher.FetcherJob;
-import org.apache.nutch.indexer.solr.SolrIndexerJob;
+import org.apache.nutch.indexer.IndexingJob;
 import org.apache.nutch.metadata.Nutch;
 import org.apache.nutch.parse.ParserJob;
 import org.apache.nutch.util.NutchTool;
@@ -89,12 +88,12 @@ public class RAMJobManager implements JobManager {
   static {
     typeToClass.put(JobType.FETCH, FetcherJob.class);
     typeToClass.put(JobType.GENERATE, GeneratorJob.class);
-    typeToClass.put(JobType.INDEX, SolrIndexerJob.class);
+    typeToClass.put(JobType.INDEX, IndexingJob.class);
     typeToClass.put(JobType.INJECT, InjectorJob.class);
     typeToClass.put(JobType.PARSE, ParserJob.class);
     typeToClass.put(JobType.UPDATEDB, DbUpdaterJob.class);
     typeToClass.put(JobType.READDB, WebTableReader.class);
-    typeToClass.put(JobType.CRAWL, Crawler.class);
+    //typeToClass.put(JobType.CRAWL, Crawler.class);
   }
 
   private void addFinishedStatus(JobStatus status) {
diff --git a/src/java/org/apache/nutch/indexer/CleaningJob.java b/src/java/org/apache/nutch/indexer/CleaningJob.java
new file mode 100644
index 0000000..f785231
--- /dev/null
+++ b/src/java/org/apache/nutch/indexer/CleaningJob.java
@@ -0,0 +1,189 @@
+/*
+ * 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.nutch.indexer;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.gora.mapreduce.StringComparator;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.NullWritable;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.mapreduce.Reducer;
+import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.hadoop.util.Tool;
+import org.apache.nutch.indexer.IndexCleanerJob.CleanMapper;
+import org.apache.nutch.metadata.Nutch;
+import org.apache.nutch.storage.StorageUtils;
+import org.apache.nutch.storage.WebPage;
+import org.apache.nutch.util.NutchConfiguration;
+import org.apache.nutch.util.NutchJob;
+import org.apache.nutch.util.NutchTool;
+import org.apache.nutch.util.ToolUtil;
+
+public class CleaningJob extends NutchTool implements Tool {
+
+  public static final String ARG_COMMIT = "commit";
+  public static final Logger LOG = LoggerFactory
+      .getLogger(IndexCleanerJob.class);
+  private Configuration conf;
+
+  private static final Collection<WebPage.Field> FIELDS = new HashSet<WebPage.Field>();
+  
+  static {
+    FIELDS.add(WebPage.Field.STATUS);
+  }
+  
+  @Override
+  public Configuration getConf() {
+    return conf;
+  }
+  
+  @Override
+  public void setConf(Configuration conf) {
+    this.conf = conf;
+  }
+  
+  public Collection<WebPage.Field> getFields(Job job) {
+    Configuration conf = job.getConfiguration();
+    Collection<WebPage.Field> columns = new HashSet<WebPage.Field>(FIELDS);
+    IndexCleaningFilters filters = new IndexCleaningFilters(conf);
+    columns.addAll(filters.getFields());
+    return columns;
+  }
+  
+  public static class CleanReducer extends
+      Reducer<String, WebPage, NullWritable, NullWritable> {
+    private int numDeletes = 0;
+    private static final int NUM_MAX_DELETE_REQUEST = 1000;
+    private boolean commit;
+    IndexWriters writers = null;
+
+    @Override
+    public void setup(Context job) throws IOException {
+      Configuration conf = job.getConfiguration();
+      writers = new IndexWriters(conf);
+      try {
+        writers.open(conf);
+      } catch (IOException e) {
+        throw new RuntimeException(e);
+      }
+      commit = conf.getBoolean(ARG_COMMIT, false);
+//      try {
+//        solr = new CommonsHttpSolrServer(conf.get(SolrConstants.SERVER_URL));
+//      } catch (MalformedURLException e) {
+//        throw new IOException(e);
+//      }
+    }
+
+    public void reduce(String key, Iterable<WebPage> values, Context context)
+        throws IOException {
+      //updateRequest.deleteById(key);
+      writers.delete(key);
+      numDeletes++;
+      context.getCounter("SolrClean", "DELETED").increment(1);
+//      if (numDeletes >= NUM_MAX_DELETE_REQUEST) {
+//        try {
+//          updateRequest.process(solr);
+//          context.getCounter("SolrClean", "DELETED").increment(numDeletes);
+//
+//        } catch (SolrServerException e) {
+//          throw new IOException(e);
+//        }
+//        updateRequest = new UpdateRequest();
+//        numDeletes = 0;
+//      }
+    }
+
+    @Override
+    public void cleanup(Context context) throws IOException {
+//      try {
+//        if (numDeletes > 0) {
+//          updateRequest.process(solr);
+//          context.getCounter("SolrClean", "DELETED").increment(numDeletes);
+//          if (commit) {
+//            solr.commit();
+//          }
+//        }
+//      } catch (SolrServerException e) {
+//        throw new IOException(e);
+//      }
+      writers.close();
+      if (numDeletes > 0 && !commit) {
+        writers.commit();
+      }   
+      LOG.info("CleaningJob: deleted a total of " + numDeletes + " documents");
+    }
+  }
+
+
+  @Override
+  public Map<String, Object> run(Map<String, Object> args) throws Exception {
+    // String solrUrl = (String) args.get(Nutch.ARG_SOLR);
+    // getConf().set(SolrConstants.SERVER_URL, solrUrl);
+    getConf().setBoolean(ARG_COMMIT, (Boolean) args.get(ARG_COMMIT));
+    currentJob = new NutchJob(getConf(), "CleaningJob");
+    currentJob.getConfiguration().setClass(
+        "mapred.output.key.comparator.class", StringComparator.class,
+        RawComparator.class);
+
+    Collection<WebPage.Field> fields = getFields(currentJob);
+    StorageUtils.initMapperJob(currentJob, fields, String.class, WebPage.class,
+        CleanMapper.class);
+    currentJob.setReducerClass(CleanReducer.class);
+    currentJob.setOutputFormatClass(NullOutputFormat.class);
+    currentJob.waitForCompletion(true);
+    ToolUtil.recordJobStatus(null, currentJob, results);
+    return results;
+  }
+
+  public int delete(boolean commit) throws Exception {
+    LOG.info("CleaningJob: starting");
+    run(ToolUtil.toArgMap(IndexCleanerJob.ARG_COMMIT, commit));
+    LOG.info("CleaningJob: done");
+    return 0;
+  }
+
+  public int run(String[] args) throws Exception {
+    if (args.length < 1) {
+      System.err.println("Usage: CleaningJob [-crawlId <id>] [-noCommit]");
+      return 1;
+    }
+
+    boolean commit = true;
+    if (args.length == 3 && args[2].equals("-noCommit")) {
+      commit = false;
+    }
+    if (args.length == 3 && "-crawlId".equals(args[0])) {
+      getConf().set(Nutch.CRAWL_ID_KEY, args[1]);
+    }
+
+    return delete(commit);
+  }
+
+  public static void main(String[] args) throws Exception {
+    int result = ToolRunner.run(NutchConfiguration.create(), new CleaningJob(),
+        args);
+    System.exit(result);
+  }
+
+}
diff --git a/src/java/org/apache/nutch/indexer/IndexCleanerJob.java b/src/java/org/apache/nutch/indexer/IndexCleanerJob.java
deleted file mode 100644
index 49f2d75..0000000
--- a/src/java/org/apache/nutch/indexer/IndexCleanerJob.java
+++ /dev/null
@@ -1,121 +0,0 @@
-/*
- * 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.nutch.indexer;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Map;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.avro.util.Utf8;
-import org.apache.gora.mapreduce.GoraMapper;
-import org.apache.gora.mapreduce.StringComparator;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.mapreduce.lib.output.NullOutputFormat;
-import org.apache.hadoop.util.Tool;
-import org.apache.nutch.crawl.CrawlStatus;
-import org.apache.nutch.indexer.IndexingException;
-import org.apache.nutch.indexer.solr.SolrConstants;
-import org.apache.nutch.metadata.Nutch;
-import org.apache.nutch.storage.StorageUtils;
-import org.apache.nutch.storage.WebPage;
-import org.apache.nutch.util.NutchJob;
-import org.apache.nutch.util.NutchTool;
-import org.apache.nutch.util.ToolUtil;
-
-public abstract class IndexCleanerJob extends NutchTool implements Tool {
-
-	public static final String ARG_COMMIT = "commit";
-	public static final Logger LOG = LoggerFactory.getLogger(IndexCleanerJob.class);
-	private Configuration conf;
-
-	private static final Collection<WebPage.Field> FIELDS = new HashSet<WebPage.Field>();
-
-	static {
-		FIELDS.add(WebPage.Field.STATUS);
-	}
-
-	@Override
-	public Configuration getConf() {
-		return conf;
-	}
-
-	@Override
-	public void setConf(Configuration conf) {
-		this.conf = conf;
-	}
-
-	public static class CleanMapper extends
-			GoraMapper<String, WebPage, String, WebPage> {
-		
-		  private IndexCleaningFilters filters;
-		
-		@Override
-		  protected void setup(Context context) throws IOException {
-		    Configuration conf = context.getConfiguration();
-		    filters = new IndexCleaningFilters(conf);
-		  }
-
-		@Override
-		public void map(String key, WebPage page, Context context)
-				throws IOException, InterruptedException {
-			try {				
-				 if(page.getStatus() == CrawlStatus.STATUS_GONE || filters.remove(key, page)) {
-					context.write(key, page);
-				}
-			} catch (IndexingException e) {
-				LOG.warn("Error indexing "+key+": "+e);
-			}
-		}
-	}
-
-	public Collection<WebPage.Field> getFields(Job job) {
-		Configuration conf = job.getConfiguration();
-	    Collection<WebPage.Field> columns = new HashSet<WebPage.Field>(FIELDS);		
-	    IndexCleaningFilters filters = new IndexCleaningFilters(conf);
-	    columns.addAll(filters.getFields());
-	    return columns;
-	}	
-	
-	public abstract Class<? extends Reducer<String, WebPage, NullWritable, NullWritable>> getReducerClass();
-	
-	@Override
-	public Map<String, Object> run(Map<String, Object> args) throws Exception {
-		String solrUrl = (String) args.get(Nutch.ARG_SOLR);
-		getConf().set(SolrConstants.SERVER_URL, solrUrl);
-		getConf().setBoolean(ARG_COMMIT,(Boolean)args.get(ARG_COMMIT));
-		currentJob = new NutchJob(getConf(), "index-clean");
-		currentJob.getConfiguration().setClass(
-				"mapred.output.key.comparator.class", StringComparator.class,
-				RawComparator.class);
-
-		Collection<WebPage.Field> fields = getFields(currentJob);
-		StorageUtils.initMapperJob(currentJob, fields, String.class,
-				WebPage.class, CleanMapper.class);
-		currentJob.setReducerClass(getReducerClass());
-		currentJob.setOutputFormatClass(NullOutputFormat.class);
-		currentJob.waitForCompletion(true);
-		ToolUtil.recordJobStatus(null, currentJob, results);
-		return results;
-	}	
-
-}
diff --git a/src/java/org/apache/nutch/indexer/IndexWriter.java b/src/java/org/apache/nutch/indexer/IndexWriter.java
new file mode 100644
index 0000000..2fe8330
--- /dev/null
+++ b/src/java/org/apache/nutch/indexer/IndexWriter.java
@@ -0,0 +1,44 @@
+/*
+ * 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.nutch.indexer;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.nutch.indexer.NutchDocument;
+import org.apache.nutch.plugin.Pluggable;
+
+public interface IndexWriter extends Configurable, Pluggable {
+  /** The name of the extension point. */
+  final static String X_POINT_ID = IndexWriter.class.getName();
+  
+  public void open(Configuration job) throws IOException;
+
+  public void write(NutchDocument doc) throws IOException;
+  
+  public void delete(String key) throws IOException;
+  
+  public void update(NutchDocument doc) throws IOException;
+  
+  public void commit() throws IOException;
+
+  public void close() throws IOException;
+  
+  /** Returns a String describing the IndexWriter instance and the specific parameters it can take */
+  public String describe();
+}
diff --git a/src/java/org/apache/nutch/indexer/IndexWriters.java b/src/java/org/apache/nutch/indexer/IndexWriters.java
new file mode 100644
index 0000000..8aa2eb2
--- /dev/null
+++ b/src/java/org/apache/nutch/indexer/IndexWriters.java
@@ -0,0 +1,148 @@
+/*
+ * 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.nutch.indexer;
+
+import java.io.IOException;
+import java.util.HashMap;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+import org.apache.nutch.indexer.NutchDocument;
+import org.apache.nutch.plugin.Extension;
+import org.apache.nutch.plugin.ExtensionPoint;
+import org.apache.nutch.plugin.PluginRepository;
+import org.apache.nutch.plugin.PluginRuntimeException;
+import org.apache.nutch.util.ObjectCache;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/** Creates and caches {@link IndexWriter} implementing plugins. */
+public class IndexWriters {
+
+  public final static Logger LOG = LoggerFactory
+      .getLogger(IndexWriters.class);
+
+  private IndexWriter[] indexWriters;
+
+  public IndexWriters(Configuration conf) {
+    ObjectCache objectCache = ObjectCache.get(conf);
+    synchronized (objectCache) {
+      this.indexWriters = (IndexWriter[]) objectCache
+          .getObject(IndexWriter.class.getName());
+      if (this.indexWriters == null) {
+        try {
+          ExtensionPoint point = PluginRepository.get(conf)
+              .getExtensionPoint(IndexWriter.X_POINT_ID);
+          if (point == null)
+            throw new RuntimeException(IndexWriter.X_POINT_ID
+                + " not found.");
+          Extension[] extensions = point.getExtensions();
+          HashMap<String, IndexWriter> indexerMap = new HashMap<String, IndexWriter>();
+          for (int i = 0; i < extensions.length; i++) {
+            Extension extension = extensions[i];
+            IndexWriter writer = (IndexWriter) extension
+                .getExtensionInstance();
+            LOG.info("Adding " + writer.getClass().getName());
+            if (!indexerMap.containsKey(writer.getClass().getName())) {
+              indexerMap.put(writer.getClass().getName(), writer);
+            }
+          }
+          objectCache.setObject(IndexWriter.class.getName(), indexerMap
+              .values().toArray(new IndexWriter[0]));
+        } catch (PluginRuntimeException e) {
+          throw new RuntimeException(e);
+        }
+        this.indexWriters = (IndexWriter[]) objectCache
+            .getObject(IndexWriter.class.getName());
+      }
+    }
+  }
+
+  public void open(Configuration conf) throws IOException {
+    for (int i = 0; i < this.indexWriters.length; i++) {
+      try {
+        this.indexWriters[i].open(conf);
+      } catch (IOException ioe) {
+        throw ioe;
+      }
+    }
+  }
+
+  public void write(NutchDocument doc) throws IOException {
+    for (int i = 0; i < this.indexWriters.length; i++) {
+      try {
+        this.indexWriters[i].write(doc);
+      } catch (IOException ioe) {
+        throw ioe;
+      }
+    }
+  }
+
+  public void update(NutchDocument doc) throws IOException {
+    for (int i = 0; i < this.indexWriters.length; i++) {
+      try {
+        this.indexWriters[i].update(doc);
+      } catch (IOException ioe) {
+        throw ioe;
+      }
+    }
+  }
+
+  public void delete(String key) throws IOException {
+    for (int i = 0; i < this.indexWriters.length; i++) {
+      try {
+        this.indexWriters[i].delete(key);
+      } catch (IOException ioe) {
+        throw ioe;
+      }
+    }
+  }
+
+  public void close() throws IOException {
+    for (int i = 0; i < this.indexWriters.length; i++) {
+      try {
+        this.indexWriters[i].close();
+      } catch (IOException ioe) {
+        throw ioe;
+      }
+    }
+  }
+
+  public void commit() throws IOException {
+    for (int i = 0; i < this.indexWriters.length; i++) {
+      try {
+        this.indexWriters[i].commit();
+      } catch (IOException ioe) {
+        throw ioe;
+      }
+    }
+  }
+
+  // lists the active IndexWriters and their configuration
+  public String describe() throws IOException {
+    StringBuffer buffer = new StringBuffer();
+    if (this.indexWriters.length == 0)
+      buffer.append("No IndexWriters activated - check your configuration\n");
+    else
+      buffer.append("Active IndexWriters :\n");
+    for (int i = 0; i < this.indexWriters.length; i++) {
+      buffer.append(this.indexWriters[i].describe()).append("\n");
+    }
+    return buffer.toString();
+  }
+
+}
\ No newline at end of file
diff --git a/src/java/org/apache/nutch/indexer/IndexerJob.java b/src/java/org/apache/nutch/indexer/IndexerJob.java
deleted file mode 100644
index 2d57b20..0000000
--- a/src/java/org/apache/nutch/indexer/IndexerJob.java
+++ /dev/null
@@ -1,145 +0,0 @@
-/*******************************************************************************
- * 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.nutch.indexer;
-
-import java.io.IOException;
-import java.util.Collection;
-import java.util.HashSet;
-
-import org.apache.avro.util.Utf8;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.RawComparator;
-import org.apache.hadoop.mapred.Counters.Counter;
-import org.apache.hadoop.mapreduce.CounterGroup;
-import org.apache.hadoop.mapreduce.Counters;
-import org.apache.hadoop.mapreduce.Job;
-import org.apache.hadoop.util.Tool;
-import org.apache.nutch.crawl.GeneratorJob;
-import org.apache.nutch.metadata.Nutch;
-import org.apache.nutch.parse.ParseStatusCodes;
-import org.apache.nutch.parse.ParseStatusUtils;
-import org.apache.nutch.scoring.ScoringFilters;
-import org.apache.nutch.storage.Mark;
-import org.apache.nutch.storage.ParseStatus;
-import org.apache.nutch.storage.StorageUtils;
-import org.apache.nutch.storage.WebPage;
-import org.apache.nutch.util.NutchJob;
-import org.apache.nutch.util.NutchTool;
-import org.apache.nutch.util.TableUtil;
-import org.apache.gora.mapreduce.GoraMapper;
-import org.apache.gora.mapreduce.StringComparator;
-import org.apache.gora.store.DataStore;
-
-public abstract class IndexerJob extends NutchTool implements Tool {
-
-  public static final Logger LOG = LoggerFactory.getLogger(IndexerJob.class);
-
-  private static final Collection<WebPage.Field> FIELDS = new HashSet<WebPage.Field>();
-
-  private static final Utf8 REINDEX = new Utf8("-reindex");
-
-  static {
-    FIELDS.add(WebPage.Field.SIGNATURE);
-    FIELDS.add(WebPage.Field.PARSE_STATUS);
-    FIELDS.add(WebPage.Field.SCORE);
-    FIELDS.add(WebPage.Field.MARKERS);
-  }
-  
-  public static class IndexerMapper
-      extends GoraMapper<String, WebPage, String, NutchDocument> {
-    public IndexUtil indexUtil;
-    public DataStore<String, WebPage> store;
-    
-    protected Utf8 batchId;
-
-    @Override
-    public void setup(Context context) throws IOException {
-      Configuration conf = context.getConfiguration();
-      batchId = new Utf8(conf.get(GeneratorJob.BATCH_ID, Nutch.ALL_BATCH_ID_STR));
-      indexUtil = new IndexUtil(conf);
-      try {
-        store = StorageUtils.createWebStore(conf, String.class, WebPage.class);
-      } catch (ClassNotFoundException e) {
-        throw new IOException(e);
-      }
-    }
-    
-    protected void cleanup(Context context) throws IOException ,InterruptedException {
-      store.close();
-    };
-
-    @Override
-    public void map(String key, WebPage page, Context context)
-    throws IOException, InterruptedException {
-      ParseStatus pstatus = page.getParseStatus();
-      if (pstatus == null || !ParseStatusUtils.isSuccess(pstatus)
-          || pstatus.getMinorCode() == ParseStatusCodes.SUCCESS_REDIRECT) {
-        return; // filter urls not parsed
-      }
-
-      Utf8 mark = Mark.UPDATEDB_MARK.checkMark(page);
-      if (!batchId.equals(REINDEX)) {
-        if (!NutchJob.shouldProcess(mark, batchId)) {
-          if (LOG.isDebugEnabled()) {
-            LOG.debug("Skipping " + TableUtil.unreverseUrl(key) + "; different batch id (" + mark + ")");
-          }
-          return;
-        }
-      }
-      
-      NutchDocument doc = indexUtil.index(key, page);
-      if (doc == null) {
-        return;
-      }
-      if (mark != null) {
-        Mark.INDEX_MARK.putMark(page, Mark.UPDATEDB_MARK.checkMark(page));
-        store.put(key, page);
-      }
-      context.write(key, doc);
-      context.getCounter("IndexerJob", "DocumentCount").increment(1);
-    }
-  }
-
-
-  private static Collection<WebPage.Field> getFields(Job job) {
-    Configuration conf = job.getConfiguration();
-    Collection<WebPage.Field> columns = new HashSet<WebPage.Field>(FIELDS);
-    IndexingFilters filters = new IndexingFilters(conf);
-    columns.addAll(filters.getFields());
-    ScoringFilters scoringFilters = new ScoringFilters(conf);
-    columns.addAll(scoringFilters.getFields());
-    return columns;
-  }
-
-  protected Job createIndexJob(Configuration conf, String jobName, String batchId)
-  throws IOException, ClassNotFoundException {
-    conf.set(GeneratorJob.BATCH_ID, batchId);
-    Job job = new NutchJob(conf, jobName);
-    // TODO: Figure out why this needs to be here
-    job.getConfiguration().setClass("mapred.output.key.comparator.class",
-        StringComparator.class, RawComparator.class);
-
-    Collection<WebPage.Field> fields = getFields(job);
-    StorageUtils.initMapperJob(job, fields, String.class, NutchDocument.class,
-        IndexerMapper.class);
-    job.setNumReduceTasks(0);
-    job.setOutputFormatClass(IndexerOutputFormat.class);
-    return job;
-  }
-}
diff --git a/src/java/org/apache/nutch/indexer/IndexerOutputFormat.java b/src/java/org/apache/nutch/indexer/IndexerOutputFormat.java
index 7e3c3c2..7b468bb 100644
--- a/src/java/org/apache/nutch/indexer/IndexerOutputFormat.java
+++ b/src/java/org/apache/nutch/indexer/IndexerOutputFormat.java
@@ -30,29 +30,29 @@ public class IndexerOutputFormat extends OutputFormat<String, NutchDocument> {
   public RecordWriter<String, NutchDocument> getRecordWriter(
       TaskAttemptContext job) throws IOException, InterruptedException {
 
-    final NutchIndexWriter[] writers =
-      NutchIndexWriterFactory.getNutchIndexWriters(job.getConfiguration());
-
-    for (final NutchIndexWriter writer : writers) {
-      writer.open(job);
-    }
+    //final IndexWriter[] writers =
+    //  NutchIndexWriterFactory.getNutchIndexWriters(job.getConfiguration());
 
+    final IndexWriters writers = new IndexWriters(job.getConfiguration());
+    
+//    for (final IndexWriter writer : writers) {
+//      writer.open(job);
+//    }
+    writers.open(job.getConfiguration());
+    
     return new RecordWriter<String, NutchDocument>() {
 
       @Override
       public void write(String key, NutchDocument doc) throws IOException {
-        for (final NutchIndexWriter writer : writers) {
-          writer.write(doc);
-        }
+        // TODO: Check Write Status for delete or write.  
+        writers.write(doc);
       }
 
       @Override
       public void close(TaskAttemptContext context) throws IOException,
       InterruptedException {
-        for (final NutchIndexWriter writer : writers) {
-          writer.close();
+          writers.close();
         }
-      }
     };
   }
 
diff --git a/src/java/org/apache/nutch/indexer/IndexingJob.java b/src/java/org/apache/nutch/indexer/IndexingJob.java
new file mode 100644
index 0000000..057ac54
--- /dev/null
+++ b/src/java/org/apache/nutch/indexer/IndexingJob.java
@@ -0,0 +1,212 @@
+/*
+ * 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.nutch.indexer;
+
+import java.io.IOException;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Map;
+
+import org.apache.avro.util.Utf8;
+import org.apache.gora.mapreduce.GoraMapper;
+import org.apache.gora.mapreduce.StringComparator;
+import org.apache.gora.store.DataStore;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.io.RawComparator;
+import org.apache.hadoop.mapreduce.Job;
+import org.apache.hadoop.util.StringUtils;
+import org.apache.hadoop.util.Tool;
+import org.apache.hadoop.util.ToolRunner;
+import org.apache.nutch.crawl.GeneratorJob;
+import org.apache.nutch.indexer.solr.SolrConstants;
+import org.apache.nutch.metadata.Nutch;
+import org.apache.nutch.parse.ParseStatusCodes;
+import org.apache.nutch.parse.ParseStatusUtils;
+import org.apache.nutch.scoring.ScoringFilters;
+import org.apache.nutch.storage.Mark;
+import org.apache.nutch.storage.ParseStatus;
+import org.apache.nutch.storage.StorageUtils;
+import org.apache.nutch.storage.WebPage;
+import org.apache.nutch.util.NutchConfiguration;
+import org.apache.nutch.util.NutchJob;
+import org.apache.nutch.util.NutchTool;
+import org.apache.nutch.util.TableUtil;
+import org.apache.nutch.util.ToolUtil;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class IndexingJob extends NutchTool implements Tool {
+
+  public static Logger LOG = LoggerFactory.getLogger(IndexingJob.class);
+
+  private static final Collection<WebPage.Field> FIELDS = new HashSet<WebPage.Field>();
+
+  private static final Utf8 REINDEX = new Utf8("-reindex");
+
+  static {
+    FIELDS.add(WebPage.Field.SIGNATURE);
+    FIELDS.add(WebPage.Field.PARSE_STATUS);
+    FIELDS.add(WebPage.Field.SCORE);
+    FIELDS.add(WebPage.Field.MARKERS);
+  }
+
+  public static class IndexerMapper extends
+      GoraMapper<String, WebPage, String, NutchDocument> {
+    public IndexUtil indexUtil;
+    public DataStore<String, WebPage> store;
+
+    protected Utf8 batchId;
+
+    @Override
+    public void setup(Context context) throws IOException {
+      Configuration conf = context.getConfiguration();
+      batchId = new Utf8(
+          conf.get(GeneratorJob.BATCH_ID, Nutch.ALL_BATCH_ID_STR));
+      indexUtil = new IndexUtil(conf);
+      try {
+        store = StorageUtils.createWebStore(conf, String.class, WebPage.class);
+      } catch (ClassNotFoundException e) {
+        throw new IOException(e);
+      }
+    }
+
+    protected void cleanup(Context context) throws IOException,
+        InterruptedException {
+      store.close();
+    };
+
+    @Override
+    public void map(String key, WebPage page, Context context)
+        throws IOException, InterruptedException {
+      ParseStatus pstatus = page.getParseStatus();
+      if (pstatus == null || !ParseStatusUtils.isSuccess(pstatus)
+          || pstatus.getMinorCode() == ParseStatusCodes.SUCCESS_REDIRECT) {
+        return; // filter urls not parsed
+      }
+
+      Utf8 mark = Mark.UPDATEDB_MARK.checkMark(page);
+      if (!batchId.equals(REINDEX)) {
+        if (!NutchJob.shouldProcess(mark, batchId)) {
+          if (LOG.isDebugEnabled()) {
+            LOG.debug("Skipping " + TableUtil.unreverseUrl(key)
+                + "; different batch id (" + mark + ")");
+          }
+          return;
+        }
+      }
+
+      NutchDocument doc = indexUtil.index(key, page);
+      if (doc == null) {
+        return;
+      }
+      if (mark != null) {
+        Mark.INDEX_MARK.putMark(page, Mark.UPDATEDB_MARK.checkMark(page));
+        store.put(key, page);
+      }
+      context.write(key, doc);
+      context.getCounter("IndexerJob", "DocumentCount").increment(1);
+    }
+  }
+
+  private static Collection<WebPage.Field> getFields(Job job) {
+    Configuration conf = job.getConfiguration();
+    Collection<WebPage.Field> columns = new HashSet<WebPage.Field>(FIELDS);
+    IndexingFilters filters = new IndexingFilters(conf);
+    columns.addAll(filters.getFields());
+    ScoringFilters scoringFilters = new ScoringFilters(conf);
+    columns.addAll(scoringFilters.getFields());
+    return columns;
+  }
+
+  @Override
+  public Map<String, Object> run(Map<String, Object> args) throws Exception {
+    String batchId = (String) args.get(Nutch.ARG_BATCH);
+
+    // NutchIndexWriterFactory.addClassToConf(getConf(), SolrWriter.class);
+
+    // NOW PASSED ON THE COMMAND LINE AS A HADOOP PARAM
+    // getConf().set(SolrConstants.SERVER_URL, solrUrl);
+
+    Configuration conf = getConf();
+    conf.set(GeneratorJob.BATCH_ID, batchId);
+
+    Job job = new NutchJob(conf, "Indexer");
+    // TODO: Figure out why this needs to be here
+    job.getConfiguration().setClass("mapred.output.key.comparator.class",
+        StringComparator.class, RawComparator.class);
+
+    Collection<WebPage.Field> fields = getFields(job);
+    StorageUtils.initMapperJob(job, fields, String.class, NutchDocument.class,
+        IndexerMapper.class);
+    job.setNumReduceTasks(0);
+    job.setOutputFormatClass(IndexerOutputFormat.class);
+
+    // currentJob = createIndexJob(getConf(), "Indexer", batchId);
+
+    job.waitForCompletion(true);
+    ToolUtil.recordJobStatus(null, job, results);
+    return results;
+  }
+
+  public void index(String batchId) throws Exception {
+    LOG.info("IndexerJob: starting");
+
+    run(ToolUtil.toArgMap(Nutch.ARG_BATCH, batchId));
+    // NOW PASSED ON THE COMMAND LINE AS A HADOOP PARAM
+    // do the commits once and for all the reducers in one go
+    // getConf().set(SolrConstants.SERVER_URL,solrUrl);
+
+    IndexWriters writers = new IndexWriters(getConf());
+    LOG.info(writers.describe());
+    
+    writers.open(getConf());
+    // SolrServer solr = SolrUtils.getCommonsHttpSolrServer(getConf());
+    if (getConf().getBoolean(SolrConstants.COMMIT_INDEX, true)) {
+      // solr.commit();
+      writers.commit();
+    }
+    LOG.info("SolrIndexerJob: done.");
+    // LOG.info("SolrIndexerJob: done. Total "+ SolrWriter.documentCount +
+    // (SolrWriter.documentCount > 1 ? " documents are " : " document is ") +
+    // "added.");
+  }
+
+  public int run(String[] args) throws Exception {
+    if (args.length < 1) {
+      System.err
+          .println("Usage: IndexerJob (<batchId> | -all | -reindex) [-crawlId <id>]");
+      return -1;
+    }
+
+    if (args.length == 3 && "-crawlId".equals(args[1])) {
+      getConf().set(Nutch.CRAWL_ID_KEY, args[2]);
+    }
+    try {
+      index(args[0]);
+      return 0;
+    } catch (final Exception e) {
+      LOG.error("SolrIndexerJob: " + StringUtils.stringifyException(e));
+      return -1;
+    }
+  }
+
+  public static void main(String[] args) throws Exception {
+    final int res = ToolRunner.run(NutchConfiguration.create(),
+        new IndexingJob(), args);
+    System.exit(res);
+  }
+}
diff --git a/src/java/org/apache/nutch/indexer/NutchIndexWriter.java b/src/java/org/apache/nutch/indexer/NutchIndexWriter.java
deleted file mode 100644
index 1e52bdc..0000000
--- a/src/java/org/apache/nutch/indexer/NutchIndexWriter.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * 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.nutch.indexer;
-
-import java.io.IOException;
-
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.nutch.indexer.NutchDocument;
-
-public interface NutchIndexWriter {
-  public void open(TaskAttemptContext job) throws IOException;
-
-  public void write(NutchDocument doc) throws IOException;
-
-  public void close() throws IOException;
-
-}
diff --git a/src/java/org/apache/nutch/indexer/NutchIndexWriterFactory.java b/src/java/org/apache/nutch/indexer/NutchIndexWriterFactory.java
deleted file mode 100644
index a81bb7b..0000000
--- a/src/java/org/apache/nutch/indexer/NutchIndexWriterFactory.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * 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.nutch.indexer;
-
-import org.apache.hadoop.conf.Configuration;
-
-public class NutchIndexWriterFactory {
-  @SuppressWarnings("unchecked")
-  public static NutchIndexWriter[] getNutchIndexWriters(Configuration conf) {
-    final String[] classes = conf.getStrings("indexer.writer.classes");
-    final NutchIndexWriter[] writers = new NutchIndexWriter[classes.length];
-    for (int i = 0; i < classes.length; i++) {
-      final String clazz = classes[i];
-      try {
-        final Class<NutchIndexWriter> implClass =
-          (Class<NutchIndexWriter>) Class.forName(clazz);
-        writers[i] = implClass.newInstance();
-      } catch (final Exception e) {
-        throw new RuntimeException("Couldn't create " + clazz, e);
-      }
-    }
-    return writers;
-  }
-
-  public static void addClassToConf(Configuration conf,
-                                    Class<? extends NutchIndexWriter> clazz) {
-    final String classes = conf.get("indexer.writer.classes");
-    final String newClass = clazz.getName();
-
-    if (classes == null) {
-      conf.set("indexer.writer.classes", newClass);
-    } else {
-      conf.set("indexer.writer.classes", classes + "," + newClass);
-    }
-
-  }
-
-}
diff --git a/src/java/org/apache/nutch/indexer/elastic/ElasticConstants.java b/src/java/org/apache/nutch/indexer/elastic/ElasticConstants.java
deleted file mode 100644
index ff0b964..0000000
--- a/src/java/org/apache/nutch/indexer/elastic/ElasticConstants.java
+++ /dev/null
@@ -1,26 +0,0 @@
-/*
- * 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.nutch.indexer.elastic;
-
-public interface ElasticConstants {
-  public static final String ELASTIC_PREFIX = "elastic.";
-
-  public static final String CLUSTER = ELASTIC_PREFIX + "cluster";
-  public static final String INDEX = ELASTIC_PREFIX + "index";
-  public static final String MAX_BULK_DOCS = ELASTIC_PREFIX + "max.bulk.docs";
-  public static final String MAX_BULK_LENGTH = ELASTIC_PREFIX + "max.bulk.size";
-}
diff --git a/src/java/org/apache/nutch/indexer/elastic/ElasticIndexerJob.java b/src/java/org/apache/nutch/indexer/elastic/ElasticIndexerJob.java
deleted file mode 100644
index b597962..0000000
--- a/src/java/org/apache/nutch/indexer/elastic/ElasticIndexerJob.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/*
- * 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.nutch.indexer.elastic;
-
-import java.util.Map;
-
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.nutch.indexer.IndexerJob;
-import org.apache.nutch.indexer.NutchIndexWriterFactory;
-import org.apache.nutch.metadata.Nutch;
-import org.apache.nutch.util.NutchConfiguration;
-import org.apache.nutch.util.ToolUtil;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Indexer for elasticsearch. Uses bulk operations with flushing in the background,
- * keeping track of elasticsearch responses by checking after every flush. When a 
- * previous flush has not finished yet before the next bulk is full, it will wait for it.
- * This mechanism will keep the servers from overloading.
- */
-public class ElasticIndexerJob extends IndexerJob {
-
-  public static Logger LOG = LoggerFactory.getLogger(ElasticIndexerJob.class);
-
-  @Override
-  public Map<String,Object> run(Map<String,Object> args) throws Exception {
-    LOG.info("Starting");
-    
-    NutchIndexWriterFactory.addClassToConf(getConf(), ElasticWriter.class);
-    String batchId = (String)args.get(Nutch.ARG_BATCH);    
-    String clusterName = (String)args.get(ElasticConstants.CLUSTER);
-    
-    getConf().set(ElasticConstants.CLUSTER, clusterName);
-    
-    currentJob = createIndexJob(getConf(), "elastic-index [" + clusterName + "]", batchId);
-    
-    currentJob.waitForCompletion(true);
-    ToolUtil.recordJobStatus(null, currentJob, results);
-    
-    LOG.info("Done");
-    return results;
-  }
-
-  public void indexElastic(String clusterName, String batchId) throws Exception {
-    run(ToolUtil.toArgMap(ElasticConstants.CLUSTER, clusterName,
-                          Nutch.ARG_BATCH, batchId));
-  }
-
-  public int run(String[] args) throws Exception {
-    if (args.length == 2) {
-      //ok
-    } else if (args.length == 4 && "-crawlId".equals(args[2])) {
-      getConf().set(Nutch.CRAWL_ID_KEY, args[3]);
-    } else {
-      System.err.println("Usage: <elastic cluster name> (<batchId> | -all | -reindex) [-crawlId <id>]");
-      return -1;      
-    }
-    indexElastic(args[0], args[1]);
-    return 0;
-  }
-
-  public static void main(String[] args) throws Exception {
-    int res = ToolRunner.run(NutchConfiguration.create(), new ElasticIndexerJob(), args);
-    System.exit(res);
-  }
-}
diff --git a/src/java/org/apache/nutch/indexer/elastic/ElasticWriter.java b/src/java/org/apache/nutch/indexer/elastic/ElasticWriter.java
deleted file mode 100644
index caeee7b..0000000
--- a/src/java/org/apache/nutch/indexer/elastic/ElasticWriter.java
+++ /dev/null
@@ -1,163 +0,0 @@
-/*
- * 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.nutch.indexer.elastic;
-
-import static org.elasticsearch.node.NodeBuilder.nodeBuilder;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.nutch.indexer.NutchDocument;
-import org.apache.nutch.indexer.NutchIndexWriter;
-import org.apache.nutch.util.TableUtil;
-import org.elasticsearch.action.ListenableActionFuture;
-import org.elasticsearch.action.bulk.BulkItemResponse;
-import org.elasticsearch.action.bulk.BulkRequestBuilder;
-import org.elasticsearch.action.bulk.BulkResponse;
-import org.elasticsearch.action.index.IndexRequestBuilder;
-import org.elasticsearch.client.Client;
-import org.elasticsearch.node.Node;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class ElasticWriter implements NutchIndexWriter {
-
-  public static Logger LOG = LoggerFactory.getLogger(ElasticWriter.class);
-
-  private static final int DEFAULT_MAX_BULK_DOCS = 500;
-  private static final int DEFAULT_MAX_BULK_LENGTH = 5001001; // ~5MB
-
-  private Client client;
-  private Node node;
-  private String defaultIndex;
-
-  private BulkRequestBuilder bulk;
-  private ListenableActionFuture<BulkResponse> execute;
-  private int maxBulkDocs;
-  private int maxBulkLength;
-  private long indexedDocs = 0;
-  private int bulkDocs = 0;
-  private int bulkLength = 0;
-
-  @Override
-  public void write(NutchDocument doc) throws IOException {
-    String id = TableUtil.reverseUrl(doc.getFieldValue("url"));
-    String type = doc.getDocumentMeta().get("type");
-    if (type == null) type = "doc";
-    IndexRequestBuilder request = client.prepareIndex(defaultIndex, type, id);
-    
-    Map<String, Object> source = new HashMap<String, Object>();
-    
-    // Loop through all fields of this doc
-    for (String fieldName : doc.getFieldNames()) {
-      if (doc.getFieldValues(fieldName).size() > 1) {
-        source.put(fieldName, doc.getFieldValues(fieldName));
-        // Loop through the values to keep track of the size of this document
-        for (String value : doc.getFieldValues(fieldName)) {
-          bulkLength += value.length();
-        }
-      } else {
-        source.put(fieldName, doc.getFieldValue(fieldName));
-        bulkLength += doc.getFieldValue(fieldName).length();
-      }
-    }
-    request.setSource(source);
-    
-    // Add this indexing request to a bulk request
-    bulk.add(request);
-    indexedDocs++;
-    bulkDocs++;
-    
-    if (bulkDocs >= maxBulkDocs || bulkLength >= maxBulkLength) {
-      LOG.info("Processing bulk request [docs = " + bulkDocs + ", length = "
-          + bulkLength + ", total docs = " + indexedDocs
-          + ", last doc in bulk = '" + id + "']");
-      // Flush the bulk of indexing requests
-      processExecute(true);
-      
-    }
-  }
-
-  private void processExecute(boolean createNewBulk) {
-    if (execute != null) {
-      // wait for previous to finish
-      long beforeWait = System.currentTimeMillis();
-      BulkResponse actionGet = execute.actionGet();
-      if (actionGet.hasFailures()) {
-        for (BulkItemResponse item : actionGet) {
-          if (item.failed()) {
-            throw new RuntimeException("First failure in bulk: "
-                + item.getFailureMessage());
-          }
-        }
-      }
-      long msWaited = System.currentTimeMillis() - beforeWait;
-      LOG.info("Previous took in ms " + actionGet.getTookInMillis()
-          + ", including wait " + msWaited);
-      execute = null;
-    }
-    if (bulk != null) {
-      if (bulkDocs > 0) {
-        // start a flush, note that this is an asynchronous call
-        execute = bulk.execute();
-      }
-      bulk = null;
-    }
-    if (createNewBulk) {
-      // Prepare a new bulk request
-      bulk = client.prepareBulk();
-      bulkDocs = 0;
-      bulkLength = 0;
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    // Flush pending requests
-    LOG.info("Processing remaining requests [docs = " + bulkDocs
-        + ", length = " + bulkLength + ", total docs = " + indexedDocs + "]");
-    processExecute(false);
-    // flush one more time to finalize the last bulk
-    LOG.info("Processing to finalize last execute");
-    processExecute(false);
-    
-    // Close
-    client.close();
-    node.close();
-  }
-
-  @Override
-  public void open(TaskAttemptContext job) throws IOException {
-    String clusterName = job.getConfiguration().get(ElasticConstants.CLUSTER);
-    if (clusterName != null) {
-      node = nodeBuilder().clusterName(clusterName).client(true).node();
-    } else {
-      node = nodeBuilder().client(true).node();
-    }
-    client = node.client();
-    
-    bulk = client.prepareBulk();
-    defaultIndex = job.getConfiguration().get(ElasticConstants.INDEX, "index");
-    maxBulkDocs = job.getConfiguration().getInt(
-        ElasticConstants.MAX_BULK_DOCS, DEFAULT_MAX_BULK_DOCS);
-    maxBulkLength = job.getConfiguration().getInt(
-        ElasticConstants.MAX_BULK_LENGTH, DEFAULT_MAX_BULK_LENGTH);
-  }
-
-}
diff --git a/src/java/org/apache/nutch/indexer/solr/SolrClean.java b/src/java/org/apache/nutch/indexer/solr/SolrClean.java
deleted file mode 100644
index 27113ec..0000000
--- a/src/java/org/apache/nutch/indexer/solr/SolrClean.java
+++ /dev/null
@@ -1,128 +0,0 @@
-/*
- * 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.nutch.indexer.solr;
-
-import java.io.IOException;
-import java.net.MalformedURLException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.avro.util.Utf8;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.io.NullWritable;
-import org.apache.hadoop.mapreduce.Reducer;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.nutch.indexer.IndexCleanerJob;
-import org.apache.nutch.metadata.Nutch;
-import org.apache.nutch.storage.WebPage;
-import org.apache.nutch.util.NutchConfiguration;
-import org.apache.nutch.util.ToolUtil;
-import org.apache.solr.client.solrj.SolrServer;
-import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
-import org.apache.solr.client.solrj.request.UpdateRequest;
-
-public class SolrClean extends IndexCleanerJob {
-
-	public static final int NUM_MAX_DELETE_REQUEST = 1000;
-	public static final Logger LOG = LoggerFactory.getLogger(SolrClean.class);	
-
-	public static class CleanReducer extends
-			Reducer<String, WebPage, NullWritable, NullWritable> {
-		private int numDeletes = 0;
-		private SolrServer solr;
-		private UpdateRequest updateRequest = new UpdateRequest();
-		private boolean commit;
-
-		@Override
-		public void setup(Context job) throws IOException {
-			Configuration conf = job.getConfiguration();
-			commit = conf.getBoolean(ARG_COMMIT, true);
-			try {
-				solr = new CommonsHttpSolrServer(
-						conf.get(SolrConstants.SERVER_URL));
-			} catch (MalformedURLException e) {
-				throw new IOException(e);
-			}
-		}
-
-		public void reduce(String key, Iterable<WebPage> values, Context context)
-				throws IOException {
-	        updateRequest.deleteById(key);
-			numDeletes++;
-			if (numDeletes >= NUM_MAX_DELETE_REQUEST) {
-				try {
-					updateRequest.process(solr);
-					context.getCounter("SolrClean", "DELETED").increment(
-							numDeletes);
-
-				} catch (SolrServerException e) {
-					throw new IOException(e);
-				}
-				updateRequest = new UpdateRequest();
-				numDeletes = 0;
-			}
-		}
-
-		@Override
-		public void cleanup(Context context) throws IOException {
-			try {
-				if (numDeletes > 0) {
-					updateRequest.process(solr);
-					context.getCounter("SolrClean", "DELETED").increment(
-							numDeletes);
-					if (commit) {
-						solr.commit();
-					}
-				}
-			} catch (SolrServerException e) {
-				throw new IOException(e);
-			}
-		}
-	}
-	
-	public Class<? extends Reducer<String, WebPage, NullWritable, NullWritable>> getReducerClass(){
-		return CleanReducer.class;
-	}	
-
-	public int delete(String solrUrl, boolean commit) throws Exception {
-		LOG.info("CleanJob: starting");
-		run(ToolUtil.toArgMap(Nutch.ARG_SOLR, solrUrl, IndexCleanerJob.ARG_COMMIT, commit));
-		LOG.info("CleanJob: done");
-		return 0;
-	}
-
-	public int run(String[] args) throws Exception {
-		if (args.length < 1) {
-			System.err.println("Usage: SolrClean <solrurl> [-noCommit]");
-			return 1;
-		}
-
-		boolean commit = true;
-		if (args.length == 2 && args[1].equals("-noCommit")) {
-			commit = false;
-		}
-
-		return delete(args[0], commit);
-	}
-
-	public static void main(String[] args) throws Exception {
-		int result = ToolRunner.run(NutchConfiguration.create(),
-				new SolrClean(), args);
-		System.exit(result);
-	}
-
-}
diff --git a/src/java/org/apache/nutch/indexer/solr/SolrIndexerJob.java b/src/java/org/apache/nutch/indexer/solr/SolrIndexerJob.java
deleted file mode 100644
index 4d7ab8c..0000000
--- a/src/java/org/apache/nutch/indexer/solr/SolrIndexerJob.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * 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.nutch.indexer.solr;
-
-import java.util.Map;
-
-import org.apache.hadoop.util.StringUtils;
-import org.apache.hadoop.util.ToolRunner;
-import org.apache.nutch.indexer.IndexerJob;
-import org.apache.nutch.indexer.NutchIndexWriterFactory;
-import org.apache.nutch.metadata.Nutch;
-import org.apache.nutch.util.NutchConfiguration;
-import org.apache.nutch.util.ToolUtil;
-import org.apache.solr.client.solrj.SolrServer;
-import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class SolrIndexerJob extends IndexerJob {
-
-  public static Logger LOG = LoggerFactory.getLogger(SolrIndexerJob.class);
-
-  @Override
-  public Map<String,Object> run(Map<String,Object> args) throws Exception {
-    String solrUrl = (String)args.get(Nutch.ARG_SOLR);
-    String batchId = (String)args.get(Nutch.ARG_BATCH);
-    NutchIndexWriterFactory.addClassToConf(getConf(), SolrWriter.class);
-    getConf().set(SolrConstants.SERVER_URL, solrUrl);
-
-    currentJob = createIndexJob(getConf(), "solr-index", batchId);
-
-    currentJob.waitForCompletion(true);
-    ToolUtil.recordJobStatus(null, currentJob, results);
-    return results;
-  }
-
-  public void indexSolr(String solrUrl, String batchId) throws Exception {
-    LOG.info("SolrIndexerJob: starting");
-
-    run(ToolUtil.toArgMap(
-        Nutch.ARG_SOLR, solrUrl,
-        Nutch.ARG_BATCH, batchId));
-    // do the commits once and for all the reducers in one go
-    getConf().set(SolrConstants.SERVER_URL,solrUrl);
-    SolrServer solr = SolrUtils.getCommonsHttpSolrServer(getConf());
-    if (getConf().getBoolean(SolrConstants.COMMIT_INDEX, true)) {
-      solr.commit();
-    }
-    LOG.info("SolrIndexerJob: done. Total "+ SolrWriter.documentCount + 
-      (SolrWriter.documentCount > 1 ? " documents are " : " document is ") + "added.");
-  }
-
-  public int run(String[] args) throws Exception {
-    if (args.length < 2) {
-      System.err.println("Usage: SolrIndexerJob <solr url> (<batchId> | -all | -reindex) [-crawlId <id>]");
-      return -1;
-    }
-
-    if (args.length == 4 && "-crawlId".equals(args[2])) {
-      getConf().set(Nutch.CRAWL_ID_KEY, args[3]);
-    }
-    try {
-      indexSolr(args[0], args[1]);
-      return 0;
-    } catch (final Exception e) {
-      LOG.error("SolrIndexerJob: " + StringUtils.stringifyException(e));
-      return -1;
-    }
-  }
-
-  public static void main(String[] args) throws Exception {
-    final int res = ToolRunner.run(NutchConfiguration.create(),
-        new SolrIndexerJob(), args);
-    System.exit(res);
-  }
-}
diff --git a/src/java/org/apache/nutch/indexer/solr/SolrMappingReader.java b/src/java/org/apache/nutch/indexer/solr/SolrMappingReader.java
deleted file mode 100644
index 360ffae..0000000
--- a/src/java/org/apache/nutch/indexer/solr/SolrMappingReader.java
+++ /dev/null
@@ -1,143 +0,0 @@
-/*
- * 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.nutch.indexer.solr;
-
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.MalformedURLException;
-import java.util.HashMap;
-import java.util.Map;
-
-import javax.xml.parsers.DocumentBuilder;
-import javax.xml.parsers.DocumentBuilderFactory;
-import javax.xml.parsers.ParserConfigurationException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.nutch.util.ObjectCache;
-import org.w3c.dom.Document;
-import org.w3c.dom.Element;
-import org.w3c.dom.NodeList;
-import org.xml.sax.InputSource;
-import org.xml.sax.SAXException;
-
-public class SolrMappingReader {
-  public static Logger LOG = LoggerFactory.getLogger(SolrMappingReader.class);
-  
-  private Configuration conf;
-  
-  private Map<String, String> keyMap = new HashMap<String, String>();
-  private Map<String, String> copyMap = new HashMap<String, String>();
-  private String uniqueKey = "id";
-  
-  public static synchronized SolrMappingReader getInstance(Configuration conf) {
-    ObjectCache cache = ObjectCache.get(conf);
-    SolrMappingReader instance = (SolrMappingReader)cache.getObject(SolrMappingReader.class.getName());
-    if (instance == null) {
-      instance = new SolrMappingReader(conf);
-      cache.setObject(SolrMappingReader.class.getName(), instance);
-    }
-    return instance;
-  }
-
-  protected SolrMappingReader(Configuration conf) {
-    this.conf = conf;
-    parseMapping();
-  }
-
-  private void parseMapping() {    
-    InputStream ssInputStream = null;
-    ssInputStream = conf.getConfResourceAsInputStream(conf.get(SolrConstants.MAPPING_FILE, "solrindex-mapping.xml"));
-
-    InputSource inputSource = new InputSource(ssInputStream);
-    try {
-      DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance();
-      DocumentBuilder builder = factory.newDocumentBuilder();
-      Document document = builder.parse(inputSource);
-      Element rootElement = document.getDocumentElement();
-      NodeList fieldList = rootElement.getElementsByTagName("field");
-      if (fieldList.getLength() > 0) {
-        for (int i = 0; i < fieldList.getLength(); i++) {
-          Element element = (Element) fieldList.item(i);
-          LOG.info("source: " + element.getAttribute("source") + " dest: " + element.getAttribute("dest"));
-          keyMap.put(element.getAttribute("source"), element.getAttribute("dest"));
-        }
-      }
-      NodeList copyFieldList = rootElement.getElementsByTagName("copyField");
-      if (copyFieldList.getLength() > 0) {
-        for (int i = 0; i < copyFieldList.getLength(); i++) {
-          Element element = (Element) copyFieldList.item(i);
-          LOG.info("source: " + element.getAttribute("source") + " dest: " + element.getAttribute("dest"));
-          copyMap.put(element.getAttribute("source"), element.getAttribute("dest"));
-        }
-      }
-      NodeList uniqueKeyItem = rootElement.getElementsByTagName("uniqueKey");
-      if (uniqueKeyItem.getLength() > 1) {
-        LOG.warn("More than one unique key definitions found in solr index mapping, using default 'id'");
-        uniqueKey = "id";
-      }
-      else if (uniqueKeyItem.getLength() == 0) {
-        LOG.warn("No unique key definition found in solr index mapping using, default 'id'");
-      }
-      else{
-    	  uniqueKey = uniqueKeyItem.item(0).getFirstChild().getNodeValue();
-      }
-    } catch (MalformedURLException e) {
-        LOG.warn(e.toString());
-    } catch (SAXException e) {
-        LOG.warn(e.toString());
-    } catch (IOException e) {
-    	LOG.warn(e.toString());
-    } catch (ParserConfigurationException e) {
-    	LOG.warn(e.toString());
-    } 
-  }
-	  
-  public Map<String, String> getKeyMap() {
-    return keyMap;
-  }
-	  
-  public Map<String, String> getCopyMap() {
-    return copyMap;
-  }
-	  
-  public String getUniqueKey() {
-    return uniqueKey;
-  }
-
-  public String hasCopy(String key) {
-    if (copyMap.containsKey(key)) {
-      key = copyMap.get(key);
-    }
-    return key;
-  }
-
-  public String mapKey(String key) throws IOException {
-    if(keyMap.containsKey(key)) {
-      key = keyMap.get(key);
-    }
-    return key;
-  }
-
-  public String mapCopyKey(String key) throws IOException {
-    if(copyMap.containsKey(key)) {
-      key = copyMap.get(key);
-    }
-    return key;
-  }
-}
diff --git a/src/java/org/apache/nutch/indexer/solr/SolrUtils.java b/src/java/org/apache/nutch/indexer/solr/SolrUtils.java
index 8d88386..1b89369 100644
--- a/src/java/org/apache/nutch/indexer/solr/SolrUtils.java
+++ b/src/java/org/apache/nutch/indexer/solr/SolrUtils.java
@@ -14,7 +14,7 @@ import java.net.MalformedURLException;
 
 public class SolrUtils {
 
-  public static Logger LOG = LoggerFactory.getLogger(SolrIndexerJob.class);
+  public static Logger LOG = LoggerFactory.getLogger(SolrUtils.class);
 
   public static CommonsHttpSolrServer getCommonsHttpSolrServer(Configuration job) throws MalformedURLException {
     HttpClient client=new HttpClient();
diff --git a/src/java/org/apache/nutch/indexer/solr/SolrWriter.java b/src/java/org/apache/nutch/indexer/solr/SolrWriter.java
deleted file mode 100644
index ce0215b..0000000
--- a/src/java/org/apache/nutch/indexer/solr/SolrWriter.java
+++ /dev/null
@@ -1,101 +0,0 @@
-/*
- * 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.nutch.indexer.solr;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map.Entry;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-import org.apache.nutch.indexer.NutchDocument;
-import org.apache.nutch.indexer.NutchIndexWriter;
-import org.apache.solr.client.solrj.SolrServer;
-import org.apache.solr.client.solrj.SolrServerException;
-import org.apache.solr.common.SolrInputDocument;
-
-public class SolrWriter implements NutchIndexWriter {
-
-  public static final Logger LOG = LoggerFactory.getLogger(SolrWriter.class);
-
-  private SolrServer solr;
-  private SolrMappingReader solrMapping;
-
-  private final List<SolrInputDocument> inputDocs =
-    new ArrayList<SolrInputDocument>();
-
-  private int commitSize; 
-  protected static long documentCount = 0;
-
-  @Override
-  public void open(TaskAttemptContext job)
-  throws IOException {
-    Configuration conf = job.getConfiguration();
-    solr = SolrUtils.getCommonsHttpSolrServer(conf);
-    commitSize = conf.getInt(SolrConstants.COMMIT_SIZE, 1000);
-    solrMapping = SolrMappingReader.getInstance(conf);
-  }
-
-  @Override
-  public void write(NutchDocument doc) throws IOException {
-    final SolrInputDocument inputDoc = new SolrInputDocument();
-    for(final Entry<String, List<String>> e : doc) {
-      for (final String val : e.getValue()) {
-
-        Object val2 = val;
-        if (e.getKey().equals("content") || e.getKey().equals("title")) {
-          val2 = SolrUtils.stripNonCharCodepoints(val);
-        }
-
-        inputDoc.addField(solrMapping.mapKey(e.getKey()), val2);
-        String sCopy = solrMapping.mapCopyKey(e.getKey());
-        if (sCopy != e.getKey()) {
-        	inputDoc.addField(sCopy, val2);
-        }
-      }
-    }
-    inputDoc.setDocumentBoost(doc.getScore());
-    inputDocs.add(inputDoc);
-    documentCount++;
-    if (inputDocs.size() >= commitSize) {
-      try {
-        LOG.info("Adding " + Integer.toString(inputDocs.size()) + " documents");
-        solr.add(inputDocs);
-      } catch (final SolrServerException e) {
-        throw new IOException(e);
-      }
-      inputDocs.clear();
-    }
-  }
-
-  @Override
-  public void close() throws IOException {
-    try {
-      if (!inputDocs.isEmpty()) {
-        LOG.info("Adding " + Integer.toString(inputDocs.size()) + " documents");
-        solr.add(inputDocs);
-        inputDocs.clear();
-      }
-    } catch (final SolrServerException e) {
-      throw new IOException(e);
-    }
-  }
-
-}
diff --git a/src/plugin/build.xml b/src/plugin/build.xml
index 7c60a63..6fbe0f6 100755
--- a/src/plugin/build.xml
+++ b/src/plugin/build.xml
@@ -30,6 +30,7 @@
      <ant dir="index-anchor" target="deploy"/>
      <ant dir="index-basic" target="deploy"/>
      <ant dir="index-more" target="deploy"/>
+     <ant dir="indexer-solr" target="deploy"/>
      <ant dir="language-identifier" target="deploy"/>
      <ant dir="lib-http" target="deploy"/>
      <ant dir="lib-nekohtml" target="deploy"/>
@@ -108,6 +109,7 @@
     <ant dir="index-anchor" target="clean"/>
     <ant dir="index-basic" target="clean"/>
     <ant dir="index-more" target="clean"/>
+    <ant dir="indexer-solr" target="clean"/>
     <ant dir="language-identifier" target="clean"/>
     <ant dir="lib-http" target="clean"/>
     <ant dir="lib-nekohtml" target="clean"/>
diff --git a/src/plugin/indexer-solr/build.xml b/src/plugin/indexer-solr/build.xml
new file mode 100644
index 0000000..8d77cdf
--- /dev/null
+++ b/src/plugin/indexer-solr/build.xml
@@ -0,0 +1,22 @@
+<?xml version="1.0"?>
+<!--
+ 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.
+-->
+<project name="indexer-solr" default="jar-core">
+
+  <import file="../build-plugin.xml" />
+
+</project>
diff --git a/src/plugin/indexer-solr/ivy.xml b/src/plugin/indexer-solr/ivy.xml
new file mode 100644
index 0000000..259e7c3
--- /dev/null
+++ b/src/plugin/indexer-solr/ivy.xml
@@ -0,0 +1,43 @@
+<?xml version="1.0" ?>
+
+<!--
+   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.
+-->
+
+<ivy-module version="1.0">
+  <info organisation="org.apache.nutch" module="${ant.project.name}">
+    <license name="Apache 2.0"/>
+    <ivyauthor name="Apache Nutch Team" url="http://nutch.apache.org"/>
+    <description>
+        Apache Nutch
+    </description>
+  </info>
+
+  <configurations>
+    <include file="../../..//ivy/ivy-configurations.xml"/>
+  </configurations>
+
+  <publications>
+    <!--get the artifact from our module name-->
+    <artifact conf="master"/>
+  </publications>
+
+  <dependencies>
+   <dependency org="org.apache.solr" name="solr-solrj" rev="3.4.0"
+		conf="*->default"/>
+  </dependencies>
+  
+</ivy-module>
diff --git a/src/plugin/indexer-solr/plugin.xml b/src/plugin/indexer-solr/plugin.xml
new file mode 100644
index 0000000..b3a263e
--- /dev/null
+++ b/src/plugin/indexer-solr/plugin.xml
@@ -0,0 +1,54 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+<plugin id="indexer-solr" name="SOLRIndexWriter" version="1.0.0"
+  provider-name="nutch.apache.org">
+
+  <runtime>
+    <library name="indexer-solr.jar">
+      <export name="*" />
+    </library>
+
+     <library name="activation-1.1.jar"/>
+     <library name="commons-codec-1.4.jar"/>
+     <library name="commons-httpclient-3.1.jar"/>
+     <library name="commons-io-1.4.jar"/>
+     <library name="commons-logging-1.1.1.jar"/>
+     <library name="geronimo-stax-api_1.0_spec-1.0.1.jar"/>
+     <library name="jline-0.9.1.jar"/>
+     <library name="log4j-1.2.15.jar"/>
+     <library name="lucene-core-3.4.0.jar"/>
+     <library name="mail-1.4.1.jar"/>
+     <library name="slf4j-api-1.6.1.jar"/>
+     <library name="solr-solrj-3.4.0.jar"/>
+     <library name="stax-api-1.0.1.jar"/>
+     <library name="wstx-asl-3.2.7.jar"/>
+     <library name="zookeeper-3.3.1.jar"/>
+  </runtime>
+
+  <requires>
+    <import plugin="nutch-extensionpoints" />
+  </requires>
+
+  <extension id="org.apache.nutch.indexwriter.solr"
+    name="SOLR Index Writer"
+    point="org.apache.nutch.indexer.IndexWriter">
+    <implementation id="SOLRIndexWriter"
+      class="org.apache.nutch.indexwriter.solr.SolrIndexWriter" />
+  </extension>
+
+</plugin>
diff --git a/src/plugin/indexer-solr/src/java/org/apache/nutch/indexwriter/solr/SolrConstants.java b/src/plugin/indexer-solr/src/java/org/apache/nutch/indexwriter/solr/SolrConstants.java
new file mode 100644
index 0000000..28d3c6c
--- /dev/null
+++ b/src/plugin/indexer-solr/src/java/org/apache/nutch/indexwriter/solr/SolrConstants.java
@@ -0,0 +1,46 @@
+/*
+ * 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.nutch.indexwriter.solr;
+
+public interface SolrConstants {
+  public static final String SOLR_PREFIX = "solr.";
+
+  public static final String SERVER_URL = SOLR_PREFIX + "server.url";
+
+  public static final String COMMIT_SIZE = SOLR_PREFIX + "commit.size";
+  
+  public static final String COMMIT_INDEX = SOLR_PREFIX + "commit.index";
+
+  public static final String MAPPING_FILE = SOLR_PREFIX + "mapping.file";
+
+  public static final String USE_AUTH = SOLR_PREFIX + "auth";
+
+  public static final String USERNAME = SOLR_PREFIX + "auth.username";
+
+  public static final String PASSWORD = SOLR_PREFIX + "auth.password";
+  
+  public static final String ID_FIELD = "id";
+  
+  public static final String URL_FIELD = "url";
+  
+  public static final String BOOST_FIELD = "boost";
+  
+  public static final String TIMESTAMP_FIELD = "tstamp";
+  
+  public static final String DIGEST_FIELD = "digest";
+
+}
diff --git a/src/plugin/indexer-solr/src/java/org/apache/nutch/indexwriter/solr/SolrIndexWriter.java b/src/plugin/indexer-solr/src/java/org/apache/nutch/indexwriter/solr/SolrIndexWriter.java
new file mode 100644
index 0000000..a2fa266
--- /dev/null
+++ b/src/plugin/indexer-solr/src/java/org/apache/nutch/indexwriter/solr/SolrIndexWriter.java
@@ -0,0 +1,169 @@
+/*
+ * 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.nutch.indexwriter.solr;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map.Entry;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.nutch.indexer.NutchDocument;
+import org.apache.nutch.indexer.IndexWriter;
+import org.apache.solr.client.solrj.SolrServer;
+import org.apache.solr.client.solrj.SolrServerException;
+import org.apache.solr.common.SolrInputDocument;
+
+public class SolrIndexWriter implements IndexWriter {
+
+  public static final Logger LOG = LoggerFactory.getLogger(SolrIndexWriter.class);
+
+  private SolrServer solr;
+  private SolrMappingReader solrMapping;
+  
+  private Configuration config;
+
+  private final List<SolrInputDocument> inputDocs =
+    new ArrayList<SolrInputDocument>();
+
+  private int batchSize;
+  private int numDeletes = 0;
+  private boolean delete = false;
+  
+  protected static long documentCount = 0;
+
+  @Override
+  public void open(Configuration conf)
+  throws IOException {
+    solr = SolrUtils.getCommonsHttpSolrServer(conf);
+    batchSize = conf.getInt(SolrConstants.COMMIT_SIZE, 1000);
+    solrMapping = SolrMappingReader.getInstance(conf);
+  }
+
+  @Override
+  public void write(NutchDocument doc) throws IOException {
+    final SolrInputDocument inputDoc = new SolrInputDocument();
+    for(final Entry<String, List<String>> e : doc) {
+      for (final String val : e.getValue()) {
+
+        Object val2 = val;
+        if (e.getKey().equals("content") || e.getKey().equals("title")) {
+          val2 = SolrUtils.stripNonCharCodepoints(val);
+        }
+
+        inputDoc.addField(solrMapping.mapKey(e.getKey()), val2);
+        String sCopy = solrMapping.mapCopyKey(e.getKey());
+        if (sCopy != e.getKey()) {
+        	inputDoc.addField(sCopy, val2);
+        }
+      }
+    }
+    inputDoc.setDocumentBoost(doc.getScore());
+    inputDocs.add(inputDoc);
+    documentCount++;
+    if (inputDocs.size() >= batchSize) {
+      try {
+        LOG.info("Adding " + Integer.toString(inputDocs.size()) + " documents");
+        solr.add(inputDocs);
+      } catch (final SolrServerException e) {
+        throw new IOException(e);
+      }
+      inputDocs.clear();
+    }
+  }
+
+  @Override
+  public void close() throws IOException {
+    try {
+      if (!inputDocs.isEmpty()) {
+        LOG.info("Adding " + Integer.toString(inputDocs.size()) + " documents");
+        solr.add(inputDocs);
+        inputDocs.clear();
+      }else if(numDeletes > 0){
+        LOG.info("Deleted " + Integer.toString(numDeletes) + " documents");
+      }
+    } catch (final SolrServerException e) {
+      throw new IOException(e);
+    }
+  }
+
+  @Override
+  public Configuration getConf() {
+    return config;
+  }
+
+  @Override
+  public void setConf(Configuration conf) {
+    config = conf;
+    String serverURL = conf.get(SolrConstants.SERVER_URL);
+    if (serverURL == null) {
+        String message = "Missing SOLR URL. Should be set via -D "
+                + SolrConstants.SERVER_URL;
+        message+="\n"+describe();
+        LOG.error(message);
+        throw new RuntimeException(message);
+    }  
+  }
+
+  @Override
+  public void delete(String key) throws IOException {
+    if (delete) {
+      try {
+        solr.deleteById(key);
+        numDeletes++;
+      } catch (final SolrServerException e) {
+        throw makeIOException(e);
+      }
+    }
+  }
+
+  @Override
+  public void update(NutchDocument doc) throws IOException {
+    write(doc);
+  }
+
+  @Override
+  public void commit() throws IOException {
+    try {
+      solr.commit();
+    } catch (SolrServerException e) {
+      throw makeIOException(e);
+    }
+  }
+
+  public static IOException makeIOException(SolrServerException e) {
+    final IOException ioe = new IOException();
+    ioe.initCause(e);
+    return ioe;
+  }
+  
+  @Override
+  public String describe() {
+    StringBuffer sb = new StringBuffer("SOLRIndexWriter\n");
+    sb.append("\t").append(SolrConstants.SERVER_URL).append(" : URL of the SOLR instance (mandatory)\n");
+    sb.append("\t").append(SolrConstants.COMMIT_SIZE).append(" : buffer size when sending to SOLR (default 1000)\n");
+    sb.append("\t").append(SolrConstants.MAPPING_FILE).append(" : name of the mapping file for fields (default solrindex-mapping.xml)\n");
+    sb.append("\t").append(SolrConstants.USE_AUTH).append(" : use authentication (default false)\n");
+    sb.append("\t").append(SolrConstants.USERNAME).append(" : use authentication (default false)\n");
+    sb.append("\t").append(SolrConstants.USE_AUTH).append(" : username for authentication\n");
+    sb.append("\t").append(SolrConstants.PASSWORD).append(" : password for authentication\n");
+    return sb.toString();
+  }
+
+}
diff --git a/src/plugin/indexer-solr/src/java/org/apache/nutch/indexwriter/solr/SolrMappingReader.java b/src/plugin/indexer-solr/src/java/org/apache/nutch/indexwriter/solr/SolrMappingReader.java
new file mode 100644
index 0000000..139011a
--- /dev/null
+++ b/src/plugin/indexer-solr/src/java/org/apache/nutch/indexwriter/solr/SolrMappingReader.java
@@ -0,0 +1,143 @@
+/*
+ * 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.nutch.indexwriter.solr;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.MalformedURLException;
+import java.util.HashMap;
+import java.util.Map;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.nutch.util.ObjectCache;
+import org.w3c.dom.Document;
+import org.w3c.dom.Element;
+import org.w3c.dom.NodeList;
+import org.xml.sax.InputSource;
+import org.xml.sax.SAXException;
+
+public class SolrMappingReader {
+  public static Logger LOG = LoggerFactory.getLogger(SolrMappingReader.class);
+  
+  private Configuration conf;
+  
+  private Map<String, String> keyMap = new HashMap<String, String>();
+  private Map<String, String> copyMap = new HashMap<String, String>();
+  private String uniqueKey = "id";
+  
+  public static synchronized SolrMappingReader getInstance(Configuration conf) {
+    ObjectCache cache = ObjectCache.get(conf);
+    SolrMappingReader instance = (SolrMappingReader)cache.getObject(SolrMappingReader.class.getName());
+    if (instance == null) {
+      instance = new SolrMappingReader(conf);
+      cache.setObject(SolrMappingReader.class.getName(), instance);
+    }
+    return instance;
+  }
+
+  protected SolrMappingReader(Configuration conf) {
+    this.conf = conf;
+    parseMapping();
+  }
+
+  private void parseMapping() {    
+    InputStream ssInputStream = null;
+    ssInputStream = conf.getConfResourceAsInputStream(conf.get(SolrConstants.MAPPING_FILE, "solrindex-mapping.xml"));
+
+    InputSource inputSource = new InputSource(ssInputStream);
+    try {
+      DocumentBuilderFactory factory = DocumentBuilderFactory.newInstance();
+      DocumentBuilder builder = factory.newDocumentBuilder();
+      Document document = builder.parse(inputSource);
+      Element rootElement = document.getDocumentElement();
+      NodeList fieldList = rootElement.getElementsByTagName("field");
+      if (fieldList.getLength() > 0) {
+        for (int i = 0; i < fieldList.getLength(); i++) {
+          Element element = (Element) fieldList.item(i);
+          LOG.info("source: " + element.getAttribute("source") + " dest: " + element.getAttribute("dest"));
+          keyMap.put(element.getAttribute("source"), element.getAttribute("dest"));
+        }
+      }
+      NodeList copyFieldList = rootElement.getElementsByTagName("copyField");
+      if (copyFieldList.getLength() > 0) {
+        for (int i = 0; i < copyFieldList.getLength(); i++) {
+          Element element = (Element) copyFieldList.item(i);
+          LOG.info("source: " + element.getAttribute("source") + " dest: " + element.getAttribute("dest"));
+          copyMap.put(element.getAttribute("source"), element.getAttribute("dest"));
+        }
+      }
+      NodeList uniqueKeyItem = rootElement.getElementsByTagName("uniqueKey");
+      if (uniqueKeyItem.getLength() > 1) {
+        LOG.warn("More than one unique key definitions found in solr index mapping, using default 'id'");
+        uniqueKey = "id";
+      }
+      else if (uniqueKeyItem.getLength() == 0) {
+        LOG.warn("No unique key definition found in solr index mapping using, default 'id'");
+      }
+      else{
+    	  uniqueKey = uniqueKeyItem.item(0).getFirstChild().getNodeValue();
+      }
+    } catch (MalformedURLException e) {
+        LOG.warn(e.toString());
+    } catch (SAXException e) {
+        LOG.warn(e.toString());
+    } catch (IOException e) {
+    	LOG.warn(e.toString());
+    } catch (ParserConfigurationException e) {
+    	LOG.warn(e.toString());
+    } 
+  }
+	  
+  public Map<String, String> getKeyMap() {
+    return keyMap;
+  }
+	  
+  public Map<String, String> getCopyMap() {
+    return copyMap;
+  }
+	  
+  public String getUniqueKey() {
+    return uniqueKey;
+  }
+
+  public String hasCopy(String key) {
+    if (copyMap.containsKey(key)) {
+      key = copyMap.get(key);
+    }
+    return key;
+  }
+
+  public String mapKey(String key) throws IOException {
+    if(keyMap.containsKey(key)) {
+      key = keyMap.get(key);
+    }
+    return key;
+  }
+
+  public String mapCopyKey(String key) throws IOException {
+    if(copyMap.containsKey(key)) {
+      key = copyMap.get(key);
+    }
+    return key;
+  }
+}
diff --git a/src/plugin/indexer-solr/src/java/org/apache/nutch/indexwriter/solr/SolrUtils.java b/src/plugin/indexer-solr/src/java/org/apache/nutch/indexwriter/solr/SolrUtils.java
new file mode 100644
index 0000000..b29ede2
--- /dev/null
+++ b/src/plugin/indexer-solr/src/java/org/apache/nutch/indexwriter/solr/SolrUtils.java
@@ -0,0 +1,61 @@
+package org.apache.nutch.indexwriter.solr;
+
+
+import org.apache.commons.httpclient.HttpClient;
+import org.apache.commons.httpclient.auth.AuthScope;
+import org.apache.commons.httpclient.UsernamePasswordCredentials;
+import org.apache.commons.httpclient.params.HttpClientParams;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.solr.client.solrj.impl.CommonsHttpSolrServer;
+
+import java.net.MalformedURLException;
+
+public class SolrUtils {
+
+  public static Logger LOG = LoggerFactory.getLogger(SolrUtils.class);
+
+  public static CommonsHttpSolrServer getCommonsHttpSolrServer(Configuration job) throws MalformedURLException {
+    HttpClient client=new HttpClient();
+
+    // Check for username/password
+    if (job.getBoolean(SolrConstants.USE_AUTH, false)) {
+      String username = job.get(SolrConstants.USERNAME);
+
+      LOG.info("Authenticating as: " + username);
+
+      AuthScope scope = new AuthScope(AuthScope.ANY_HOST, AuthScope.ANY_PORT, AuthScope.ANY_REALM, AuthScope.ANY_SCHEME);
+
+      client.getState().setCredentials(scope, new UsernamePasswordCredentials(username, job.get(SolrConstants.PASSWORD)));
+
+      HttpClientParams params = client.getParams();
+      params.setAuthenticationPreemptive(true);
+
+      client.setParams(params);
+    }
+
+    return new CommonsHttpSolrServer(job.get(SolrConstants.SERVER_URL), client);
+  }
+
+  public static String stripNonCharCodepoints(String input) {
+    StringBuilder retval = new StringBuilder();
+    char ch;
+
+    for (int i = 0; i < input.length(); i++) {
+      ch = input.charAt(i);
+
+      // Strip all non-characters http://unicode.org/cldr/utility/list-unicodeset.jsp?a=[:Noncharacter_Code_Point=True:]
+      // and non-printable control characters except tabulator, new line and carriage return
+      if (ch % 0x10000 != 0xffff && // 0xffff - 0x10ffff range step 0x10000
+              ch % 0x10000 != 0xfffe && // 0xfffe - 0x10fffe range
+              (ch <= 0xfdd0 || ch >= 0xfdef) && // 0xfdd0 - 0xfdef
+              (ch > 0x1F || ch == 0x9 || ch == 0xa || ch == 0xd)) {
+
+        retval.append(ch);
+      }
+    }
+
+    return retval.toString();
+  }
+}
\ No newline at end of file
diff --git a/src/plugin/nutch-extensionpoints/plugin.xml b/src/plugin/nutch-extensionpoints/plugin.xml
index 43fe045..d567f82 100644
--- a/src/plugin/nutch-extensionpoints/plugin.xml
+++ b/src/plugin/nutch-extensionpoints/plugin.xml
@@ -56,4 +56,7 @@
       id="org.apache.nutch.scoring.ScoringFilter"
       name="Nutch Scoring"/>
 
+<extension-point
+      id="org.apache.nutch.indexer.IndexWriter"
+      name="Nutch Index Writer"/>
 </plugin>
