You are viewing a plain text version of this content. The canonical link for it is here.
Posted to solr-commits@lucene.apache.org by yo...@apache.org on 2006/01/26 06:40:05 UTC

svn commit: r372455 [8/11] - in /incubator/solr/trunk: ./ src/ src/apps/ src/apps/SolarTest/ src/apps/SolarTest/src/ src/java/ src/java/org/ src/java/org/apache/ src/java/org/apache/solr/ src/java/org/apache/solr/analysis/ src/java/org/apache/solr/core...

Added: incubator/solr/trunk/src/java/org/apache/solr/update/DirectUpdateHandler.java
URL: http://svn.apache.org/viewcvs/incubator/solr/trunk/src/java/org/apache/solr/update/DirectUpdateHandler.java?rev=372455&view=auto
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/update/DirectUpdateHandler.java (added)
+++ incubator/solr/trunk/src/java/org/apache/solr/update/DirectUpdateHandler.java Wed Jan 25 21:37:29 2006
@@ -0,0 +1,381 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.
+ */
+
+/**
+ * @author yonik
+ */
+
+package org.apache.solr.update;
+
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.TermDocs;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.search.Query;
+
+import java.util.HashSet;
+import java.util.concurrent.Future;
+import java.util.concurrent.ExecutionException;
+import java.util.logging.Level;
+import java.io.IOException;
+import java.net.URL;
+
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.search.QueryParsing;
+import org.apache.solr.util.NamedList;
+import org.apache.solr.update.UpdateHandler;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrException;
+
+/**
+ * <code>DirectUpdateHandler</code> implements an UpdateHandler where documents are added
+ * directly to the main lucene index as opposed to adding to a separate smaller index.
+ * For this reason, not all combinations to/from pending and committed are supported.
+ *
+ * @author yonik
+ * @version $Id: DirectUpdateHandler.java,v 1.13 2005/06/17 20:44:42 yonik Exp $
+ * @since solr 0.9
+ */
+
+public class DirectUpdateHandler extends UpdateHandler {
+
+  // the set of ids in the "pending set" (those docs that have been added, but
+  // that are not yet visible.
+  final HashSet<String> pset;
+  IndexWriter writer;
+  SolrIndexSearcher searcher;
+  int numAdds=0;     // number of docs added to the pending set
+  int numPending=0;  // number of docs currently in this pending set
+  int numDeleted=0;  // number of docs deleted or
+
+
+  public DirectUpdateHandler(SolrCore core) throws IOException {
+    super(core);
+    pset = new HashSet<String>(256);
+  }
+
+
+  protected void openWriter() throws IOException {
+    if (writer==null) {
+      writer = createMainIndexWriter("DirectUpdateHandler");
+    }
+  }
+
+  protected void closeWriter() throws IOException {
+    try {
+      if (writer!=null) writer.close();
+    } finally {
+      // TODO: if an exception causes the writelock to not be
+      // released, we could delete it here.
+      writer=null;
+    }
+  }
+
+  protected void openSearcher() throws IOException {
+    if (searcher==null) {
+      searcher = core.newSearcher("DirectUpdateHandler");
+    }
+  }
+
+  protected void closeSearcher() throws IOException {
+    try {
+      if (searcher!=null) searcher.close();
+    } finally {
+      // TODO: if an exception causes the writelock to not be
+      // released, we could delete it here.
+      searcher=null;
+    }
+  }
+
+  protected void doAdd(Document doc) throws IOException {
+    closeSearcher(); openWriter();
+    writer.addDocument(doc);
+  }
+
+  protected boolean existsInIndex(String id) throws IOException {
+    if (idField == null) throw new SolrException(2,"Operation requires schema to have a unique key field");
+
+    closeWriter(); openSearcher();
+    IndexReader ir = searcher.getReader();
+    TermDocs tdocs = null;
+    boolean exists=false;
+    try {
+      tdocs = ir.termDocs(idTerm(id));
+      if (tdocs.next()) exists=true;
+    } finally {
+      try { if (tdocs != null) tdocs.close(); } catch (Exception e) {}
+    }
+    return exists;
+  }
+
+
+  protected int deleteInIndex(String id) throws IOException {
+    if (idField == null) throw new SolrException(2,"Operation requires schema to have a unique key field");
+
+    closeWriter(); openSearcher();
+    IndexReader ir = searcher.getReader();
+    TermDocs tdocs = null;
+    int num=0;
+    try {
+      num = ir.delete(idTerm(id));
+      if (SolrCore.log.isLoggable(Level.FINEST)) {
+        SolrCore.log.finest("deleted " + num + " docs matching id " + id);
+      }
+    } finally {
+      try { if (tdocs != null) tdocs.close(); } catch (Exception e) {}
+    }
+    return num;
+  }
+
+  protected void overwrite(String id, Document doc) throws IOException {
+    if (id==null) id=getId(doc);
+    deleteInIndex(id);
+    doAdd(doc);
+  }
+
+  /************** Direct update handler - pseudo code ***********
+  def add(doc, id, allowDups, overwritePending, overwriteCommitted):
+    if not overwritePending and not overwriteCommitted:
+      #special case... no need to check pending set, and we don't keep
+      #any state around about this addition
+      if allowDups:
+        committed[id]=doc  #100
+        return
+      else:
+        #if no dups allowed, we must check the *current* index (pending and committed)
+        if not committed[id]: committed[id]=doc  #000
+        return
+    #001  (searchd addConditionally)
+    if not allowDups and not overwritePending and pending[id]: return
+    del committed[id]  #delete from pending and committed  111 011
+    committed[id]=doc
+    pending[id]=True
+  ****************************************************************/
+
+  // could return the number of docs deleted, but is that always possible to know???
+  public void delete(DeleteUpdateCommand cmd) throws IOException {
+    if (!cmd.fromPending && !cmd.fromCommitted)
+      throw new SolrException(400,"meaningless command: " + cmd);
+    if (!cmd.fromPending || !cmd.fromCommitted)
+      throw new SolrException(400,"operation not supported" + cmd);
+
+    synchronized(this) {
+      deleteInIndex(cmd.id);
+      pset.remove(cmd.id);
+    }
+  }
+
+  // TODO - return number of docs deleted?
+  // Depending on implementation, we may not be able to immediately determine num...
+  public void deleteByQuery(DeleteUpdateCommand cmd) throws IOException {
+    if (!cmd.fromPending && !cmd.fromCommitted)
+      throw new SolrException(400,"meaningless command: " + cmd);
+    if (!cmd.fromPending || !cmd.fromCommitted)
+      throw new SolrException(400,"operation not supported" + cmd);
+
+    Query q = QueryParsing.parseQuery(cmd.query, schema);
+
+    int totDeleted = 0;
+    synchronized(this) {
+      closeWriter(); openSearcher();
+
+      // if we want to count the number of docs that were deleted, then
+      // we need a new instance of the DeleteHitCollector
+      final DeleteHitCollector deleter = new DeleteHitCollector(searcher);
+      searcher.search(q, null, deleter);
+      totDeleted = deleter.deleted;
+    }
+
+    if (SolrCore.log.isLoggable(Level.FINE)) {
+      SolrCore.log.fine("docs deleted:" + totDeleted);
+    }
+
+  }
+
+  /**************** old hit collector... new one is in base class
+  // final DeleteHitCollector deleter = new DeleteHitCollector();
+  class DeleteHitCollector extends HitCollector {
+    public int deleted=0;
+    public void collect(int doc, float score) {
+      try {
+        searcher.getReader().delete(doc);
+        deleted++;
+      } catch (IOException e) {
+        try { closeSearcher(); } catch (Exception ee) { SolrException.log(SolrCore.log,ee); }
+        SolrException.log(SolrCore.log,e);
+        throw new SolrException(500,"Error deleting doc# "+doc,e);
+      }
+    }
+  }
+  ***************************/
+
+  public void commit(CommitUpdateCommand cmd) throws IOException {
+    Future[] waitSearcher = null;
+    if (cmd.waitSearcher) {
+      waitSearcher = new Future[1];
+    }
+
+    synchronized (this) {
+      pset.clear();
+      closeSearcher();  // flush any deletes
+      if (cmd.optimize) {
+        openWriter();  // writer needs to be open to optimize
+        writer.optimize();
+      }
+      closeWriter();
+
+      callPostCommitCallbacks();
+
+      core.getSearcher(true,false,waitSearcher);
+    }
+
+    if (waitSearcher[0] != null) {
+      try {
+        waitSearcher[0].get();
+      } catch (InterruptedException e) {
+        SolrException.log(log,e);
+      } catch (ExecutionException e) {
+        SolrException.log(log,e);
+      }
+    }
+
+    return;
+  }
+
+
+
+  ///////////////////////////////////////////////////////////////////
+  /////////////////// helper method for each add type ///////////////
+  ///////////////////////////////////////////////////////////////////
+
+  protected int addNoOverwriteNoDups(AddUpdateCommand cmd) throws IOException {
+    if (cmd.id==null) {
+      cmd.id=getId(cmd.doc);
+    }
+    synchronized (this) {
+      if (existsInIndex(cmd.id)) return 0;
+      doAdd(cmd.doc);
+    }
+    return 1;
+  }
+
+  protected int addConditionally(AddUpdateCommand cmd) throws IOException {
+    if (cmd.id==null) {
+      cmd.id=getId(cmd.doc);
+    }
+    synchronized(this) {
+      if (pset.contains(cmd.id)) return 0;
+      // since case 001 is currently the only case to use pset, only add
+      // to it in that instance.
+      pset.add(cmd.id);
+      overwrite(cmd.id,cmd.doc);
+      return 1;
+    }
+  }
+
+
+  // overwrite both pending and committed
+  protected synchronized int overwriteBoth(AddUpdateCommand cmd) throws IOException {
+    overwrite(cmd.id, cmd.doc);
+    return 1;
+  }
+
+
+  // add without checking
+  protected synchronized int allowDups(AddUpdateCommand cmd) throws IOException {
+    doAdd(cmd.doc);
+    return 1;
+  }
+
+
+  public int addDoc(AddUpdateCommand cmd) throws IOException {
+    if (!cmd.allowDups && !cmd.overwritePending && !cmd.overwriteCommitted) {
+      return addNoOverwriteNoDups(cmd);
+    } else if (!cmd.allowDups && !cmd.overwritePending && cmd.overwriteCommitted) {
+      return addConditionally(cmd);
+    } else if (!cmd.allowDups && cmd.overwritePending && !cmd.overwriteCommitted) {
+      // return overwriteBoth(cmd);
+      throw new SolrException(400,"unsupported param combo:" + cmd);
+    } else if (!cmd.allowDups && cmd.overwritePending && cmd.overwriteCommitted) {
+      return overwriteBoth(cmd);
+    } else if (cmd.allowDups && !cmd.overwritePending && !cmd.overwriteCommitted) {
+      return allowDups(cmd);
+    } else if (cmd.allowDups && !cmd.overwritePending && cmd.overwriteCommitted) {
+      // return overwriteBoth(cmd);
+      throw new SolrException(400,"unsupported param combo:" + cmd);
+    } else if (cmd.allowDups && cmd.overwritePending && !cmd.overwriteCommitted) {
+      // return overwriteBoth(cmd);
+      throw new SolrException(400,"unsupported param combo:" + cmd);
+    } else if (cmd.allowDups && cmd.overwritePending && cmd.overwriteCommitted) {
+      return overwriteBoth(cmd);
+    }
+    throw new SolrException(400,"unsupported param combo:" + cmd);
+  }
+
+  public void close() throws IOException {
+    synchronized(this) {
+      closeSearcher();
+      closeWriter();
+    }
+  }
+
+
+
+  /////////////////////////////////////////////////////////////////////
+  // SolrInfoMBean stuff: Statistics and Module Info
+  /////////////////////////////////////////////////////////////////////
+
+  public String getName() {
+    return DirectUpdateHandler.class.getName();
+  }
+
+  public String getVersion() {
+    return SolrCore.version;
+  }
+
+  public String getDescription() {
+    return "Update handler that directly changes the on-disk main lucene index";
+  }
+
+  public Category getCategory() {
+    return Category.CORE;
+  }
+
+  public String getCvsId() {
+    return "$Id: DirectUpdateHandler.java,v 1.13 2005/06/17 20:44:42 yonik Exp $";
+  }
+
+  public String getCvsName() {
+    return "$Name:  $";
+  }
+
+  public String getCvsSource() {
+    return "$Source: /cvs/main/searching/solr/solarcore/src/solr/DirectUpdateHandler.java,v $";
+  }
+
+  public URL[] getDocs() {
+    return null;
+  }
+
+  public NamedList getStatistics() {
+    NamedList lst = new NamedList();
+    return lst;
+  }
+
+
+
+
+}

Propchange: incubator/solr/trunk/src/java/org/apache/solr/update/DirectUpdateHandler.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/solr/trunk/src/java/org/apache/solr/update/DirectUpdateHandler2.java
URL: http://svn.apache.org/viewcvs/incubator/solr/trunk/src/java/org/apache/solr/update/DirectUpdateHandler2.java?rev=372455&view=auto
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/update/DirectUpdateHandler2.java (added)
+++ incubator/solr/trunk/src/java/org/apache/solr/update/DirectUpdateHandler2.java Wed Jan 25 21:37:29 2006
@@ -0,0 +1,569 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed 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.
+ */
+
+/**
+ * @author yonik
+ */
+
+package org.apache.solr.update;
+
+import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.index.Term;
+import org.apache.lucene.index.IndexReader;
+import org.apache.lucene.index.TermDocs;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.search.Query;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.Future;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.logging.Level;
+import java.io.IOException;
+import java.net.URL;
+
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.search.QueryParsing;
+import org.apache.solr.util.NamedList;
+import org.apache.solr.core.SolrCore;
+import org.apache.solr.core.SolrException;
+
+/**
+ * <code>DirectUpdateHandler2</code> implements an UpdateHandler where documents are added
+ * directly to the main Lucene index as opposed to adding to a separate smaller index.
+ * For this reason, not all combinations to/from pending and committed are supported.
+ * This version supports efficient removal of duplicates on a commit.  It works by maintaining
+ * a related count for every document being added or deleted.  At commit time, for every id with a count,
+ * all but the last "count" docs with that id are deleted.
+ * <p>
+ *
+ * Supported add command parameters:
+ <TABLE BORDER>
+  <TR>
+    <TH>allowDups</TH>
+    <TH>overwritePending</TH>
+    <TH>overwriteCommitted</TH>
+    <TH>efficiency</TH>
+  </TR>
+  <TR>
+        <TD>false</TD>
+        <TD>false</TD>
+        <TD>true</TD>
+
+        <TD>fast</TD>
+  </TR>
+  <TR>
+        <TD>true or false</TD>
+        <TD>true</TD>
+        <TD>true</TD>
+
+        <TD>fast</TD>
+  </TR>
+  <TR>
+        <TD>true</TD>
+        <TD>false</TD>
+        <TD>false</TD>
+        <TD>fastest</TD>
+  </TR>
+
+</TABLE>
+
+ <p>Supported delete commands:
+ <TABLE BORDER>
+  <TR>
+    <TH>command</TH>
+    <TH>fromPending</TH>
+    <TH>fromCommitted</TH>
+    <TH>efficiency</TH>
+  </TR>
+  <TR>
+        <TD>delete</TD>
+        <TD>true</TD>
+        <TD>true</TD>
+        <TD>fast</TD>
+  </TR>
+  <TR>
+        <TD>deleteByQuery</TD>
+        <TD>true</TD>
+        <TD>true</TD>
+        <TD>very slow*</TD>
+  </TR>
+</TABLE>
+
+  <p>* deleteByQuery causes a commit to happen (close current index writer, open new index reader)
+  before it can be processed.  If deleteByQuery functionality is needed, it's best if they can
+  be batched and executed together so they may share the same index reader.
+
+ *
+ * @author yonik
+ * @version $Id: DirectUpdateHandler2.java,v 1.12 2005/06/17 20:44:42 yonik Exp $
+ * @since solr 0.9
+ */
+
+public class DirectUpdateHandler2 extends UpdateHandler {
+
+  // stats
+  AtomicLong addCommands = new AtomicLong();
+  AtomicLong addCommandsCumulative = new AtomicLong();
+  AtomicLong deleteByIdCommands= new AtomicLong();
+  AtomicLong deleteByIdCommandsCumulative= new AtomicLong();
+  AtomicLong deleteByQueryCommands= new AtomicLong();
+  AtomicLong deleteByQueryCommandsCumulative= new AtomicLong();
+  AtomicLong commitCommands= new AtomicLong();
+  AtomicLong optimizeCommands= new AtomicLong();
+  AtomicLong numDocsDeleted= new AtomicLong();
+  AtomicLong numDocsPending= new AtomicLong();
+  AtomicLong numErrors = new AtomicLong();
+  AtomicLong numErrorsCumulative = new AtomicLong();
+
+
+
+  // The key is the id, the value (Integer) is the number
+  // of docs to save (delete all except the last "n" added)
+  protected final HashMap<String,Integer> pset;
+
+  // commonly used constants for the count in the pset
+  protected final static Integer ZERO = 0;
+  protected final static Integer ONE = 1;
+
+  protected IndexWriter writer;
+  protected SolrIndexSearcher searcher;
+
+  public DirectUpdateHandler2(SolrCore core) throws IOException {
+    super(core);
+    pset = new HashMap<String,Integer>(256); // 256 is just an optional head-start
+  }
+
+  protected void openWriter() throws IOException {
+    if (writer==null) {
+      writer = createMainIndexWriter("DirectUpdateHandler2");
+    }
+  }
+
+  protected void closeWriter() throws IOException {
+    try {
+      numDocsPending.set(0);
+      if (writer!=null) writer.close();
+    } finally {
+      // if an exception causes the writelock to not be
+      // released, we could try and delete it here
+      writer=null;
+    }
+  }
+
+  protected void openSearcher() throws IOException {
+    if (searcher==null) {
+      searcher = core.newSearcher("DirectUpdateHandler2");
+    }
+  }
+
+  protected void closeSearcher() throws IOException {
+    try {
+      if (searcher!=null) searcher.close();
+    } finally {
+      // if an exception causes a lock to not be
+      // released, we could try to delete it.
+      searcher=null;
+    }
+  }
+
+  protected void doAdd(Document doc) throws IOException {
+    closeSearcher(); openWriter();
+    writer.addDocument(doc);
+  }
+
+
+
+  public int addDoc(AddUpdateCommand cmd) throws IOException {
+    addCommands.incrementAndGet();
+    addCommandsCumulative.incrementAndGet();
+    int rc=-1;
+    try {
+      if (!cmd.allowDups && !cmd.overwritePending && !cmd.overwriteCommitted) {
+        throw new SolrException(400,"unsupported param combo:" + cmd);
+        // this would need a reader to implement (to be able to check committed
+        // before adding.)
+        // return addNoOverwriteNoDups(cmd);
+      } else if (!cmd.allowDups && !cmd.overwritePending && cmd.overwriteCommitted) {
+        rc = addConditionally(cmd);
+        return rc;
+      } else if (!cmd.allowDups && cmd.overwritePending && !cmd.overwriteCommitted) {
+        throw new SolrException(400,"unsupported param combo:" + cmd);
+      } else if (!cmd.allowDups && cmd.overwritePending && cmd.overwriteCommitted) {
+        rc = overwriteBoth(cmd);
+        return rc;
+      } else if (cmd.allowDups && !cmd.overwritePending && !cmd.overwriteCommitted) {
+        rc = allowDups(cmd);
+        return rc;
+      } else if (cmd.allowDups && !cmd.overwritePending && cmd.overwriteCommitted) {
+        throw new SolrException(400,"unsupported param combo:" + cmd);
+      } else if (cmd.allowDups && cmd.overwritePending && !cmd.overwriteCommitted) {
+        throw new SolrException(400,"unsupported param combo:" + cmd);
+      } else if (cmd.allowDups && cmd.overwritePending && cmd.overwriteCommitted) {
+        rc = overwriteBoth(cmd);
+        return rc;
+      }
+      throw new SolrException(400,"unsupported param combo:" + cmd);
+    } finally {
+      if (rc!=1) {
+        numErrors.incrementAndGet();
+        numErrorsCumulative.incrementAndGet();
+      } else {
+        numDocsPending.incrementAndGet();
+      }
+    }
+  }
+
+
+  // could return the number of docs deleted, but is that always possible to know???
+  public void delete(DeleteUpdateCommand cmd) throws IOException {
+    deleteByIdCommands.incrementAndGet();
+    deleteByIdCommandsCumulative.incrementAndGet();
+
+    if (!cmd.fromPending && !cmd.fromCommitted) {
+      numErrors.incrementAndGet();
+      numErrorsCumulative.incrementAndGet();
+      throw new SolrException(400,"meaningless command: " + cmd);
+    }
+    if (!cmd.fromPending || !cmd.fromCommitted) {
+      numErrors.incrementAndGet();
+      numErrorsCumulative.incrementAndGet();
+      throw new SolrException(400,"operation not supported" + cmd);
+    }
+
+    synchronized(this) {
+      pset.put(cmd.id, ZERO);
+    }
+  }
+
+  // why not return number of docs deleted?
+  // Depending on implementation, we may not be able to immediately determine the num...
+   public void deleteByQuery(DeleteUpdateCommand cmd) throws IOException {
+     deleteByQueryCommands.incrementAndGet();
+     deleteByQueryCommandsCumulative.incrementAndGet();
+
+     if (!cmd.fromPending && !cmd.fromCommitted) {
+       numErrors.incrementAndGet();
+       numErrorsCumulative.incrementAndGet();
+       throw new SolrException(400,"meaningless command: " + cmd);
+     }
+     if (!cmd.fromPending || !cmd.fromCommitted) {
+       numErrors.incrementAndGet();
+       numErrorsCumulative.incrementAndGet();
+       throw new SolrException(400,"operation not supported" + cmd);
+     }
+
+    boolean madeIt=false;
+    try {
+     Query q = QueryParsing.parseQuery(cmd.query, schema);
+
+     int totDeleted = 0;
+     synchronized(this) {
+       // we need to do much of the commit logic (mainly doing queued
+       // deletes since deleteByQuery can throw off our counts.
+       doDeletions();
+
+       closeWriter();
+       openSearcher();
+
+       // if we want to count the number of docs that were deleted, then
+       // we need a new instance of the DeleteHitCollector
+       final DeleteHitCollector deleter = new DeleteHitCollector(searcher);
+       searcher.search(q, null, deleter);
+       totDeleted = deleter.deleted;
+     }
+
+     if (SolrCore.log.isLoggable(Level.FINE)) {
+       SolrCore.log.fine("docs deleted by query:" + totDeleted);
+     }
+     numDocsDeleted.getAndAdd(totDeleted);
+     madeIt=true;
+    } finally {
+      if (!madeIt) {
+        numErrors.incrementAndGet();
+        numErrorsCumulative.incrementAndGet();
+      }
+    }
+   }
+
+
+  ///////////////////////////////////////////////////////////////////
+  /////////////////// helper method for each add type ///////////////
+  ///////////////////////////////////////////////////////////////////
+
+
+  protected int addConditionally(AddUpdateCommand cmd) throws IOException {
+    if (cmd.id==null) {
+      cmd.id=getId(cmd.doc);
+    }
+    synchronized(this) {
+      Integer saveCount = pset.get(cmd.id);
+      if (saveCount!=null && saveCount!=0) {
+        // a doc with this id already exists in the pending set
+        return 0;
+      }
+      pset.put(cmd.id, ONE);
+      doAdd(cmd.doc);
+      return 1;
+    }
+  }
+
+
+  // overwrite both pending and committed
+  protected synchronized int overwriteBoth(AddUpdateCommand cmd) throws IOException {
+    if (cmd.id==null) {
+      cmd.id=getId(cmd.doc);
+    }
+    synchronized (this) {
+      pset.put(cmd.id, ONE);
+      doAdd(cmd.doc);
+    }
+    return 1;
+  }
+
+
+  // add without checking
+  protected synchronized int allowDups(AddUpdateCommand cmd) throws IOException {
+    if (cmd.id==null) {
+      cmd.id=getOptId(cmd.doc);
+    }
+    synchronized(this) {
+      doAdd(cmd.doc);
+
+      if (cmd.id != null) {
+        Integer saveCount = pset.get(cmd.id);
+
+        // if there weren't any docs marked for deletion before, then don't mark
+        // any for deletion now.
+        if (saveCount == null) return 1;
+
+        // If there were docs marked for deletion, then increment the number of
+        // docs to save at the end.
+
+        // the following line is optional, but it saves an allocation in the common case.
+        if (saveCount == ZERO) saveCount=ONE;
+        else saveCount++;
+
+        pset.put(cmd.id, saveCount);
+      }
+    }
+    return 1;
+  }
+
+  // NOT FOR USE OUTSIDE OF A "synchronized(this)" BLOCK
+  private int[] docnums;
+
+  //
+  // do all needed deletions.
+  // call in a synchronized context.
+  //
+  protected void doDeletions() throws IOException {
+
+    if (pset.size() > 0) { // optimization: only open searcher if there is something to delete...
+      log.info("DirectUpdateHandler2 deleting and removing dups for " + pset.size() +" ids");
+      int numDeletes=0;
+
+      closeWriter();
+      openSearcher();
+      IndexReader reader = searcher.getReader();
+      TermDocs tdocs = reader.termDocs();
+      String fieldname = idField.getName();
+
+      for (Map.Entry<String,Integer> entry : pset.entrySet()) {
+        String id = entry.getKey();
+        int saveLast = entry.getValue();  // save the last "saveLast" documents
+
+        //expand our array that keeps track of docs if needed.
+        if (docnums==null || saveLast > docnums.length) {
+          docnums = new int[saveLast];
+        }
+
+        // initialize all docnums in the list to -1 (unused)
+        for (int i=0; i<saveLast; i++) {
+          docnums[i] = -1;
+        }
+
+        tdocs.seek(new Term(fieldname,id));
+
+        //
+        // record the docs for this term in the "docnums" array and wrap around
+        // at size "saveLast".  If we reuse a slot in the array, then we delete
+        // the doc that was there from the index.
+        //
+        int pos=0;
+        while (tdocs.next()) {
+          if (saveLast==0) {
+            // special case - delete all the docs as we see them.
+            reader.delete(tdocs.doc());
+            numDeletes++;
+            continue;
+          }
+
+          int prev=docnums[pos];
+          docnums[pos]=tdocs.doc();
+          if (prev != -1) {
+            reader.delete(prev);
+            numDeletes++;
+          }
+
+          if (++pos >= saveLast) pos=0;
+        }
+      }
+
+      // should we ever shrink it again, or just clear it?
+      pset.clear();
+      log.info("DirectUpdateHandler2 docs deleted=" + numDeletes);
+      numDocsDeleted.addAndGet(numDeletes);
+    }
+
+  }
+
+
+
+  public void commit(CommitUpdateCommand cmd) throws IOException {
+
+    if (cmd.optimize) {
+      optimizeCommands.incrementAndGet();
+    } else {
+      commitCommands.incrementAndGet();
+    }
+
+    Future[] waitSearcher = null;
+    if (cmd.waitSearcher) {
+      waitSearcher = new Future[1];
+    }
+
+    boolean error=true;
+    try {
+      synchronized (this) {
+        log.info("start "+cmd);
+        doDeletions();
+
+        if (cmd.optimize) {
+          closeSearcher();
+          openWriter();
+          writer.optimize();
+        }
+
+        closeSearcher();
+        closeWriter();
+
+        callPostCommitCallbacks();
+
+        // open a new searcher in the sync block to avoid opening it
+        // after a deleteByQuery changed the index, or in between deletes
+        // and adds of another commit being done.
+        core.getSearcher(true,false,waitSearcher);
+
+        log.info("end_commit_flush");
+      }  // end synchronized block
+
+      error=false;
+    }
+    finally {
+      addCommands.set(0);
+      deleteByIdCommands.set(0);
+      deleteByQueryCommands.set(0);
+      numErrors.set(error ? 1 : 0);
+    }
+
+    // if we are supposed to wait for the searcher to be registered, then we should do it
+    // outside of the synchronized block so that other update operations can proceed.
+    if (waitSearcher!=null && waitSearcher[0] != null) {
+       try {
+        waitSearcher[0].get();
+      } catch (InterruptedException e) {
+        SolrException.log(log,e);
+      } catch (ExecutionException e) {
+        SolrException.log(log,e);
+      }
+    }
+
+    return;
+  }
+
+
+  public void close() throws IOException {
+    log.info("closing " + this);
+    synchronized(this) {
+      doDeletions();
+      closeSearcher();
+      closeWriter();
+    }
+    log.info("closed " + this);
+  }
+
+  /////////////////////////////////////////////////////////////////////
+  // SolrInfoMBean stuff: Statistics and Module Info
+  /////////////////////////////////////////////////////////////////////
+
+  public String getName() {
+    return DirectUpdateHandler2.class.getName();
+  }
+
+  public String getVersion() {
+    return SolrCore.version;
+  }
+
+  public String getDescription() {
+    return "Update handler that efficiently directly updates the on-disk main lucene index";
+  }
+
+  public Category getCategory() {
+    return Category.UPDATEHANDLER;
+  }
+
+  public String getCvsId() {
+    return "$Id: DirectUpdateHandler2.java,v 1.12 2005/06/17 20:44:42 yonik Exp $";
+  }
+
+  public String getCvsName() {
+    return "$Name:  $";
+  }
+
+  public String getCvsSource() {
+    return "$Source: /cvs/main/searching/solr/solarcore/src/solr/DirectUpdateHandler2.java,v $";
+  }
+
+  public URL[] getDocs() {
+    return null;
+  }
+
+  public NamedList getStatistics() {
+    NamedList lst = new NamedList();
+    lst.add("commits", commitCommands.get());
+    lst.add("optimizes", optimizeCommands.get());
+    lst.add("docsPending", numDocsPending.get());
+    // pset.size() not synchronized, but it should be fine to access.
+    lst.add("deletesPending", pset.size());
+    lst.add("adds", addCommands.get());
+    lst.add("deletesById", deleteByIdCommands.get());
+    lst.add("deletesByQuery", deleteByQueryCommands.get());
+    lst.add("errors", numErrors.get());
+    lst.add("cumulative_adds", addCommandsCumulative.get());
+    lst.add("cumulative_deletesById", deleteByIdCommandsCumulative.get());
+    lst.add("cumulative_deletesByQuery", deleteByQueryCommandsCumulative.get());
+    lst.add("cumulative_errors", numErrorsCumulative.get());
+    lst.add("docsDeleted", numDocsDeleted.get());
+
+    return lst;
+  }
+
+  public String toString() {
+    return "DirectUpdateHandler2" + getStatistics();
+  }
+}

Propchange: incubator/solr/trunk/src/java/org/apache/solr/update/DirectUpdateHandler2.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/solr/trunk/src/java/org/apache/solr/update/DocumentBuilder.java
URL: http://svn.apache.org/viewcvs/incubator/solr/trunk/src/java/org/apache/solr/update/DocumentBuilder.java?rev=372455&view=auto
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/update/DocumentBuilder.java (added)
+++ incubator/solr/trunk/src/java/org/apache/solr/update/DocumentBuilder.java Wed Jan 25 21:37:29 2006
@@ -0,0 +1,104 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.update;
+
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Field;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.core.SolrException;
+
+import java.util.HashMap;
+
+/**
+ * @author yonik
+ * @version $Id: DocumentBuilder.java,v 1.7 2005/12/02 04:31:06 yonik Exp $
+ */
+
+
+// Not thread safe - by design.  Create a new builder for each thread.
+public class DocumentBuilder {
+  private final IndexSchema schema;
+  private Document doc;
+  private HashMap<String,String> map = new HashMap<String,String>();
+
+  public DocumentBuilder(IndexSchema schema) {
+    this.schema = schema;
+  }
+
+  public void startDoc() {
+    doc = new Document();
+    map.clear();
+  }
+
+  protected void addSingleField(SchemaField sfield, String val, float boost) {
+    //System.out.println("###################ADDING FIELD "+sfield+"="+val);
+
+    // we don't check for a null val ourselves because a solr.FieldType
+    // might actually want to map it to something.  If createField()
+    // returns null, then we don't store the field.
+    Field field = sfield.createField(val, boost);
+    if (field != null) {
+      if (!sfield.multiValued()) {
+        String oldValue = map.put(sfield.getName(), val);
+        if (oldValue != null) {
+          throw new SolrException(400,"ERROR: multiple values encountered for non multiValued field " + sfield.getName()
+                  + ": first='" + oldValue + "' second='" + val + "'");
+        }
+      }
+      // field.setBoost(boost);
+      doc.add(field);
+    }
+  }
+
+
+  public void addField(SchemaField sfield, String val, float boost) {
+    addSingleField(sfield,val,boost);
+
+    // Check if we should copy this field to any other fields.
+    SchemaField[] destArr = schema.getCopyFields(sfield.getName());
+    if (destArr != null) {
+      for (SchemaField destField : destArr) {
+        addSingleField(destField,val,boost);
+      }
+    }
+  }
+
+  public void addField(String name, String val) {
+    SchemaField ftype = schema.getField(name);
+    // fields.get(name);
+    addField(ftype,val,1.0f);
+  }
+
+  public void addField(String name, String val, float boost) {
+    SchemaField ftype = schema.getField(name);
+    addField(ftype,val,boost);
+  }
+
+  public void setBoost(float boost) {
+    doc.setBoost(boost);
+  }
+
+  public void endDoc() {
+  }
+
+  // specific to this type of document builder
+  public Document getDoc() {
+    Document ret = doc; doc=null;
+    return ret;
+  }
+}

Propchange: incubator/solr/trunk/src/java/org/apache/solr/update/DocumentBuilder.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/solr/trunk/src/java/org/apache/solr/update/SolrIndexConfig.java
URL: http://svn.apache.org/viewcvs/incubator/solr/trunk/src/java/org/apache/solr/update/SolrIndexConfig.java?rev=372455&view=auto
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/update/SolrIndexConfig.java (added)
+++ incubator/solr/trunk/src/java/org/apache/solr/update/SolrIndexConfig.java Wed Jan 25 21:37:29 2006
@@ -0,0 +1,62 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.update;
+
+import org.apache.solr.core.SolrConfig;
+
+//
+// For performance reasons, we don't want to re-read
+// config params each time an index writer is created.
+// This config object encapsulates IndexWriter config params.
+//
+/**
+ * @author yonik
+ * @version $Id$
+ */
+public class SolrIndexConfig {
+  public static final String defaultsName ="indexDefaults";
+
+  //default values
+  public static final boolean defUseCompoundFile=SolrConfig.config.getBool(defaultsName +"/useCompoundFile", true);
+  public static final int defMaxBufferedDocs=SolrConfig.config.getInt(defaultsName +"/maxBufferedDocs", -1);
+  public static final int defMaxMergeDocs=SolrConfig.config.getInt(defaultsName +"/maxMergeDocs", -1);
+  public static final int defMergeFactor=SolrConfig.config.getInt(defaultsName +"/mergeFactor", -1);
+  public static final int defMaxFieldLength=SolrConfig.config.getInt(defaultsName +"/maxFieldLength", -1);
+  public static final int writeLockTimeout=SolrConfig.config.getInt(defaultsName +"/writeLockTimeout", -1);
+  public static final int commitLockTimeout=SolrConfig.config.getInt(defaultsName +"/commitLockTimeout", -1);
+
+  /*** These are "final" in lucene 1.9
+  static {
+    if (writeLockTimeout != -1) IndexWriter.WRITE_LOCK_TIMEOUT=writeLockTimeout;
+    if (commitLockTimeout != -1) IndexWriter.COMMIT_LOCK_TIMEOUT=commitLockTimeout;
+  }
+  ***/
+  
+  public final boolean useCompoundFile;
+  public final int maxBufferedDocs;
+  public final int maxMergeDocs;
+  public final int mergeFactor;
+  public final int maxFieldLength;
+
+  public SolrIndexConfig(String prefix)  {
+    useCompoundFile=SolrConfig.config.getBool(prefix+"/useCompoundFile", defUseCompoundFile);
+    maxBufferedDocs=SolrConfig.config.getInt(prefix+"/maxBufferedDocs",defMaxBufferedDocs);
+    maxMergeDocs=SolrConfig.config.getInt(prefix+"/maxMergeDocs",defMaxMergeDocs);
+    mergeFactor=SolrConfig.config.getInt(prefix+"/mergeFactor",defMergeFactor);
+    maxFieldLength= SolrConfig.config.getInt(prefix+"/maxFieldLength",defMaxFieldLength);
+  }
+}
\ No newline at end of file

Propchange: incubator/solr/trunk/src/java/org/apache/solr/update/SolrIndexConfig.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/solr/trunk/src/java/org/apache/solr/update/SolrIndexWriter.java
URL: http://svn.apache.org/viewcvs/incubator/solr/trunk/src/java/org/apache/solr/update/SolrIndexWriter.java?rev=372455&view=auto
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/update/SolrIndexWriter.java (added)
+++ incubator/solr/trunk/src/java/org/apache/solr/update/SolrIndexWriter.java Wed Jan 25 21:37:29 2006
@@ -0,0 +1,105 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.update;
+
+import org.apache.lucene.index.IndexWriter;
+import org.apache.solr.schema.IndexSchema;
+
+import java.util.logging.Logger;
+import java.io.IOException;
+
+/**
+ * An IndexWriter that is configured via Solr config mechanisms.
+ *
+* @author yonik
+* @version $Id: SolrIndexWriter.java,v 1.9 2006/01/09 03:51:44 yonik Exp $
+* @since solr 0.9
+*/
+
+
+public class SolrIndexWriter extends IndexWriter {
+  private static Logger log = Logger.getLogger(SolrIndexWriter.class.getName());
+
+  String name;
+  IndexSchema schema;
+
+  private void init(String name, IndexSchema schema, SolrIndexConfig config) {
+    log.fine("Opened Writer " + name);
+    this.name = name;
+    this.schema = schema;
+    setSimilarity(schema.getSimilarity());
+    // setUseCompoundFile(false);
+
+    if (config != null) {
+      setUseCompoundFile(config.useCompoundFile);
+      if (config.maxBufferedDocs != -1) minMergeDocs=config.maxBufferedDocs;
+      if (config.maxMergeDocs != -1) maxMergeDocs=config.maxMergeDocs;
+      if (config.mergeFactor != -1)  mergeFactor =config.mergeFactor;
+      if (config.maxFieldLength != -1)  maxFieldLength =config.maxFieldLength;
+    }
+
+  }
+
+  public SolrIndexWriter(String name, String path, boolean create, IndexSchema schema) throws IOException {
+    super(path, schema.getAnalyzer(), create);
+    init(name, schema, null);
+  }
+
+  public SolrIndexWriter(String name, String path, boolean create, IndexSchema schema, SolrIndexConfig config) throws IOException {
+    super(path, schema.getAnalyzer(), create);
+    init(name, schema,config);
+  }
+
+  /*** use DocumentBuilder now...
+  private final void addField(Document doc, String name, String val) {
+      SchemaField ftype = schema.getField(name);
+
+      // we don't check for a null val ourselves because a solr.FieldType
+      // might actually want to map it to something.  If createField()
+      // returns null, then we don't store the field.
+
+      Field field = ftype.createField(val, boost);
+      if (field != null) doc.add(field);
+  }
+
+
+  public void addRecord(String[] fieldNames, String[] fieldValues) throws IOException {
+    Document doc = new Document();
+    for (int i=0; i<fieldNames.length; i++) {
+      String name = fieldNames[i];
+      String val = fieldNames[i];
+
+      // first null is end of list.  client can reuse arrays if they want
+      // and just write a single null if there is unused space.
+      if (name==null) break;
+
+      addField(doc,name,val);
+    }
+    addDocument(doc);
+  }
+  ******/
+
+  public void close() throws IOException {
+    log.fine("Closing Writer " + name);
+    super.close();
+  }
+
+  void finalizer() {
+    try {super.close();} catch (IOException e) {}
+  }
+
+}

Propchange: incubator/solr/trunk/src/java/org/apache/solr/update/SolrIndexWriter.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/solr/trunk/src/java/org/apache/solr/update/UpdateCommand.java
URL: http://svn.apache.org/viewcvs/incubator/solr/trunk/src/java/org/apache/solr/update/UpdateCommand.java?rev=372455&view=auto
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/update/UpdateCommand.java (added)
+++ incubator/solr/trunk/src/java/org/apache/solr/update/UpdateCommand.java Wed Jan 25 21:37:29 2006
@@ -0,0 +1,37 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.update;
+
+import org.apache.lucene.document.Document;
+
+
+/** An index update command encapsulated in an object (Command pattern)
+ *
+ * @author yonik
+ * @version $Id: UpdateCommand.java,v 1.4 2005/05/25 04:26:47 yonik Exp $
+ */
+  public class UpdateCommand {
+    protected String commandName;
+
+    public UpdateCommand(String commandName) {
+      this.commandName = commandName;
+    }
+
+    public String toString() {
+      return commandName;
+    }
+  }

Propchange: incubator/solr/trunk/src/java/org/apache/solr/update/UpdateCommand.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/solr/trunk/src/java/org/apache/solr/update/UpdateHandler.java
URL: http://svn.apache.org/viewcvs/incubator/solr/trunk/src/java/org/apache/solr/update/UpdateHandler.java?rev=372455&view=auto
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/update/UpdateHandler.java (added)
+++ incubator/solr/trunk/src/java/org/apache/solr/update/UpdateHandler.java Wed Jan 25 21:37:29 2006
@@ -0,0 +1,148 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.update;
+
+
+import org.apache.lucene.index.Term;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.search.HitCollector;
+import org.w3c.dom.NodeList;
+import org.w3c.dom.Node;
+
+import java.util.logging.Logger;
+import java.util.Vector;
+import java.io.IOException;
+
+import org.apache.solr.search.SolrIndexSearcher;
+import org.apache.solr.util.DOMUtil;
+import org.apache.solr.schema.IndexSchema;
+import org.apache.solr.schema.SchemaField;
+import org.apache.solr.schema.FieldType;
+import org.apache.solr.core.*;
+
+import javax.xml.xpath.XPathConstants;
+
+/**
+ * <code>UpdateHandler</code> handles requests to change the index
+ * (adds, deletes, commits, optimizes, etc).
+ *
+ * @author yonik
+ * @version $Id: UpdateHandler.java,v 1.15 2005/06/21 20:24:25 yonik Exp $
+ * @since solr 0.9
+ */
+
+public abstract class UpdateHandler implements SolrInfoMBean {
+  protected final static Logger log = Logger.getLogger(UpdateHandler.class.getName());
+
+  protected final SolrCore core;
+  protected final IndexSchema schema;
+
+  protected final SchemaField idField;
+  protected final FieldType idFieldType;
+
+  protected Vector<SolrEventListener> commitCallbacks = new Vector<SolrEventListener>();
+
+  private void parseEventListeners() {
+    NodeList nodes = (NodeList) SolrConfig.config.evaluate("updateHandler/listener[@event=\"postCommit\"]", XPathConstants.NODESET);
+    if (nodes!=null) {
+      for (int i=0; i<nodes.getLength(); i++) {
+        Node node = nodes.item(i);
+        try {
+          String className = DOMUtil.getAttr(node,"class");
+          Class clazz = Class.forName(className);
+          SolrEventListener listener = (SolrEventListener)clazz.newInstance();
+          listener.init(DOMUtil.childNodesToNamedList(node));
+          // listener.init(DOMUtil.toMapExcept(node.getAttributes(),"class","synchronized"));
+          commitCallbacks.add(listener);
+          log.info("added SolrEventListener for postCommit: " + listener);
+        } catch (Exception e) {
+          throw new SolrException(1,"error parsing event listevers", e, false);
+        }
+      }
+    }
+  }
+
+  protected void callPostCommitCallbacks() {
+    for (SolrEventListener listener : commitCallbacks) {
+      listener.postCommit();
+    }
+  }
+
+  public UpdateHandler(SolrCore core)  {
+    this.core=core;
+    schema = core.getSchema();
+    idField = schema.getUniqueKeyField();
+    idFieldType = idField!=null ? idField.getType() : null;
+
+    parseEventListeners();
+    SolrInfoRegistry.getRegistry().put("updateHandler", this);
+  }
+
+  protected SolrIndexWriter createMainIndexWriter(String name) throws IOException {
+    SolrIndexWriter writer = new SolrIndexWriter(name,core.getDir(), false, schema,SolrCore.mainIndexConfig);
+    return writer;
+  }
+
+  protected final Term idTerm(String id) {
+    // to correctly create the Term, the string needs to be run
+    // through the Analyzer for that field.
+    return new Term(idField.getName(), idFieldType.toInternal(id));
+  }
+
+  protected final String getId(Document doc) {
+    if (idField == null) throw new SolrException(400,"Operation requires schema to have a unique key field");
+    String id = doc.get(idField.getName());
+    if (id == null) throw new SolrException(400,"Document is missing uniqueKey field " + idField.getName());
+    return id;
+  }
+
+  protected final String getOptId(Document doc) {
+    if (idField == null) return null;
+    return doc.get(idField.getName());
+  }
+
+
+  public abstract int addDoc(AddUpdateCommand cmd) throws IOException;
+  public abstract void delete(DeleteUpdateCommand cmd) throws IOException;
+  public abstract void deleteByQuery(DeleteUpdateCommand cmd) throws IOException;
+  public abstract void commit(CommitUpdateCommand cmd) throws IOException;
+  public abstract void close() throws IOException;
+
+
+  class DeleteHitCollector extends HitCollector {
+    public int deleted=0;
+    public final SolrIndexSearcher searcher;
+
+    public DeleteHitCollector(SolrIndexSearcher searcher) {
+      this.searcher = searcher;
+    }
+
+    public void collect(int doc, float score) {
+      try {
+        searcher.getReader().delete(doc);
+        deleted++;
+      } catch (IOException e) {
+        // don't try to close the searcher on failure for now...
+        // try { closeSearcher(); } catch (Exception ee) { SolrException.log(log,ee); }
+        throw new SolrException(500,"Error deleting doc# "+doc,e,false);
+      }
+    }
+  }
+
+}
+
+

Propchange: incubator/solr/trunk/src/java/org/apache/solr/update/UpdateHandler.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/solr/trunk/src/java/org/apache/solr/util/BCDUtils.java
URL: http://svn.apache.org/viewcvs/incubator/solr/trunk/src/java/org/apache/solr/util/BCDUtils.java?rev=372455&view=auto
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/util/BCDUtils.java (added)
+++ incubator/solr/trunk/src/java/org/apache/solr/util/BCDUtils.java Wed Jan 25 21:37:29 2006
@@ -0,0 +1,532 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.util;
+
+
+/**
+ * @author yonik
+ * @version $Id$
+ */
+public class BCDUtils {
+  // idiv is expensive...
+  // use fixed point math to multiply by 1/10
+  // http://www.cs.uiowa.edu/~jones/bcd/divide.html
+  private static int div10(int a) { return (a * 0xcccd) >>> 19; }
+  private static int mul10(int a) { return (a*10); }
+  // private static int mul10(int a) { return ((a<<3)+(a<<1)); }
+  // private static int mul10(int a) { return (a+(a<<2))<<1; } // attempt to use LEA instr
+  // (imul32 on AMD64 only has a 3 cycle latency in any case)
+
+
+  // something that won't clash with other base100int
+// chars (something >= 100)
+  private static final char NEG_CHAR=(char)126;
+  // The zero exponent.
+// NOTE: for smaller integer representations, this current implementation
+// combines sign and exponent into the first char.  sign is negative if
+// exponent is less than the zero point (no negative exponents themselves)
+  private static final int ZERO_EXPONENT='a';  // 97
+
+  // WARNING: assumption is that this is a legal int...
+// no validation is done.  [+-]?digit*
+//
+// Normalization of zeros *is* done...
+//  0004, 004, 04, 4 will all end up being equal
+//  0,-0 are normalized to '' (zero length)
+//
+// The value is written to the output buffer
+// from the end to the start.  The return value
+// is the start of the Base100 int in the output buffer.
+//
+// As the output will be smaller than the input, arr and
+// out may refer to the same array if desired.
+//
+  public static int base10toBase100(char[] arr, int start, int end,
+                                    char[] out, int outend
+                                    )
+  {
+    int wpos=outend;  // write position
+    boolean neg=false;
+
+    while (--end >= start) {
+      int val = arr[end];
+      if (val=='+') { break; }
+      else if (val=='-') { neg=!neg; break; }
+      else {
+        val = val - '0';
+        if (end > start) {
+          int val2 = arr[end-1];
+          if (val2=='+') { out[--wpos]=(char)val; break; }
+          if (val2=='-') { out[--wpos]=(char)val; neg=!neg; break; }
+          end--;
+          val = val + (val2 - '0')*10;
+        }
+        out[--wpos] = (char)val;
+      }
+    }
+
+    // remove leading base100 zeros
+    while (wpos<outend && out[wpos]==0) wpos++;
+
+    // check for a zero value
+    if (wpos==outend) {
+      // if zero, don't add negative sign
+    } else if (neg) {
+      out[--wpos]=NEG_CHAR;
+    }
+
+    return wpos;  // the start of the base100 int
+  }
+
+  // Converts a base100 number to base10 character form
+// returns number of chars written.
+// At least 1 char is always written.
+  public static int base100toBase10(char[] arr, int start, int end,
+                                    char[] out, int offset)
+  {
+    int wpos=offset;  // write position
+    boolean firstDigit=true;
+    for (int i=start; i<end; i++) {
+      int val = arr[i];
+      if (val== NEG_CHAR) { out[wpos++]='-'; continue; }
+      char tens = (char)(val / 10 + '0');
+      if (!firstDigit || tens!='0') {  // skip leading 0
+        out[wpos++] = (char)(val / 10 + '0');    // tens position
+      }
+      out[wpos++] = (char)(val % 10 + '0');    // ones position
+      firstDigit=false;
+    }
+    if (firstDigit) out[wpos++]='0';
+    return wpos-offset;
+  }
+
+  public static String base10toBase100SortableInt(String val) {
+    char[] arr = new char[val.length()+1];
+    val.getChars(0,val.length(),arr,0);
+    int len = base10toBase100SortableInt(arr,0,val.length(),arr,arr.length);
+    return new String(arr,arr.length-len,len);
+  }
+
+  public static String base100SortableIntToBase10(String val) {
+    int slen = val.length();
+    char[] arr = new char[slen<<2];
+    val.getChars(0,slen,arr,0);
+    int len = base100SortableIntToBase10(arr,0,slen,arr,slen);
+    return new String(arr,slen,len);
+  }
+
+  public static String base10toBase10kSortableInt(String val) {
+    char[] arr = new char[val.length()+1];
+    val.getChars(0,val.length(),arr,0);
+    int len = base10toBase10kSortableInt(arr,0,val.length(),arr,arr.length);
+    return new String(arr,arr.length-len,len);
+  }
+
+  public static String base10kSortableIntToBase10(String val) {
+    int slen = val.length();
+    char[] arr = new char[slen*5]; // +1 time for orig, +4 for new
+    val.getChars(0,slen,arr,0);
+    int len = base10kSortableIntToBase10(arr,0,slen,arr,slen);
+    return new String(arr,slen,len);
+  }
+
+  /********* FUTURE
+    // the zero exponent... exponents above this point are positive
+    // and below are negative.
+    // It is desirable to make ordinary numbers have a single byte
+    // exponent when converted to UTF-8
+    // For integers, the exponent will always be >=0, but this format
+    // is meant to be valid for floating point numbers as well...
+    private static final int ZERO_EXPONENT='a';  // 97
+
+    // if exponent is larger than what can be represented
+    // in a single byte (char), then this is the multibyte
+    // escape char.
+    // UCS-2 surrogates start at 0xD800
+    private static final int POSITIVE_EXPONENT_ESCAPE=0x3fff;
+
+    // if exponent is smaller than what can be represented in
+    // a single byte, then this is the multibyte escape
+    private static final int NEGATIVE_EXPONENT_ESCAPE=1;
+
+    // if number is negative, it starts with this optional value
+    // this should not overlap with any exponent values
+    private static final int NEGATIVE_SIGN=0;
+  **********/
+
+    // WARNING: assumption is that this is a legal int...
+    // no validation is done.  [+-]?digit*
+    //
+    // Normalization of zeros *is* done...
+    //  0004, 004, 04, 4 will all end up being equal
+    //  0,-0 are normalized to '' (zero length)
+    //
+    // The value is written to the output buffer
+    // from the end to the start.  The return value
+    // is the start of the Base100 int in the output buffer.
+    //
+    // As the output will be smaller than the input, arr and
+    // out may refer to the same array if desired.
+    //
+    public static int base10toBase100SortableInt(char[] arr, int start, int end,
+                                                 char[] out, int outend
+                                      )
+    {
+      int wpos=outend;  // write position
+      boolean neg=false;
+      --end;  // position end pointer *on* the last char
+
+      // read signs and leading zeros
+      while (start <= end) {
+        char val = arr[start];
+        if (val=='-') neg=!neg;
+        else if (val>='1' && val<='9') break;
+        start++;
+      }
+
+      // eat whitespace on RHS?
+      outer: while (start <= end) {
+        switch(arr[end]) {
+          case ' ':
+          case '\t':
+          case '\n':
+          case '\r': end--; break;
+          default: break outer;
+        }
+      }
+
+      int hundreds=0;
+      /******************************************************
+       * remove RHS zero normalization since it only helps 1 in 100
+       * numbers and complicates both encoding and decoding.
+
+      // remove pairs of zeros on the RHS and keep track of
+      // the count.
+      while (start <= end) {
+        char val = arr[end];
+
+        if (val=='0' && start <= end) {
+          val=arr[end-1];
+          if (val=='0') {
+            hundreds++;
+            end-=2;
+            continue;
+          }
+        }
+
+        break;
+      }
+      *************************************************************/
+
+
+      // now start at the end and work our way forward
+      // encoding two base 10 digits into 1 base 100 digit
+      while (start <= end) {
+        int val = arr[end--];
+        val = val - '0';
+        if (start <= end) {
+          int val2 = arr[end--];
+          val = val + (val2 - '0')*10;
+        }
+        out[--wpos] = neg ? (char)(99-val) : (char)val;
+      }
+
+      /****** FUTURE: not needed for this implementation of exponent combined with sign
+      // normalize all zeros to positive values
+      if (wpos==outend) neg=false;
+      ******/
+
+      // adjust exponent by the number of base 100 chars written
+      hundreds += outend - wpos;
+
+      // write the exponent and sign combined
+      out[--wpos] = neg ? (char)(ZERO_EXPONENT - hundreds) : (char)(ZERO_EXPONENT + hundreds);
+
+      return outend-wpos;  // the length of the base100 int
+    }
+
+  // Converts a base100 sortable number to base10 character form
+// returns number of chars written.
+// At least 1 char is always written.
+  public static int base100SortableIntToBase10(char[] arr, int start, int end,
+                                               char[] out, int offset)
+  {
+    // Take care of "0" case first.  It's the only number that is represented
+    // in one char.
+    if (end-start == 1) {
+      out[offset]='0';
+      return 1;
+    }
+
+    int wpos = offset;  // write position
+    boolean neg = false;
+    int exp = arr[start++];
+    if (exp < ZERO_EXPONENT) {
+      neg=true;
+      exp = ZERO_EXPONENT - exp;
+      out[wpos++]='-';
+    }
+
+    boolean firstDigit=true;
+    while (start < end) {
+      int val = arr[start++];
+      if (neg) val = 99 - val;
+      // opt - if we ever want a faster version we can avoid one integer
+      // divide by using fixed point math to multiply by 1/10
+      // http://www.cs.uiowa.edu/~jones/bcd/divide.html
+      // TIP: write a small function in gcc or cl and see what
+      // the optimized assemply output looks like (and which is fastest).
+      // In C you can specify "unsigned" which gives the compiler more
+      // info than the Java compiler has.
+      char tens = (char)(val / 10 + '0');
+      if (!firstDigit || tens!='0') {  // skip leading 0
+        out[wpos++] = tens;      // write tens position
+      }
+      out[wpos++] = (char)(val % 10 + '0');    // write ones position
+      firstDigit=false;
+    }
+
+    // OPTIONAL: if trailing zeros were truncated, then this is where
+    // we would restore them (compare number of chars read vs exponent)
+
+    return wpos-offset;
+  }
+
+  public static int base10toBase10kSortableInt(char[] arr, int start, int end,
+                                               char[] out, int outend
+                                    )
+  {
+    int wpos=outend;  // write position
+    boolean neg=false;
+    --end;  // position end pointer *on* the last char
+
+    // read signs and leading zeros
+    while (start <= end) {
+      char val = arr[start];
+      if (val=='-') neg=!neg;
+      else if (val>='1' && val<='9') break;
+      start++;
+    }
+
+    // eat whitespace on RHS?
+    outer: while (start <= end) {
+      switch(arr[end]) {
+        case ' ': // fallthrough
+        case '\t': // fallthrough
+        case '\n': // fallthrough
+        case '\r': end--; break;
+        default: break outer;
+      }
+    }
+
+    int exp=0;
+
+    /******************************************************
+     * remove RHS zero normalization since it only helps 1 in 100
+     * numbers and complicates both encoding and decoding.
+
+    // remove pairs of zeros on the RHS and keep track of
+    // the count.
+    while (start <= end) {
+      char val = arr[end];
+
+      if (val=='0' && start <= end) {
+        val=arr[end-1];
+        if (val=='0') {
+          hundreds++;
+          end-=2;
+          continue;
+        }
+      }
+
+      break;
+    }
+    *************************************************************/
+
+
+    // now start at the end and work our way forward
+    // encoding two base 10 digits into 1 base 100 digit
+    while (start <= end) {
+      int val = arr[end--] - '0';          // ones
+      if (start <= end) {
+        val += (arr[end--] - '0')*10;      // tens
+        if (start <= end) {
+          val += (arr[end--] - '0')*100;    // hundreds
+          if (start <= end) {
+            val += (arr[end--] - '0')*1000;  // thousands
+          }
+        }
+      }
+      out[--wpos] = neg ? (char)(9999-val) : (char)val;
+    }
+
+
+    /****** FUTURE: not needed for this implementation of exponent combined with sign
+    // normalize all zeros to positive values
+    if (wpos==outend) neg=false;
+    ******/
+
+    // adjust exponent by the number of base 100 chars written
+    exp += outend - wpos;
+
+    // write the exponent and sign combined
+    out[--wpos] = neg ? (char)(ZERO_EXPONENT - exp) : (char)(ZERO_EXPONENT + exp);
+
+    return outend-wpos;  // the length of the base100 int
+  }
+
+  // Converts a base100 sortable number to base10 character form
+// returns number of chars written.
+// At least 1 char is always written.
+  public static int base10kSortableIntToBase10(char[] arr, int start, int end,
+                                               char[] out, int offset)
+  {
+    // Take care of "0" case first.  It's the only number that is represented
+    // in one char since we don't chop trailing zeros.
+    if (end-start == 1) {
+      out[offset]='0';
+      return 1;
+    }
+
+    int wpos = offset;  // write position
+    boolean neg;
+    int exp = arr[start++];
+    if (exp < ZERO_EXPONENT) {
+      neg=true;
+      // We don't currently use exp on decoding...
+      // exp = ZERO_EXPONENT - exp;
+      out[wpos++]='-';
+    } else {
+      neg=false;
+    }
+
+    // since so many values will fall in one char, pull it
+    // out of the loop (esp since the first value must
+    // be special-cased to not print leading zeros.
+    // integer division is still expensive, so it's best to check
+    // if you actually need to do it.
+    //
+    // TIP: write a small function in gcc or cl and see what
+    // the optimized assemply output looks like (and which is fastest).
+    // In C you can specify "unsigned" which gives the compiler more
+    // info than the Java compiler has.
+    int val = arr[start++];
+    if (neg) val = 9999 - val;
+
+    /***
+    if (val < 10) {
+      out[wpos++] = (char)(val + '0');
+    } else if (val < 100) {
+      out[wpos++] = (char)(val/10 + '0');
+      out[wpos++] = (char)(val%10 + '0');
+    } else if (val < 1000) {
+      out[wpos++] = (char)(val/100 + '0');
+      out[wpos++] = (char)((val/10)%10 + '0');
+      out[wpos++] = (char)(val%10 + '0');
+    } else {
+      out[wpos++] = (char)(val/1000 + '0');
+      out[wpos++] = (char)((val/100)%10 + '0');
+      out[wpos++] = (char)((val/10)%10 + '0');
+      out[wpos++] = (char)(val % 10 + '0');
+    }
+    ***/
+
+    if (val < 10) {
+      out[wpos++] = (char)(val + '0');
+    } else if (val < 100) {
+      int div = div10(val);
+      int ones = val - mul10(div); // mod 10
+      out[wpos++] = (char)(div + '0');
+      out[wpos++] = (char)(ones + '0');
+    } else if (val < 1000) {
+      int div = div10(val);
+      int ones = val - mul10(div); // mod 10
+      val=div;
+      div = div10(val);
+      int tens = val - mul10(div); // mod 10
+      out[wpos++] = (char)(div + '0');
+      out[wpos++] = (char)(tens + '0');
+      out[wpos++] = (char)(ones + '0');
+    } else {
+      int div = div10(val);
+      int ones = val - mul10(div); // mod 10
+      val=div;
+      div = div10(val);
+      int tens = val - mul10(div); // mod 10
+      val=div;
+      div = div10(val);
+      int hundreds = val - mul10(div); // mod 10
+
+      out[wpos++] = (char)(div + '0');
+      out[wpos++] = (char)(hundreds + '0');
+      out[wpos++] = (char)(tens + '0');
+      out[wpos++] = (char)(ones + '0');
+    }
+
+
+    while (start < end) {
+      val = arr[start++];
+      if (neg) val = 9999 - val;
+
+      int div = div10(val);
+      int ones = val - mul10(div); // mod 10
+      val=div;
+      div = div10(val);
+      int tens = val - mul10(div); // mod 10
+      val=div;
+      div = div10(val);
+      int hundreds = val - mul10(div); // mod 10
+
+      /***
+      int ones = val % 10;
+      val /= 10;
+      int tens = val!=0 ? val % 10 : 0;
+      val /= 10;
+      int hundreds = val!=0 ? val % 10 : 0;
+      val /= 10;
+      int thousands = val!=0 ? val % 10 : 0;
+      ***/
+
+      /***
+      int thousands = val>=1000 ? val/1000 : 0;
+      int hundreds  = val>=100 ? (val/100)%10 : 0;
+      int tens      = val>=10 ? (val/10)%10 : 0;
+      int ones      = val % 10;
+      ***/
+
+      /***
+      int thousands =  val/1000;
+      int hundreds  = (val/100)%10;
+      int tens      = (val/10)%10;
+      int ones      = val % 10;
+      ***/
+
+      out[wpos++] = (char)(div + '0');
+      out[wpos++] = (char)(hundreds + '0');
+      out[wpos++] = (char)(tens + '0');
+      out[wpos++] = (char)(ones + '0');
+    }
+
+    // OPTIONAL: if trailing zeros were truncated, then this is where
+    // we would restore them (compare number of chars read vs exponent)
+
+    return wpos-offset;
+  }
+
+
+
+}

Propchange: incubator/solr/trunk/src/java/org/apache/solr/util/BCDUtils.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/solr/trunk/src/java/org/apache/solr/util/DOMUtil.java
URL: http://svn.apache.org/viewcvs/incubator/solr/trunk/src/java/org/apache/solr/util/DOMUtil.java?rev=372455&view=auto
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/util/DOMUtil.java (added)
+++ incubator/solr/trunk/src/java/org/apache/solr/util/DOMUtil.java Wed Jan 25 21:37:29 2006
@@ -0,0 +1,152 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.util;
+
+import org.w3c.dom.NamedNodeMap;
+import org.w3c.dom.Node;
+import org.w3c.dom.NodeList;
+
+import java.util.Map;
+import java.util.HashMap;
+import java.util.List;
+import java.util.ArrayList;
+
+/**
+ * @author yonik
+ * @version $Id: DOMUtil.java,v 1.3 2005/12/01 16:50:11 yonik Exp $
+ */
+public class DOMUtil {
+
+  public static Map<String,String> toMap(NamedNodeMap attrs) {
+    return toMapExcept(attrs);
+  }
+
+  public static Map<String,String> toMapExcept(NamedNodeMap attrs, String... exclusions) {
+    Map<String,String> args = new HashMap<String,String>();
+    outer: for (int j=0; j<attrs.getLength(); j++) {
+      Node attr = attrs.item(j);
+      String attrName = attr.getNodeName();
+      for (String ex : exclusions)
+        if (ex.equals(attrName)) continue outer;
+      String val = attr.getNodeValue();
+      args.put(attrName, val);
+    }
+    return args;
+  }
+
+  public static Node getChild(Node node, String name) {
+    if (!node.hasChildNodes()) return null;
+    NodeList lst = node.getChildNodes();
+    if (lst == null) return null;
+    for (int i=0; i<lst.getLength(); i++) {
+      Node child = lst.item(i);
+      if (name.equals(child.getNodeName())) return child;
+    }
+    return null;
+  }
+
+  public static String getAttr(NamedNodeMap attrs, String name) {
+    return getAttr(attrs,name,null);
+  }
+
+  public static String getAttr(Node nd, String name) {
+    return getAttr(nd.getAttributes(), name);
+  }
+
+  public static String getAttr(NamedNodeMap attrs, String name, String missing_err) {
+    Node attr = attrs==null? null : attrs.getNamedItem(name);
+    if (attr==null) {
+      if (missing_err==null) return null;
+      throw new RuntimeException(missing_err + ": missing mandatory attribute '" + name + "'");
+    }
+    String val = attr.getNodeValue();
+    return val;
+  }
+
+  public static String getAttr(Node node, String name, String missing_err) {
+    return getAttr(node.getAttributes(), name, missing_err);
+  }
+
+  //////////////////////////////////////////////////////////
+  // Routines to parse XML in the syntax of the Solr query
+  // response schema.
+  // Should these be moved to Config?  Should all of these things?
+  //////////////////////////////////////////////////////////
+  public static NamedList childNodesToNamedList(Node nd) {
+    return nodesToNamedList(nd.getChildNodes());
+  }
+
+  public static List childNodesToList(Node nd) {
+    return nodesToList(nd.getChildNodes());
+  }
+
+  public static NamedList nodesToNamedList(NodeList nlst) {
+    NamedList clst = new NamedList();
+    for (int i=0; i<nlst.getLength(); i++) {
+      addToNamedList(nlst.item(i), clst, null);
+    }
+    return clst;
+  }
+
+  public static List nodesToList(NodeList nlst) {
+    List lst = new ArrayList();
+    for (int i=0; i<nlst.getLength(); i++) {
+      addToNamedList(nlst.item(i), null, lst);
+    }
+    return lst;
+  }
+
+
+  public static void addToNamedList(Node nd, NamedList nlst, List arr) {
+    // Nodes often include whitespace, etc... so just return if this
+    // is not an Element.
+    if (nd.getNodeType() != Node.ELEMENT_NODE) return;
+
+    String type = nd.getNodeName();
+
+    String name = null;
+    if (nd.hasAttributes()) {
+      NamedNodeMap attrs = nd.getAttributes();
+      Node nameNd = attrs.getNamedItem("name");
+      if (nameNd != null) name=nameNd.getNodeValue();
+    }
+
+    Object val=null;
+
+    if ("str".equals(type)) {
+      val = nd.getTextContent();
+    } else if ("int".equals(type)) {
+      val = Integer.valueOf(nd.getTextContent());
+    } else if ("long".equals(type)) {
+      val = Long.valueOf(nd.getTextContent());
+    } else if ("float".equals(type)) {
+      val = Float.valueOf(nd.getTextContent());
+    } else if ("double".equals(type)) {
+      val = Double.valueOf(nd.getTextContent());
+    } else if ("bool".equals(type)) {
+      val = Boolean.valueOf(nd.getTextContent());
+    } else if ("lst".equals(type)) {
+      val = childNodesToNamedList(nd);
+    } else if ("arr".equals(type)) {
+      val = childNodesToList(nd);
+    }
+
+    if (nlst != null) nlst.add(name,val);
+    if (arr != null) arr.add(val);
+  }
+
+}

Propchange: incubator/solr/trunk/src/java/org/apache/solr/util/DOMUtil.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/solr/trunk/src/java/org/apache/solr/util/NamedList.java
URL: http://svn.apache.org/viewcvs/incubator/solr/trunk/src/java/org/apache/solr/util/NamedList.java?rev=372455&view=auto
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/util/NamedList.java (added)
+++ incubator/solr/trunk/src/java/org/apache/solr/util/NamedList.java Wed Jan 25 21:37:29 2006
@@ -0,0 +1,143 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.util;
+
+import java.util.*;
+import java.io.Serializable;
+
+/**
+ * @author yonik
+ * @version $Id$
+ */
+//
+// A quick hack of a class to represent a list of name-value pairs.
+// Unlike a map, order is maintained, and names may
+// be repeated.  Names and values may be null.
+//
+// In the future, it would be nice if this extended Map or Collection,
+// had iterators, used java5 generics, had a faster lookup for
+// large lists, etc...
+// It could also have an interface, and multiple implementations.
+// One might have indexed lookup, one might not.
+//
+public class NamedList implements Cloneable, Serializable {
+  protected final List nvPairs;
+
+  public NamedList() {
+    nvPairs = new ArrayList();
+  }
+
+  public NamedList(List nameValuePairs) {
+    nvPairs=nameValuePairs;
+  }
+
+  public int size() {
+    return nvPairs.size() >> 1;
+  }
+
+  public String getName(int idx) {
+    return (String)nvPairs.get(idx << 1);
+  }
+
+  public Object getVal(int idx) {
+    return nvPairs.get((idx << 1) + 1);
+  }
+
+  public void add(String name, Object val) {
+    nvPairs.add(name);
+    nvPairs.add(val);
+  }
+
+  public void setName(int idx, String name) {
+    nvPairs.set(idx<<1, name);
+  }
+
+  public void setVal(int idx, Object val) {
+    nvPairs.set((idx<<1)+1, val);
+  }
+
+  public int indexOf(String name, int start) {
+    int sz = size();
+    for (int i=start; i<sz; i++) {
+      String n = getName(i);
+      if (name==null) {
+        if (n==null) return i; // matched null
+      } else if (name.equals(n)) {
+        return i;
+      }
+    }
+    return -1;
+  }
+
+
+  // gets the value for the first specified name. returns null if not
+  // found or if the value stored was null.
+  public Object get(String name) {
+    return get(name,0);
+  }
+
+  // gets the value for the first specified name starting start.
+  // returns null if not found or if the value stored was null.
+  public Object get(String name, int start) {
+    int sz = size();
+    for (int i=start; i<sz; i++) {
+      String n = getName(i);
+      if (name==null) {
+        if (n==null) return getVal(i);
+      } else if (name.equals(n)) {
+        return getVal(i);
+      }
+    }
+    return null;
+  }
+
+  public String toString() {
+    StringBuffer sb = new StringBuffer();
+    sb.append('{');
+    int sz = size();
+    for (int i=0; i<sz; i++) {
+      if (i != 0) sb.append(',');
+      sb.append(getName(i));
+      sb.append('=');
+      sb.append(getVal(i));
+    }
+    sb.append('}');
+
+    return sb.toString();
+  }
+
+
+  public boolean addAll(Map args) {
+    Set eset = args.entrySet();
+    Iterator iter = eset.iterator();
+    while (iter.hasNext()) {
+      Map.Entry entry = (Map.Entry)iter.next();
+      add(entry.getKey().toString(), entry.getValue());
+    }
+    return false;
+  }
+
+  /**
+   * Makes a *shallow copy* of the named list.
+   */
+  public NamedList clone() {
+    ArrayList newList = new ArrayList(nvPairs.size());
+    newList.addAll(nvPairs);
+    return new NamedList(newList);
+  }
+
+}

Propchange: incubator/solr/trunk/src/java/org/apache/solr/util/NamedList.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/solr/trunk/src/java/org/apache/solr/util/NumberUtils.java
URL: http://svn.apache.org/viewcvs/incubator/solr/trunk/src/java/org/apache/solr/util/NumberUtils.java?rev=372455&view=auto
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/util/NumberUtils.java (added)
+++ incubator/solr/trunk/src/java/org/apache/solr/util/NumberUtils.java Wed Jan 25 21:37:29 2006
@@ -0,0 +1,159 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.util;
+
+/**
+ * @author yonik
+ * @version $Id$
+ */
+public class NumberUtils {
+
+
+
+
+
+  public static String int2sortableStr(int val) {
+    char[] arr = new char[3];
+    int2sortableStr(val,arr,0);
+    return new String(arr,0,3);
+  }
+
+  public static String int2sortableStr(String val) {
+    return int2sortableStr(Integer.parseInt(val));
+  }
+
+  public static String SortableStr2int(String val) {
+    int ival = SortableStr2int(val,0,3);
+    return Integer.toString(ival);
+  }
+
+
+  public static String long2sortableStr(long val) {
+    char[] arr = new char[5];
+    long2sortableStr(val,arr,0);
+    return new String(arr,0,5);
+  }
+
+  public static String long2sortableStr(String val) {
+    return long2sortableStr(Long.parseLong(val));
+  }
+
+  public static String SortableStr2long(String val) {
+    long ival = SortableStr2long(val,0,5);
+    return Long.toString(ival);
+  }
+
+  //
+  // IEEE floating point format is defined so that it sorts correctly
+  // when interpreted as a signed integer (or signed long in the case
+  // of a double) for positive values.  For negative values, all the bits except
+  // the sign bit must be inverted.
+  // This correctly handles all possible float values including -Infinity and +Infinity.
+  // Note that in float-space, NaN<x is false, NaN>x is false, NaN==x is false, NaN!=x is true
+  // for all x (including NaN itself).  Internal to Solr, NaN==NaN is true and NaN
+  // sorts higher than Infinity, so a range query of [-Infinity TO +Infinity] will
+  // exclude NaN values, but a query of "NaN" will find all NaN values.
+  // Also, -0==0 in float-space but -0<0 after this transformation.
+  //
+  public static String float2sortableStr(float val) {
+    int f = Float.floatToRawIntBits(val);
+    if (f<0) f ^= 0x7fffffff;
+    return int2sortableStr(f);
+  }
+
+  public static String float2sortableStr(String val) {
+    return float2sortableStr(Float.parseFloat(val));
+  }
+
+  public static float SortableStr2float(String val) {
+    int f = SortableStr2int(val,0,3);
+    if (f<0) f ^= 0x7fffffff;
+    return Float.intBitsToFloat(f);
+  }
+
+  public static String SortableStr2floatStr(String val) {
+    return Float.toString(SortableStr2float(val));
+  }
+
+
+  public static String double2sortableStr(double val) {
+    long f = Double.doubleToRawLongBits(val);
+    if (f<0) f ^= 0x7fffffffffffffffL;
+    return long2sortableStr(f);
+  }
+
+  public static String double2sortableStr(String val) {
+    return double2sortableStr(Double.parseDouble(val));
+  }
+
+  public static double SortableStr2double(String val) {
+    long f = SortableStr2long(val,0,6);
+    if (f<0) f ^= 0x7fffffffffffffffL;
+    return Double.longBitsToDouble(f);
+  }
+
+  public static String SortableStr2doubleStr(String val) {
+    return Double.toString(SortableStr2double(val));
+  }
+
+
+
+  // uses binary representation of an int to build a string of
+  // chars that will sort correctly.  Only char ranges
+  // less than 0xd800 will be used to avoid UCS-16 surrogates.
+  public static int int2sortableStr(int val, char[] out, int offset) {
+    val += Integer.MIN_VALUE;
+    out[offset++] = (char)(val >>> 24);
+    out[offset++] = (char)((val >>> 12) & 0x0fff);
+    out[offset++] = (char)(val & 0x0fff);
+    return 3;
+  }
+
+  public static int SortableStr2int(String sval, int offset, int len) {
+    int val = sval.charAt(offset++) << 24;
+    val |= sval.charAt(offset++) << 12;
+    val |= sval.charAt(offset++);
+    val -= Integer.MIN_VALUE;
+    return val;
+  }
+
+  // uses binary representation of an int to build a string of
+  // chars that will sort correctly.  Only char ranges
+  // less than 0xd800 will be used to avoid UCS-16 surrogates.
+  // we can use the lowest 15 bits of a char, (or a mask of 0x7fff)
+  public static int long2sortableStr(long val, char[] out, int offset) {
+    val += Long.MIN_VALUE;
+    out[offset++] = (char)(val >>>60);
+    out[offset++] = (char)(val >>>45 & 0x7fff);
+    out[offset++] = (char)(val >>>30 & 0x7fff);
+    out[offset++] = (char)(val >>>15 & 0x7fff);
+    out[offset] = (char)(val & 0x7fff);
+    return 5;
+  }
+
+  public static long SortableStr2long(String sval, int offset, int len) {
+    long val = (long)(sval.charAt(offset++)) << 60;
+    val |= ((long)sval.charAt(offset++)) << 45;
+    val |= ((long)sval.charAt(offset++)) << 30;
+    val |= sval.charAt(offset++) << 15;
+    val |= sval.charAt(offset);
+    val -= Long.MIN_VALUE;
+    return val;
+  }
+
+
+}

Propchange: incubator/solr/trunk/src/java/org/apache/solr/util/NumberUtils.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: incubator/solr/trunk/src/java/org/apache/solr/util/RefCounted.java
URL: http://svn.apache.org/viewcvs/incubator/solr/trunk/src/java/org/apache/solr/util/RefCounted.java?rev=372455&view=auto
==============================================================================
--- incubator/solr/trunk/src/java/org/apache/solr/util/RefCounted.java (added)
+++ incubator/solr/trunk/src/java/org/apache/solr/util/RefCounted.java Wed Jan 25 21:37:29 2006
@@ -0,0 +1,34 @@
+/**
+ * Copyright 2006 The Apache Software Foundation
+ *
+ * Licensed under the Apache License, Version 2.0 (the "License");
+ * you may not use this file except in compliance with the License.
+ * You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.solr.util;
+
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * @author yonik
+ * @version $Id: RefCounted.java,v 1.2 2005/09/07 20:37:57 yonik Exp $
+ */
+
+public abstract class RefCounted<Type> {
+  protected final Type resource;
+  protected final AtomicInteger refcount= new AtomicInteger();
+  public RefCounted(Type resource) { this.resource = resource; }
+  public final RefCounted<Type> incref() { refcount.incrementAndGet(); return this; }
+  public final Type get() { return resource; }
+  public void decref() { if (refcount.decrementAndGet()==0) close(); }
+  protected abstract void close();
+}

Propchange: incubator/solr/trunk/src/java/org/apache/solr/util/RefCounted.java
------------------------------------------------------------------------------
    svn:eol-style = native