You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@manifoldcf.apache.org by kw...@apache.org on 2014/07/15 14:58:51 UTC

svn commit: r1610676 - in /manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler: interfaces/IJobManager.java jobs/Carrydown.java jobs/HopCount.java jobs/IntrinsicLink.java jobs/JobManager.java

Author: kwright
Date: Tue Jul 15 12:58:51 2014
New Revision: 1610676

URL: http://svn.apache.org/r1610676
Log:
Add reversion code for restoring carrydown and hopcount links to their original condition.  This code may not wind up being used, but the refactoring involved in creating it will be helpful regardless.

Modified:
    manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IJobManager.java
    manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/Carrydown.java
    manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/HopCount.java
    manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/IntrinsicLink.java
    manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/JobManager.java

Modified: manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IJobManager.java
URL: http://svn.apache.org/viewvc/manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IJobManager.java?rev=1610676&r1=1610675&r2=1610676&view=diff
==============================================================================
--- manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IJobManager.java (original)
+++ manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IJobManager.java Tue Jul 15 12:58:51 2014
@@ -717,6 +717,18 @@ public interface IJobManager
     String[] parentIdentifierHashes, int hopcountMethod)
     throws ManifoldCFException;
 
+  /** Undo the addition of child documents to the queue, for a set of documents.
+  * This method is called at the end of document processing, to back out any incomplete additions to the queue, and restore
+  * the status quo ante prior to the incomplete additions.  Call this method instead of finishDocuments() if the
+  * addition of documents was not completed.
+  *@param jobID is the job identifier.
+  *@param legalLinkTypes is the set of legal link types that this connector generates.
+  *@param parentIdentifierHashes are the hashes of the document identifiers for whom child link extraction just took place.
+  */
+  public void revertDocuments(Long jobID, String[] legalLinkTypes,
+    String[] parentIdentifierHashes)
+    throws ManifoldCFException;
+
   /** Retrieve specific parent data for a given document.
   *@param jobID is the job identifier.
   *@param docIDHash is the hash of the document identifier.

Modified: manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/Carrydown.java
URL: http://svn.apache.org/viewvc/manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/Carrydown.java?rev=1610676&r1=1610675&r2=1610676&view=diff
==============================================================================
--- manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/Carrydown.java (original)
+++ manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/Carrydown.java Tue Jul 15 12:58:51 2014
@@ -457,57 +457,102 @@ public class Carrydown extends org.apach
       presentMap.put(vr,vr);
     }
   }
+  
+  /** Revert all records belonging to the specified parent documents to their original,
+  * pre-modified, state.
+  */
+  public void revertRecords(Long jobID, String[] parentDocumentIDHashes)
+    throws ManifoldCFException
+  {
+    int maxClause = getMaxInClause();
+    StringBuilder sb = new StringBuilder();
+    List<String> list = new ArrayList<String>();
+    int k = 0;
+    for (String parentDocumentIDHash : parentDocumentIDHashes)
+    {
+      if (k == maxClause)
+      {
+        performRevertRecords(sb.toString(),jobID,list);
+        sb.setLength(0);
+        list.clear();
+        k = 0;
+      }
+      if (k > 0)
+        sb.append(",");
+      sb.append("?");
+      list.add(parentDocumentIDHash);
+      k++;
+    }
+
+    if (k > 0)
+      performRevertRecords(sb.toString(),jobID,list);
+  }
+  
+  protected void performRevertRecords(String query, Long jobID, List<String> list)
+    throws ManifoldCFException
+  {
+    // Delete new records
+    StringBuilder sb = new StringBuilder("WHERE ");
+    ArrayList newList = new ArrayList();
+    
+    sb.append(buildConjunctionClause(newList,new ClauseDescription[]{
+      new UnitaryClause(jobIDField,jobID),
+      new MultiClause(parentIDHashField,list)})).append(" AND ");
+      
+    sb.append(newField).append("=?");
+    newList.add(statusToString(ISNEW_NEW));
+    performDelete(sb.toString(),newList,null);
+
+    // Restore old values
+    sb = new StringBuilder("WHERE ");
+    newList.clear();
+
+    sb.append(buildConjunctionClause(newList,new ClauseDescription[]{
+      new UnitaryClause(jobIDField,jobID),
+      new MultiClause(parentIDHashField,list)})).append(" AND ");
+
+    sb.append(newField).append("=?");
+    newList.add(statusToString(ISNEW_EXISTING));
+    
+    HashMap map = new HashMap();
+    map.put(newField,statusToString(ISNEW_BASE));
+    map.put(processIDField,null);
+    performUpdate(map,sb.toString(),newList,null);
+    
+    noteModifications(0,list.size(),0);
+  }
+
   /** Return all records belonging to the specified parent documents to the base state,
   * and delete the old (eliminated) child records.
   */
   public void restoreRecords(Long jobID, String[] parentDocumentIDHashes)
     throws ManifoldCFException
   {
-    beginTransaction();
-    try
+    int maxClause = getMaxInClause();
+    StringBuilder sb = new StringBuilder();
+    List<String> list = new ArrayList<String>();
+    int k = 0;
+    for (String parentDocumentIDHash : parentDocumentIDHashes)
     {
-      int maxClause = getMaxInClause();
-      StringBuilder sb = new StringBuilder();
-      ArrayList list = new ArrayList();
-      int i = 0;
-      int k = 0;
-      while (i < parentDocumentIDHashes.length)
+      if (k == maxClause)
       {
-        if (k == maxClause)
-        {
-          performRestoreRecords(sb.toString(),jobID,list);
-          sb.setLength(0);
-          list.clear();
-          k = 0;
-        }
-        if (k > 0)
-          sb.append(",");
-        sb.append("?");
-        String parentDocumentIDHash = parentDocumentIDHashes[i++];
-        list.add(parentDocumentIDHash);
-        k++;
+        performRestoreRecords(sb.toString(),jobID,list);
+        sb.setLength(0);
+        list.clear();
+        k = 0;
       }
-
       if (k > 0)
-        performRestoreRecords(sb.toString(),jobID,list);
-    }
-    catch (ManifoldCFException e)
-    {
-      signalRollback();
-      throw e;
-    }
-    catch (Error e)
-    {
-      signalRollback();
-      throw e;
-    }
-    finally
-    {
-      endTransaction();
+        sb.append(",");
+      sb.append("?");
+      list.add(parentDocumentIDHash);
+      k++;
     }
+
+    if (k > 0)
+      performRestoreRecords(sb.toString(),jobID,list);
   }
 
-  protected void performRestoreRecords(String query, Long jobID, ArrayList list)
+  protected void performRestoreRecords(String query, Long jobID, List<String> list)
     throws ManifoldCFException
   {
     // Delete
@@ -547,45 +592,23 @@ public class Carrydown extends org.apach
   public void deleteRecords(Long jobID, String[] documentIDHashes)
     throws ManifoldCFException
   {
-    beginTransaction();
-    try
+    int maxClause = maxClausePerformDeleteRecords(jobID);
+    List<String> list = new ArrayList<String>();
+    int k = 0;
+    for (String documentIDHash : documentIDHashes)
     {
-      int maxClause = maxClausePerformDeleteRecords(jobID);
-      ArrayList list = new ArrayList();
-      int i = 0;
-      int k = 0;
-      while (i < documentIDHashes.length)
+      if (k == maxClause)
       {
-        if (k == maxClause)
-        {
-          performDeleteRecords(jobID,list);
-          list.clear();
-          k = 0;
-        }
-        list.add(documentIDHashes[i++]);
-        k++;
-      }
-
-      if (k > 0)
         performDeleteRecords(jobID,list);
-
-
-    }
-    catch (ManifoldCFException e)
-    {
-      signalRollback();
-      throw e;
-    }
-    catch (Error e)
-    {
-      signalRollback();
-      throw e;
-    }
-    finally
-    {
-      endTransaction();
+        list.clear();
+        k = 0;
+      }
+      list.add(documentIDHash);
+      k++;
     }
 
+    if (k > 0)
+      performDeleteRecords(jobID,list);
   }
 
   protected int maxClausePerformDeleteRecords(Long jobID)
@@ -594,7 +617,7 @@ public class Carrydown extends org.apach
       new UnitaryClause(jobIDField,jobID)});
   }
     
-  protected void performDeleteRecords(Long jobID, ArrayList list)
+  protected void performDeleteRecords(Long jobID, List<String> list)
     throws ManifoldCFException
   {
     StringBuilder sb = new StringBuilder("WHERE ");

Modified: manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/HopCount.java
URL: http://svn.apache.org/viewvc/manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/HopCount.java?rev=1610676&r1=1610675&r2=1610676&view=diff
==============================================================================
--- manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/HopCount.java (original)
+++ manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/HopCount.java Tue Jul 15 12:58:51 2014
@@ -370,14 +370,26 @@ public class HopCount extends org.apache
     doFinish(jobID,legalLinkTypes,sourceDocumentHashes,hopcountMethod);
   }
 
+  /** Revert newly-added links, because of a possibly incomplete document processing phase.
+  * All child links marked as "new" will be removed, and all links marked as "existing" will be
+  * reset to be "base".
+  */
+  public void revertParents(Long jobID, String[] sourceDocumentHashes)
+    throws ManifoldCFException
+  {
+    intrinsicLinkManager.revertLinks(jobID,sourceDocumentHashes);
+  }
+  
   /** Do the work of recording source-target references. */
   protected boolean[] doRecord(Long jobID, String[] legalLinkTypes, String sourceDocumentIDHash, String[] targetDocumentIDHashes, String linkType,
     int hopcountMethod, String processID)
     throws ManifoldCFException
   {
-
+    // NOTE: This needs to be revised to not process any additions until the finishParents() call above.
+    // It is now possible to revert before finishParents() is ever called.
     // We have to both add the reference, AND invalidate appropriate cached hopcounts (if it is a NEW
     // link.)
+    // ???
     boolean[] rval = new boolean[targetDocumentIDHashes.length];
     for (int i = 0; i < rval.length; i++)
     {

Modified: manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/IntrinsicLink.java
URL: http://svn.apache.org/viewvc/manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/IntrinsicLink.java?rev=1610676&r1=1610675&r2=1610676&view=diff
==============================================================================
--- manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/IntrinsicLink.java (original)
+++ manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/IntrinsicLink.java Tue Jul 15 12:58:51 2014
@@ -232,19 +232,17 @@ public class IntrinsicLink extends org.a
     String[] targetDocumentIDHashes, String linkType, String processID)
     throws ManifoldCFException
   {
-    HashMap duplicateRemoval = new HashMap();
+    Set<String> duplicateRemoval = new HashSet<String>();
     int maxClause = maxClausePerformExistsCheck(jobID,linkType,sourceDocumentIDHash);
-    ArrayList list = new ArrayList();
+    List<String> list = new ArrayList<String>();
     int i = 0;
-    int k = 0;
     // Keep track of the document identifiers that have been seen vs. those that were unseen.
-    HashMap presentMap = new HashMap();
-    while (k < targetDocumentIDHashes.length)
+    Set<String> presentMap = new HashSet<String>();
+    for (String targetDocumentIDHash : targetDocumentIDHashes)
     {
-      String targetDocumentIDHash = targetDocumentIDHashes[k++];
-      if (duplicateRemoval.get(targetDocumentIDHash) != null)
+      if (duplicateRemoval.contains(targetDocumentIDHash))
         continue;
-      duplicateRemoval.put(targetDocumentIDHash,targetDocumentIDHash);
+      duplicateRemoval.add(targetDocumentIDHash);
       if (i == maxClause)
       {
         // Do the query and record the results
@@ -262,22 +260,22 @@ public class IntrinsicLink extends org.a
     // an update.
     // We have to count these by hand, in case there are duplicates in the array.
     int count = 0;
-    Iterator iter = duplicateRemoval.keySet().iterator();
+    Iterator<String> iter = duplicateRemoval.iterator();
     while (iter.hasNext())
     {
-      String targetDocumentIDHash = (String)iter.next();
-      if (presentMap.get(targetDocumentIDHash) == null)
+      String targetDocumentIDHash = iter.next();
+      if (!presentMap.contains(targetDocumentIDHash))
         count++;
     }
     String[] newReferences = new String[count];
     int j = 0;
     // Note: May be able to make this more efficient if we update things in batches...
-    iter = duplicateRemoval.keySet().iterator();
+    iter = duplicateRemoval.iterator();
     while (iter.hasNext())
     {
-      String targetDocumentIDHash = (String)iter.next();
+      String targetDocumentIDHash = iter.next();
 
-      if (presentMap.get(targetDocumentIDHash) == null)
+      if (!presentMap.contains(targetDocumentIDHash))
       {
         newReferences[j++] = targetDocumentIDHash;
         HashMap map = new HashMap();
@@ -319,7 +317,7 @@ public class IntrinsicLink extends org.a
   }
     
   /** Do the exists check, in batch. */
-  protected void performExistsCheck(Map presentMap, Long jobID, String linkType, String childIDHash, ArrayList list)
+  protected void performExistsCheck(Set<String> presentMap, Long jobID, String linkType, String childIDHash, List<String> list)
     throws ManifoldCFException
   {
     ArrayList newList = new ArrayList();
@@ -330,12 +328,11 @@ public class IntrinsicLink extends org.a
       new UnitaryClause(childIDHashField,childIDHash)});
 
     IResultSet result = performQuery("SELECT "+parentIDHashField+" FROM "+getTableName()+" WHERE "+query+" FOR UPDATE",newList,null,null);
-    int i = 0;
-    while (i < result.getRowCount())
+    for (int i = 0; i < result.getRowCount(); i++)
     {
-      IResultRow row = result.getRow(i++);
+      IResultRow row = result.getRow(i);
       String parentIDHash = (String)row.getValue(parentIDHashField);
-      presentMap.put(parentIDHash,parentIDHash);
+      presentMap.add(parentIDHash);
     }
   }
 
@@ -375,10 +372,9 @@ public class IntrinsicLink extends org.a
     throws ManifoldCFException
   {
     int maxClause = maxClausePerformRemoveDocumentLinks(jobID);
-    ArrayList list = new ArrayList();
-    int i = 0;
+    List<String> list = new ArrayList<String>();
     int k = 0;
-    while (i < documentIDHashes.length)
+    for (String documentIDHash : documentIDHashes)
     {
       if (k == maxClause)
       {
@@ -386,7 +382,7 @@ public class IntrinsicLink extends org.a
         list.clear();
         k = 0;
       }
-      list.add(documentIDHashes[i++]);
+      list.add(documentIDHash);
       k++;
     }
 
@@ -401,7 +397,7 @@ public class IntrinsicLink extends org.a
       new UnitaryClause(jobIDField,jobID)});
   }
     
-  protected void performRemoveDocumentLinks(ArrayList list, Long jobID)
+  protected void performRemoveDocumentLinks(List<String> list, Long jobID)
     throws ManifoldCFException
   {
     StringBuilder sb = new StringBuilder("WHERE ");
@@ -424,10 +420,9 @@ public class IntrinsicLink extends org.a
     throws ManifoldCFException
   {
     int maxClause = maxClausePerformRemoveLinks(jobID);
-    ArrayList list = new ArrayList();
-    int i = 0;
+    List<String> list = new ArrayList<String>();
     int k = 0;
-    while (i < sourceDocumentIDHashes.length)
+    for (String sourceDocumentIDHash : sourceDocumentIDHashes)
     {
       if (k == maxClause)
       {
@@ -435,7 +430,7 @@ public class IntrinsicLink extends org.a
         list.clear();
         k = 0;
       }
-      list.add(sourceDocumentIDHashes[i++]);
+      list.add(sourceDocumentIDHash);
       k++;
     }
 
@@ -450,7 +445,7 @@ public class IntrinsicLink extends org.a
       new UnitaryClause(jobIDField,jobID)});
   }
     
-  protected void performRemoveLinks(ArrayList list, Long jobID, String commonNewExpression,
+  protected void performRemoveLinks(List<String> list, Long jobID, String commonNewExpression,
     ArrayList commonNewParams)
     throws ManifoldCFException
   {
@@ -474,10 +469,9 @@ public class IntrinsicLink extends org.a
     throws ManifoldCFException
   {
     int maxClause = maxClausesPerformRestoreLinks(jobID);
-    ArrayList list = new ArrayList();
-    int i = 0;
+    List<String> list = new ArrayList<String>();
     int k = 0;
-    while (i < sourceDocumentIDHashes.length)
+    for (String sourceDocumentIDHash : sourceDocumentIDHashes)
     {
       if (k == maxClause)
       {
@@ -485,7 +479,7 @@ public class IntrinsicLink extends org.a
         list.clear();
         k = 0;
       }
-      list.add(sourceDocumentIDHashes[i++]);
+      list.add(sourceDocumentIDHash);
       k++;
     }
 
@@ -500,7 +494,7 @@ public class IntrinsicLink extends org.a
       new UnitaryClause(jobIDField,jobID)});
   }
   
-  protected void performRestoreLinks(Long jobID, ArrayList list)
+  protected void performRestoreLinks(Long jobID, List<String> list)
     throws ManifoldCFException
   {
     HashMap map = new HashMap();
@@ -519,6 +513,67 @@ public class IntrinsicLink extends org.a
     performUpdate(map,sb.toString(),newList,null);
   }
 
+  /** Throw away links added during (aborted) processing.
+  */
+  public void revertLinks(Long jobID, String[] sourceDocumentIDHashes)
+    throws ManifoldCFException
+  {
+    int maxClause = maxClausesPerformRevertLinks(jobID);
+    List<String> list = new ArrayList<String>();
+    int k = 0;
+    for (String sourceDocumentIDHash : sourceDocumentIDHashes)
+    {
+      if (k == maxClause)
+      {
+        performRevertLinks(jobID,list);
+        list.clear();
+        k = 0;
+      }
+      list.add(sourceDocumentIDHash);
+      k++;
+    }
+
+    if (k > 0)
+      performRevertLinks(jobID,list);
+    noteModifications(0,sourceDocumentIDHashes.length,0);
+  }
+
+  protected int maxClausesPerformRevertLinks(Long jobID)
+  {
+    return findConjunctionClauseMax(new ClauseDescription[]{
+      new UnitaryClause(jobIDField,jobID)});
+  }
+  
+  protected void performRevertLinks(Long jobID, List<String> list)
+    throws ManifoldCFException
+  {
+    // First, delete everything marked as "new"
+    StringBuilder sb = new StringBuilder("WHERE ");
+    ArrayList newList = new ArrayList();
+
+    sb.append(buildConjunctionClause(newList,new ClauseDescription[]{
+      new UnitaryClause(jobIDField,jobID),
+      new MultiClause(childIDHashField,list)})).append(" AND ")
+      .append(newField).append("=?");
+    newList.add(statusToString(LINKSTATUS_NEW));
+    performDelete(sb.toString(),newList,null);
+
+    // Now map everything marked as "EXISTING" back to "BASE".
+    HashMap map = new HashMap();
+    map.put(newField,statusToString(LINKSTATUS_BASE));
+    map.put(processIDField,null);
+    
+    sb = new StringBuilder();
+    newList.clear();
+    
+    sb.append(buildConjunctionClause(newList,new ClauseDescription[]{
+      new UnitaryClause(jobIDField,jobID),
+      new MultiClause(childIDHashField,list)})).append(" AND ")
+      .append(newField).append("=?");
+    newList.add(statusToString(LINKSTATUS_EXISTING));
+    performUpdate(map,sb.toString(),newList,null);
+  }
+
   /** Get document's children.
   *@return rows that contain the children.  Column names are 'linktype','childidentifier'.
   */
@@ -547,11 +602,10 @@ public class IntrinsicLink extends org.a
     IResultSet set = performQuery("SELECT DISTINCT "+parentIDHashField+" FROM "+
       getTableName()+" WHERE "+query,list,null,null);
     String[] rval = new String[set.getRowCount()];
-    int i = 0;
-    while (i < rval.length)
+    for (int i = 0; i < rval.length; i++)
     {
       IResultRow row = set.getRow(i);
-      rval[i++] = (String)row.getValue(parentIDHashField);
+      rval[i] = (String)row.getValue(parentIDHashField);
     }
     return rval;
   }

Modified: manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/JobManager.java
URL: http://svn.apache.org/viewvc/manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/JobManager.java?rev=1610676&r1=1610675&r2=1610676&view=diff
==============================================================================
--- manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/JobManager.java (original)
+++ manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/JobManager.java Tue Jul 15 12:58:51 2014
@@ -5188,6 +5188,85 @@ public class JobManager implements IJobM
       new Object[][][]{dataValues},currentTime,new IPriorityCalculator[]{priority},new String[][]{prereqEventNames});
   }
 
+  /** Undo the addition of child documents to the queue, for a set of documents.
+  * This method is called at the end of document processing, to back out any incomplete additions to the queue, and restore
+  * the status quo ante prior to the incomplete additions.  Call this method instead of finishDocuments() if the
+  * addition of documents was not completed.
+  *@param jobID is the job identifier.
+  *@param legalLinkTypes is the set of legal link types that this connector generates.
+  *@param parentIdentifierHashes are the hashes of the document identifiers for whom child link extraction just took place.
+  */
+  @Override
+  public void revertDocuments(Long jobID, String[] legalLinkTypes,
+    String[] parentIdentifierHashes)
+    throws ManifoldCFException
+  {
+    if (parentIdentifierHashes.length == 0)
+      return;
+    
+    if (legalLinkTypes.length == 0)
+    {
+      while (true)
+      {
+        long sleepAmt = 0L;
+        database.beginTransaction(database.TRANSACTION_SERIALIZED);
+        try
+        {
+          // Revert carrydown records
+          carryDown.revertRecords(jobID,parentIdentifierHashes);
+          database.performCommit();
+          break;
+        }
+        catch (Error e)
+        {
+          database.signalRollback();
+          throw e;
+        }
+        catch (RuntimeException e)
+        {
+          database.signalRollback();
+          throw e;
+        }
+        finally
+        {
+          database.endTransaction();
+          sleepFor(sleepAmt);
+        }
+      }
+    }
+    else
+    {
+      // Revert both hopcount and carrydown
+      while (true)
+      {
+        long sleepAmt = 0L;
+        database.beginTransaction(database.TRANSACTION_SERIALIZED);
+        try
+        {
+          carryDown.revertRecords(jobID,parentIdentifierHashes);
+          hopCount.revertParents(jobID,parentIdentifierHashes);
+          database.performCommit();
+          break;
+        }
+        catch (Error e)
+        {
+          database.signalRollback();
+          throw e;
+        }
+        catch (RuntimeException e)
+        {
+          database.signalRollback();
+          throw e;
+        }
+        finally
+        {
+          database.endTransaction();
+          sleepFor(sleepAmt);
+        }
+      }
+    }
+  }
+
   /** Complete adding child documents to the queue, for a set of documents.
   * This method is called at the end of document processing, to help the hopcount tracking engine do its bookkeeping.
   *@param jobID is the job identifier.
@@ -5240,6 +5319,11 @@ public class JobManager implements IJobM
           database.signalRollback();
           throw e;
         }
+        catch (RuntimeException e)
+        {
+          database.signalRollback();
+          throw e;
+        }
         finally
         {
           database.endTransaction();
@@ -5299,6 +5383,11 @@ public class JobManager implements IJobM
           database.signalRollback();
           throw e;
         }
+        catch (RuntimeException e)
+        {
+          database.signalRollback();
+          throw e;
+        }
         finally
         {
           database.endTransaction();