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/06/19 16:23:37 UTC

svn commit: r1603886 - in /manifoldcf/trunk: ./ framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/connectors/ framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/ framework/pull-agent/src/main/java/org/apache/mani...

Author: kwright
Date: Thu Jun 19 14:23:36 2014
New Revision: 1603886

URL: http://svn.apache.org/r1603886
Log:
Fix for CONNECTORS-971.  WARNING: Schema change!!

Modified:
    manifoldcf/trunk/   (props changed)
    manifoldcf/trunk/CHANGES.txt
    manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/connectors/BaseRepositoryConnector.java
    manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IJobManager.java
    manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IRepositoryConnector.java
    manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/JobSeedingRecord.java
    manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/JobStartRecord.java
    manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/JobManager.java
    manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/Jobs.java
    manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/SeedingThread.java
    manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/StartupThread.java

Propchange: manifoldcf/trunk/
------------------------------------------------------------------------------
  Merged /manifoldcf/branches/CONNECTORS-971:r1603835-1603885

Modified: manifoldcf/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/manifoldcf/trunk/CHANGES.txt?rev=1603886&r1=1603885&r2=1603886&view=diff
==============================================================================
--- manifoldcf/trunk/CHANGES.txt (original)
+++ manifoldcf/trunk/CHANGES.txt Thu Jun 19 14:23:36 2014
@@ -3,6 +3,11 @@ $Id$
 
 ======================= 1.7-dev =====================
 
+CONNECTORS-971: Use a generic "seeding version string" to track the last
+seeding event for every job.  This abstracts away from time intervals and
+permits seeding based on things like transaction IDs.
+(Karl Wright, Piergiorgio Lucidi)
+
 CONNECTORS-967: Add links to Java 1.7 and ManifoldCF framework Javadoc
 for ManifoldCF Javadocs.
 (Shinichiro Abe)

Modified: manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/connectors/BaseRepositoryConnector.java
URL: http://svn.apache.org/viewvc/manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/connectors/BaseRepositoryConnector.java?rev=1603886&r1=1603885&r2=1603886&view=diff
==============================================================================
--- manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/connectors/BaseRepositoryConnector.java (original)
+++ manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/connectors/BaseRepositoryConnector.java Thu Jun 19 14:23:36 2014
@@ -132,6 +132,50 @@ public abstract class BaseRepositoryConn
   * It is not a big problem if the connector chooses to create more seeds than are
   * strictly necessary; it is merely a question of overall work required.
   *
+  * The end time and seeding version string passed to this method may be interpreted for greatest efficiency.
+  * For continuous crawling jobs, this method will
+  * be called once, when the job starts, and at various periodic intervals as the job executes.
+  *
+  * When a job's specification is changed, the framework automatically resets the seeding version string to null.  The
+  * seeding version string may also be set to null on each job run, depending on the connector model returned by
+  * getConnectorModel().
+  *
+  * Note that it is always ok to send MORE documents rather than less to this method.
+  * The connector will be connected before this method can be called.
+  *@param activities is the interface this method should use to perform whatever framework actions are desired.
+  *@param spec is a document specification (that comes from the job).
+  *@param seedTime is the end of the time range of documents to consider, exclusive.
+  *@param lastSeedVersionString is the last seeding version string for this job, or null if the job has no previous seeding version string.
+  *@param jobMode is an integer describing how the job is being run, whether continuous or once-only.
+  *@return an updated seeding version string, to be stored with the job.
+  */
+  @Override
+  public String addSeedDocumentsWithVersion(ISeedingActivity activities, Specification spec,
+    String lastSeedVersion, long seedTime, int jobMode)
+    throws ManifoldCFException, ServiceInterruption
+  {
+    long startTime;
+    if (lastSeedVersion == null)
+      startTime = 0L;
+    else
+    {
+      // Unpack seed time from seed version string
+      startTime = new Long(lastSeedVersion).longValue();
+    }
+    addSeedDocuments(activities,spec,startTime,seedTime,jobMode);
+    return new Long(seedTime).toString();
+  }
+
+  /** Queue "seed" documents.  Seed documents are the starting places for crawling activity.  Documents
+  * are seeded when this method calls appropriate methods in the passed in ISeedingActivity object.
+  *
+  * This method can choose to find repository changes that happen only during the specified time interval.
+  * The seeds recorded by this method will be viewed by the framework based on what the
+  * getConnectorModel() method returns.
+  *
+  * It is not a big problem if the connector chooses to create more seeds than are
+  * strictly necessary; it is merely a question of overall work required.
+  *
   * The times passed to this method may be interpreted for greatest efficiency.  The time ranges
   * any given job uses with this connector will not overlap, but will proceed starting at 0 and going
   * to the "current time", each time the job is run.  For continuous crawling jobs, this method will
@@ -148,7 +192,6 @@ public abstract class BaseRepositoryConn
   *@param endTime is the end of the time range to consider, exclusive.
   *@param jobMode is an integer describing how the job is being run, whether continuous or once-only.
   */
-  @Override
   public void addSeedDocuments(ISeedingActivity activities, Specification spec,
     long startTime, long endTime, int jobMode)
     throws ManifoldCFException, ServiceInterruption

Modified: manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IJobManager.java
URL: http://svn.apache.org/viewvc/manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IJobManager.java?rev=1603886&r1=1603885&r2=1603886&view=diff
==============================================================================
--- manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IJobManager.java (original)
+++ manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IJobManager.java Thu Jun 19 14:23:36 2014
@@ -905,15 +905,16 @@ public interface IJobManager
   /** Note job started.
   *@param jobID is the job id.
   *@param startTime is the job start time.
+  *@param seedingVersion is the seeding version to record with the job start.
   */
-  public void noteJobStarted(Long jobID, long startTime)
+  public void noteJobStarted(Long jobID, long startTime, String seedingVersion)
     throws ManifoldCFException;
 
   /** Note job seeded.
   *@param jobID is the job id.
-  *@param startTime is the job seed time.
+  *@param seedingVersion is the seeding version string to record.
   */
-  public void noteJobSeeded(Long jobID, long startTime)
+  public void noteJobSeeded(Long jobID, String seedingVersion)
     throws ManifoldCFException;
 
   /**  Note the deregistration of a connector used by the specified connections.

Modified: manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IRepositoryConnector.java
URL: http://svn.apache.org/viewvc/manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IRepositoryConnector.java?rev=1603886&r1=1603885&r2=1603886&view=diff
==============================================================================
--- manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IRepositoryConnector.java (original)
+++ manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IRepositoryConnector.java Thu Jun 19 14:23:36 2014
@@ -161,25 +161,25 @@ public interface IRepositoryConnector ex
   * It is not a big problem if the connector chooses to create more seeds than are
   * strictly necessary; it is merely a question of overall work required.
   *
-  * The times passed to this method may be interpreted for greatest efficiency.  The time ranges
-  * any given job uses with this connector will not overlap, but will proceed starting at 0 and going
-  * to the "current time", each time the job is run.  For continuous crawling jobs, this method will
+  * The end time and seeding version string passed to this method may be interpreted for greatest efficiency.
+  * For continuous crawling jobs, this method will
   * be called once, when the job starts, and at various periodic intervals as the job executes.
   *
-  * When a job's specification is changed, the framework automatically resets the seeding start time to 0.  The
-  * seeding start time may also be set to 0 on each job run, depending on the connector model returned by
+  * When a job's specification is changed, the framework automatically resets the seeding version string to null.  The
+  * seeding version string may also be set to null on each job run, depending on the connector model returned by
   * getConnectorModel().
   *
   * Note that it is always ok to send MORE documents rather than less to this method.
   * The connector will be connected before this method can be called.
   *@param activities is the interface this method should use to perform whatever framework actions are desired.
   *@param spec is a document specification (that comes from the job).
-  *@param startTime is the beginning of the time range to consider, inclusive.
-  *@param endTime is the end of the time range to consider, exclusive.
+  *@param seedTime is the end of the time range of documents to consider, exclusive.
+  *@param lastSeedVersionString is the last seeding version string for this job, or null if the job has no previous seeding version string.
   *@param jobMode is an integer describing how the job is being run, whether continuous or once-only.
+  *@return an updated seeding version string, to be stored with the job.
   */
-  public void addSeedDocuments(ISeedingActivity activities, Specification spec,
-    long startTime, long endTime, int jobMode)
+  public String addSeedDocumentsWithVersion(ISeedingActivity activities, Specification spec,
+    String lastSeedVersion, long seedTime, int jobMode)
     throws ManifoldCFException, ServiceInterruption;
 
   /** Get document versions given an array of document identifiers.

Modified: manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/JobSeedingRecord.java
URL: http://svn.apache.org/viewvc/manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/JobSeedingRecord.java?rev=1603886&r1=1603885&r2=1603886&view=diff
==============================================================================
--- manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/JobSeedingRecord.java (original)
+++ manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/JobSeedingRecord.java Thu Jun 19 14:23:36 2014
@@ -25,8 +25,8 @@ public class JobSeedingRecord extends Jo
 {
   public static final String _rcsid = "@(#)$Id$";
 
-  /** The last synch time */
-  protected final long synchTime;
+  /** The last seeding version */
+  protected final String seedingVersionString;
   /** The fail time, or -1L if none */
   protected final long failTime;
   /** The fail count, or -1 if none */
@@ -34,20 +34,20 @@ public class JobSeedingRecord extends Jo
 
   /** Constructor.
   */
-  public JobSeedingRecord(Long jobID, long synchTime, long failTime, int failRetryCount)
+  public JobSeedingRecord(Long jobID, String seedingVersionString, long failTime, int failRetryCount)
   {
     super(jobID);
-    this.synchTime = synchTime;
+    this.seedingVersionString = seedingVersionString;
     this.failTime = failTime;
     this.failRetryCount = failRetryCount;
   }
 
-  /** Get the synch time.
-  *@return the time.
+  /** Get the seeding version string.
+  *@return the string.
   */
-  public long getSynchTime()
+  public String getSeedingVersionString()
   {
-    return synchTime;
+    return seedingVersionString;
   }
 
   /** Get the hard fail time.

Modified: manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/JobStartRecord.java
URL: http://svn.apache.org/viewvc/manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/JobStartRecord.java?rev=1603886&r1=1603885&r2=1603886&view=diff
==============================================================================
--- manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/JobStartRecord.java (original)
+++ manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/JobStartRecord.java Thu Jun 19 14:23:36 2014
@@ -25,8 +25,8 @@ public class JobStartRecord extends JobR
 {
   public static final String _rcsid = "@(#)$Id: JobStartRecord.java 988245 2010-08-23 18:39:35Z kwright $";
 
-  /** The last synch time */
-  protected final long synchTime;
+  /** The last seeding version */
+  protected final String seedingVersionString;
   /** The requestMinimum flag */
   protected final boolean requestMinimum;
   /** The fail time, or -1L if none */
@@ -36,21 +36,21 @@ public class JobStartRecord extends JobR
 
   /** Constructor.
   */
-  public JobStartRecord(Long jobID, long synchTime, boolean requestMinimum, long failTime, int failRetryCount)
+  public JobStartRecord(Long jobID, String seedingVersionString, boolean requestMinimum, long failTime, int failRetryCount)
   {
     super(jobID);
-    this.synchTime = synchTime;
+    this.seedingVersionString = seedingVersionString;
     this.requestMinimum = requestMinimum;
     this.failTime = failTime;
     this.failRetryCount = failRetryCount;
   }
 
-  /** Get the synch time.
-  *@return the time.
+  /** Get the seeding version string.
+  *@return the string.
   */
-  public long getSynchTime()
+  public String getSeedingVersionString()
   {
-    return synchTime;
+    return seedingVersionString;
   }
 
   /** Get the requestMinimum flag.

Modified: manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/JobManager.java
URL: http://svn.apache.org/viewvc/manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/JobManager.java?rev=1603886&r1=1603885&r2=1603886&view=diff
==============================================================================
--- manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/JobManager.java (original)
+++ manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/JobManager.java Thu Jun 19 14:23:36 2014
@@ -6245,23 +6245,23 @@ public class JobManager implements IJobM
   *@param startTime is the job start time.
   */
   @Override
-  public void noteJobStarted(Long jobID, long startTime)
+  public void noteJobStarted(Long jobID, long startTime, String seedingVersion)
     throws ManifoldCFException
   {
-    jobs.noteJobStarted(jobID,startTime);
+    jobs.noteJobStarted(jobID,startTime,seedingVersion);
     if (Logging.jobs.isDebugEnabled())
       Logging.jobs.debug("Job "+jobID+" is now started");
   }
 
   /** Note job seeded.
   *@param jobID is the job id.
-  *@param seedTime is the job seed time.
+  *@param seedingVersion is the job seeding version string to record.
   */
   @Override
-  public void noteJobSeeded(Long jobID, long seedTime)
+  public void noteJobSeeded(Long jobID, String seedingVersion)
     throws ManifoldCFException
   {
-    jobs.noteJobSeeded(jobID,seedTime);
+    jobs.noteJobSeeded(jobID,seedingVersion);
     if (Logging.jobs.isDebugEnabled())
       Logging.jobs.debug("Job "+jobID+" has been successfully reseeded");
   }
@@ -6780,7 +6780,7 @@ public class JobManager implements IJobM
         ArrayList list = new ArrayList();
         
         sb.append(jobs.idField).append(",")
-          .append(jobs.lastCheckTimeField).append(",")
+          .append(jobs.seedingVersionField).append(",")
           .append(jobs.failTimeField).append(",")
           .append(jobs.failCountField).append(",")
           .append(jobs.reseedIntervalField)
@@ -6802,10 +6802,7 @@ public class JobManager implements IJobM
         {
           IResultRow row = set.getRow(i);
           Long jobID = (Long)row.getValue(jobs.idField);
-          Long x = (Long)row.getValue(jobs.lastCheckTimeField);
-          long synchTime = 0;
-          if (x != null)
-            synchTime = x.longValue();
+          String seedingVersionString = (String)row.getValue(jobs.seedingVersionField);
 
           Long r = (Long)row.getValue(jobs.reseedIntervalField);
           Long reseedTime;
@@ -6834,8 +6831,7 @@ public class JobManager implements IJobM
           {
             Logging.jobs.debug("Marked job "+jobID+" for seeding");
           }
-
-          rval[i] = new JobSeedingRecord(jobID,synchTime,failTime,failRetryCount);
+          rval[i] = new JobSeedingRecord(jobID,seedingVersionString,failTime,failRetryCount);
           i++;
         }
         database.performCommit();
@@ -6957,7 +6953,7 @@ public class JobManager implements IJobM
         sb.append(jobs.idField).append(",")
           .append(jobs.failTimeField).append(",")
           .append(jobs.failCountField).append(",")
-          .append(jobs.lastCheckTimeField).append(",")
+          .append(jobs.seedingVersionField).append(",")
           .append(jobs.statusField)
           .append(" FROM ").append(jobs.getTableName()).append(" WHERE ")
           .append(database.buildConjunctionClause(list,new ClauseDescription[]{
@@ -6974,7 +6970,7 @@ public class JobManager implements IJobM
         {
           IResultRow row = set.getRow(i);
           Long jobID = (Long)row.getValue(jobs.idField);
-          Long x = (Long)row.getValue(jobs.lastCheckTimeField);
+          String seedingVersionString = (String)row.getValue(jobs.seedingVersionField);
           int status = jobs.stringToStatus((String)row.getValue(jobs.statusField));
           Long failTimeLong = (Long)row.getValue(jobs.failTimeField);
           Long failRetryCountLong = (Long)row.getValue(jobs.failCountField);
@@ -6991,10 +6987,6 @@ public class JobManager implements IJobM
 
           boolean requestMinimum = (status == jobs.STATUS_READYFORSTARTUPMINIMAL);
           
-          long synchTime = 0;
-          if (x != null)
-            synchTime = x.longValue();
-
           // Mark status of job as "starting"
           jobs.writeTransientStatus(jobID,requestMinimum?jobs.STATUS_STARTINGUPMINIMAL:jobs.STATUS_STARTINGUP,processID);
           if (Logging.jobs.isDebugEnabled())
@@ -7002,7 +6994,7 @@ public class JobManager implements IJobM
             Logging.jobs.debug("Marked job "+jobID+" for startup");
           }
 
-          rval[i] = new JobStartRecord(jobID,synchTime,requestMinimum,failTime,failRetryCount);
+          rval[i] = new JobStartRecord(jobID,seedingVersionString,requestMinimum,failTime,failRetryCount);
           i++;
         }
         database.performCommit();

Modified: manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/Jobs.java
URL: http://svn.apache.org/viewvc/manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/Jobs.java?rev=1603886&r1=1603885&r2=1603886&view=diff
==============================================================================
--- manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/Jobs.java (original)
+++ manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/jobs/Jobs.java Thu Jun 19 14:23:36 2014
@@ -38,6 +38,7 @@ import java.util.*;
  * <tr><td>lasttime</td><td>BIGINT</td><td>operational field</td></tr>
  * <tr><td>starttime</td><td>BIGINT</td><td>operational field</td></tr>
  * <tr><td>lastchecktime</td><td>BIGINT</td><td>operational field</td></tr>
+ * <tr><td>seedingversion</td><td>LONGTEXT</td><td>operational field</td></tr>
  * <tr><td>endtime</td><td>BIGINT</td><td>operational field</td></tr>
  * <tr><td>docspec</td><td>LONGTEXT</td><td></td></tr>
  * <tr><td>connectionname</td><td>VARCHAR(32)</td><td>Reference:repoconnections.connectionname</td></tr>
@@ -175,9 +176,8 @@ public class Jobs extends org.apache.man
   public final static String lastTimeField = "lasttime";
   /** If active, paused, activewait, or pausedwait, the start time of the current session, else null. */
   public final static String startTimeField = "starttime";
-  /** The time of the LAST session, if any.  This is the place where the "last repository change check time"
-  * is gotten from. */
-  public final static String lastCheckTimeField = "lastchecktime";
+  /** This text data represents the seeding version string, which for many connectors is simply the last time seeding was done */
+  public final static String seedingVersionField = "seedingversion";
   /** If inactive, the end time of the LAST session, if any. */
   public final static String endTimeField = "endtime";
   /** If non-null, this is the time that the current execution window closes, in ms since epoch. */
@@ -377,6 +377,7 @@ public class Jobs extends org.apache.man
       // These are fields we want to get rid of.
       String oldOutputSpecField = "outputspec";
       String oldOutputNameField = "outputname";
+      String oldLastCheckTimeField = "lastchecktime";
 
       // A place to keep the outputs we find, so we can add them into the pipeline at the end.
       IResultSet outputSet = null;
@@ -390,7 +391,7 @@ public class Jobs extends org.apache.man
         map.put(statusField,new ColumnDescription("CHAR(1)",false,false,null,null,false));
         map.put(lastTimeField,new ColumnDescription("BIGINT",false,false,null,null,false));
         map.put(startTimeField,new ColumnDescription("BIGINT",false,true,null,null,false));
-        map.put(lastCheckTimeField,new ColumnDescription("BIGINT",false,true,null,null,false));
+        map.put(seedingVersionField,new ColumnDescription("LONGTEXT",false,true,null,null,false));
         map.put(endTimeField,new ColumnDescription("BIGINT",false,true,null,null,false));
         map.put(documentSpecField,new ColumnDescription("LONGTEXT",false,true,null,null,false));
         map.put(this.connectionNameField,new ColumnDescription("VARCHAR(32)",false,false,connectionTableName,connectionNameField,false));
@@ -461,6 +462,33 @@ public class Jobs extends org.apache.man
           map.put(statusField,statusToString(STATUS_ACTIVESEEDING_UNINSTALLED));
           performUpdate(map,"WHERE "+query,list,null);
         }
+        if (existing.get(seedingVersionField) == null)
+        {
+          Map insertMap = new HashMap();
+          insertMap.put(seedingVersionField,new ColumnDescription("LONGTEXT",false,true,null,null,false));
+          performAlter(insertMap,null,null,null);
+          // Populate it with data from the old last check version field
+          IResultSet set = performQuery("SELECT "+idField+","+oldLastCheckTimeField+" FROM "+getTableName(),null,null,null);
+          for (int i = 0; i < set.getRowCount(); i++)
+          {
+            IResultRow row = set.getRow(i);
+            Long jobID = (Long)row.getValue(idField);
+            Long oldTime = (Long)row.getValue(oldLastCheckTimeField);
+            if (oldTime != null)
+            {
+              HashMap map = new HashMap();
+              map.put(seedingVersionField,oldTime.toString());
+              ArrayList list = new ArrayList();
+              String query = buildConjunctionClause(list,new ClauseDescription[]{
+                new UnitaryClause(idField,jobID)});
+              performUpdate(map,"WHERE "+query,list,null);
+            }
+          }
+          List<String> deleteList = new ArrayList<String>();
+          deleteList.add(oldLastCheckTimeField);
+          performAlter(null,null,deleteList,null);
+        }
+        
         if (existing.get(oldOutputNameField) != null)
         {
           // Remove output name and spec fields, but first read them so we can put them into the pipeline manager
@@ -471,6 +499,8 @@ public class Jobs extends org.apache.man
           deleteList.add(oldOutputNameField);
           performAlter(null,null,deleteList,null);
         }
+        // Need upgrade for seedingversionfield and to get rid of lastcheckfield
+        // MHL
       }
 
       // Handle related tables
@@ -937,7 +967,7 @@ public class Jobs extends org.apache.man
             if (set.getRowCount() > 0)
             {
               // Update
-              // We need to reset the lastCheckTimeField if there are any changes that
+              // We need to reset the seedingVersionField if there are any changes that
               // could affect what set of documents we allow!!!
 
               IResultRow row = set.getRow(0);
@@ -969,7 +999,7 @@ public class Jobs extends org.apache.man
                 isSame = forcedParamManager.compareRows(id,jobDescription);
 
               if (!isSame)
-                values.put(lastCheckTimeField,null);
+                values.put(seedingVersionField,null);
 
               params.clear();
               query = buildConjunctionClause(params,new ClauseDescription[]{
@@ -984,7 +1014,7 @@ public class Jobs extends org.apache.man
             {
               // Insert
               values.put(startTimeField,null);
-              values.put(lastCheckTimeField,null);
+              values.put(seedingVersionField,null);
               values.put(endTimeField,null);
               values.put(statusField,statusToString(STATUS_INACTIVE));
               values.put(lastTimeField,new Long(System.currentTimeMillis()));
@@ -1045,7 +1075,7 @@ public class Jobs extends org.apache.man
     throws ManifoldCFException
   {
     Map values = new HashMap();
-    values.put(lastCheckTimeField,null);
+    values.put(seedingVersionField,null);
     ArrayList params = new ArrayList();
     String query = buildConjunctionClause(params,new ClauseDescription[]{
       new UnitaryClause(idField,jobID)});
@@ -1539,7 +1569,7 @@ public class Jobs extends org.apache.man
   {
     // No cache keys need invalidation, since we're changing the start time, not the status.
     HashMap newValues = new HashMap();
-    newValues.put(lastCheckTimeField,null);
+    newValues.put(seedingVersionField,null);
     ArrayList list = new ArrayList();
     String query = buildConjunctionClause(list,new ClauseDescription[]{
       new UnitaryClause(connectionNameField,connectionName)});
@@ -1555,7 +1585,7 @@ public class Jobs extends org.apache.man
   {
     // No cache keys need invalidation, since we're changing the start time, not the status.
     HashMap newValues = new HashMap();
-    newValues.put(lastCheckTimeField,null);
+    newValues.put(seedingVersionField,null);
     ArrayList list = new ArrayList();
     String query = buildConjunctionClause(list,new ClauseDescription[]{
       new JoinClause(getTableName()+"."+idField,pipelineManager.ownerIDField),
@@ -1571,7 +1601,7 @@ public class Jobs extends org.apache.man
   {
     // No cache keys need invalidation, since we're changing the start time, not the status.
     HashMap newValues = new HashMap();
-    newValues.put(lastCheckTimeField,null);
+    newValues.put(seedingVersionField,null);
     ArrayList list = new ArrayList();
     String query = buildConjunctionClause(list,new ClauseDescription[]{
       new JoinClause(getTableName()+"."+idField,pipelineManager.ownerIDField),
@@ -2129,7 +2159,8 @@ public class Jobs extends org.apache.man
       {
         map.put(startTimeField,new Long(startTime));
       }
-      map.put(lastCheckTimeField,new Long(startTime));
+      // Clear out seeding version, in case we wind up keeping the job and rerunning it
+      map.put(seedingVersionField,null);
       performUpdate(map,"WHERE "+query,list,new StringSet(getJobStatusKey()));
     }
     catch (ManifoldCFException e)
@@ -2156,8 +2187,9 @@ public class Jobs extends org.apache.man
   /** Make job active, and set the start time field.
   *@param jobID is the job identifier.
   *@param startTime is the current time in milliseconds from start of epoch.
+  *@param seedVersionString is the version string to record for the seeding.
   */
-  public void noteJobStarted(Long jobID, long startTime)
+  public void noteJobStarted(Long jobID, long startTime, String seedVersionString)
     throws ManifoldCFException
   {
     beginTransaction();
@@ -2208,7 +2240,7 @@ public class Jobs extends org.apache.man
         map.put(startTimeField,new Long(startTime));
       }
       // The seeding was complete or we wouldn't have gotten called, so at least note that.
-      map.put(lastCheckTimeField,new Long(startTime));
+      map.put(seedingVersionField,seedVersionString);
       // Clear out the retry fields we might have set
       map.put(failTimeField,null);
       map.put(failCountField,null);
@@ -2238,9 +2270,9 @@ public class Jobs extends org.apache.man
 
   /** Note job seeded.
   *@param jobID is the job id.
-  *@param seedTime is the job seed time.
+  *@param seedVersionString is the job seed version string.
   */
-  public void noteJobSeeded(Long jobID, long seedTime)
+  public void noteJobSeeded(Long jobID, String seedVersionString)
     throws ManifoldCFException
   {
     // We have to convert the current status to the non-seeding equivalent
@@ -2295,7 +2327,7 @@ public class Jobs extends org.apache.man
       HashMap map = new HashMap();
       map.put(statusField,statusToString(newStatus));
       map.put(processIDField,null);
-      map.put(lastCheckTimeField,new Long(seedTime));
+      map.put(seedingVersionField,seedVersionString);
       map.put(failTimeField,null);
       map.put(failCountField,null);
       performUpdate(map,"WHERE "+query,list,new StringSet(getJobStatusKey()));

Modified: manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/SeedingThread.java
URL: http://svn.apache.org/viewvc/manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/SeedingThread.java?rev=1603886&r1=1603885&r2=1603886&view=diff
==============================================================================
--- manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/SeedingThread.java (original)
+++ manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/SeedingThread.java Thu Jun 19 14:23:36 2014
@@ -116,7 +116,7 @@ public class SeedingThread extends Threa
               Long jobID = jsr.getJobID();
               try
               {
-                long lastJobTime = jsr.getSynchTime();
+                String lastSeedingVersion = jsr.getSeedingVersionString();
                 IJobDescription jobDescription = jobManager.load(jobID,true);
                 int jobType = jobDescription.getType();
 
@@ -127,8 +127,11 @@ public class SeedingThread extends Threa
                 // Null will come back if the connector instance could not be obtained, so just skip in that case.
                 if (connector == null)
                   continue;
+
+                String newSeedingVersion = null;
                 try
                 {
+                  
                   // Get the number of link types.
                   String[] legalLinkTypes = connector.getRelationshipTypes();
 
@@ -144,7 +147,7 @@ public class SeedingThread extends Threa
                     if (Logging.threads.isDebugEnabled())
                       Logging.threads.debug("Seeding thread: Getting seeds for job "+jobID.toString());
 
-                    connector.addSeedDocuments(activity,jobDescription.getSpecification(),lastJobTime,currentTime,jobType);
+                    newSeedingVersion = connector.addSeedDocumentsWithVersion(activity,jobDescription.getSpecification(),lastSeedingVersion,currentTime,jobType);
 
                     activity.doneSeeding(model==connector.MODEL_PARTIAL);
 
@@ -180,7 +183,7 @@ public class SeedingThread extends Threa
                       else
                       {
                         // Not sure this can happen -- but just transition silently to active state
-                        jobManager.noteJobSeeded(jobID,currentTime);
+                        jobManager.noteJobSeeded(jobID,newSeedingVersion);
                         jsr.noteStarted();
                       }
                     }
@@ -204,7 +207,7 @@ public class SeedingThread extends Threa
                   Logging.threads.debug("Seeding thread: Successfully reseeded job "+jobID.toString());
 
                 // Note that this job has been seeded!
-                jobManager.noteJobSeeded(jobID,currentTime);
+                jobManager.noteJobSeeded(jobID,newSeedingVersion);
                 jsr.noteStarted();
 
               }

Modified: manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/StartupThread.java
URL: http://svn.apache.org/viewvc/manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/StartupThread.java?rev=1603886&r1=1603885&r2=1603886&view=diff
==============================================================================
--- manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/StartupThread.java (original)
+++ manifoldcf/trunk/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/StartupThread.java Thu Jun 19 14:23:36 2014
@@ -100,14 +100,13 @@ public class StartupThread extends Threa
 
 
             // Loop through jobs
-            int i = 0;
-            while (i < startupJobs.length)
+            for (int i = 0; i < startupJobs.length; i++)
             {
-              JobStartRecord jsr = startupJobs[i++];
+              JobStartRecord jsr = startupJobs[i];
               Long jobID = jsr.getJobID();
               try
               {
-                long lastJobTime = jsr.getSynchTime();
+                String lastSeedingVersion = jsr.getSeedingVersionString();
                 IJobDescription jobDescription = jobManager.load(jobID,true);
 
                 int jobType = jobDescription.getType();
@@ -120,6 +119,7 @@ public class StartupThread extends Threa
                 if (connector == null)
                   continue;
 
+                String newSeedingVersion = null;
                 try
                 {
                   // Only now record the fact that we are trying to start the job.
@@ -136,7 +136,7 @@ public class StartupThread extends Threa
                   if (Logging.threads.isDebugEnabled())
                     Logging.threads.debug("Preparing job "+jobID.toString()+" for execution...");
                   jobManager.prepareJobScan(jobID,legalLinkTypes,hopcountMethod,
-                    model,jobType == IJobDescription.TYPE_CONTINUOUS,lastJobTime == 0L,
+                    model,jobType == IJobDescription.TYPE_CONTINUOUS,lastSeedingVersion == null,
                     requestMinimum);
                   if (Logging.threads.isDebugEnabled())
                     Logging.threads.debug("Prepared job "+jobID.toString()+" for execution.");
@@ -150,7 +150,7 @@ public class StartupThread extends Threa
                     if (Logging.threads.isDebugEnabled())
                       Logging.threads.debug("Adding initial seed documents for job "+jobID.toString()+"...");
                     // Get the initial seed documents, and make sure those are added
-                    connector.addSeedDocuments(activity,jobDescription.getSpecification(),lastJobTime,currentTime,jobType);
+                    newSeedingVersion = connector.addSeedDocumentsWithVersion(activity,jobDescription.getSpecification(),lastSeedingVersion,currentTime,jobType);
                     // Flush anything left
                     activity.doneSeeding(model==connector.MODEL_PARTIAL);
                     if (Logging.threads.isDebugEnabled())
@@ -183,7 +183,7 @@ public class StartupThread extends Threa
                       else
                       {
                         // Not sure this can happen -- but just transition silently to active state
-                        jobManager.noteJobStarted(jobID,currentTime);
+                        jobManager.noteJobStarted(jobID,currentTime,newSeedingVersion);
                         jsr.noteStarted();
                       }
                     }
@@ -203,7 +203,7 @@ public class StartupThread extends Threa
                 }
 
                 // Start this job!
-                jobManager.noteJobStarted(jobID,currentTime);
+                jobManager.noteJobStarted(jobID,currentTime,newSeedingVersion);
                 jsr.noteStarted();
               }
               catch (ManifoldCFException e)