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/14 16:10:37 UTC

svn commit: r1610414 - in /manifoldcf/branches/CONNECTORS-990/framework: agents/src/main/java/org/apache/manifoldcf/agents/incrementalingest/ agents/src/main/java/org/apache/manifoldcf/agents/interfaces/ pull-agent/src/main/java/org/apache/manifoldcf/c...

Author: kwright
Date: Mon Jul 14 14:10:36 2014
New Revision: 1610414

URL: http://svn.apache.org/r1610414
Log:
Revamp API to meet CONNECTORS-990 goal.  Doesn't compile yet

Added:
    manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IExistingVersions.java   (with props)
Modified:
    manifoldcf/branches/CONNECTORS-990/framework/agents/src/main/java/org/apache/manifoldcf/agents/incrementalingest/IncrementalIngester.java
    manifoldcf/branches/CONNECTORS-990/framework/agents/src/main/java/org/apache/manifoldcf/agents/interfaces/IIncrementalIngester.java
    manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/connectors/BaseRepositoryConnector.java
    manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IProcessActivity.java
    manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IRepositoryConnector.java
    manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/WorkerThread.java

Modified: manifoldcf/branches/CONNECTORS-990/framework/agents/src/main/java/org/apache/manifoldcf/agents/incrementalingest/IncrementalIngester.java
URL: http://svn.apache.org/viewvc/manifoldcf/branches/CONNECTORS-990/framework/agents/src/main/java/org/apache/manifoldcf/agents/incrementalingest/IncrementalIngester.java?rev=1610414&r1=1610413&r2=1610414&view=diff
==============================================================================
--- manifoldcf/branches/CONNECTORS-990/framework/agents/src/main/java/org/apache/manifoldcf/agents/incrementalingest/IncrementalIngester.java (original)
+++ manifoldcf/branches/CONNECTORS-990/framework/agents/src/main/java/org/apache/manifoldcf/agents/incrementalingest/IncrementalIngester.java Mon Jul 14 14:10:36 2014
@@ -609,27 +609,35 @@ public class IncrementalIngester extends
   }
 
   /** Record a document version, but don't ingest it.
-  * The purpose of this method is to keep track of the frequency at which ingestion "attempts" take place.
-  * ServiceInterruption is thrown if this action must be rescheduled.
+  * The purpose of this method is to update document version information without reindexing the document.
   *@param pipelineSpecificationBasic is the basic pipeline specification needed.
   *@param identifierClass is the name of the space in which the identifier hash should be interpreted.
   *@param identifierHash is the hashed document identifier.
   *@param documentVersion is the document version.
   *@param recordTime is the time at which the recording took place, in milliseconds since epoch.
-  *@param activities is the object used in case a document needs to be removed from the output index as the result of this operation.
   */
   @Override
   public void documentRecord(
     IPipelineSpecificationBasic pipelineSpecificationBasic,
     String identifierClass, String identifierHash,
-    String documentVersion, long recordTime,
-    IOutputActivity activities)
-    throws ManifoldCFException, ServiceInterruption
+    String documentVersion, long recordTime)
+    throws ManifoldCFException
   {
+    // This method is called when a connector decides that the last indexed version of the document is in fact just fine,
+    // but the document version information should be updated.
+    // The code pathway is therefore similar to that of document indexing, EXCEPT that no indexing will ever
+    // take place.  This has some interesting side effects.  For example:
+    // (1) In the case of a document collision with another job using the same repository connection, the last document
+    //    indexed cannot be changed.  Updating the version string for the document would therefore be misleading.  This
+    //    case should be detected and prevented from occurring, by refusing to perform the update.
+    //    On the other hand, only one thread at a time can be processing the document at a given time, and therefore
+    //    since the connector detected "no change", we are safe to presume we can just update the version info.
+    // (2) In the case of a URL conflict with another job, since nothing changes and no new URL is recorded, no cleanup
+    //    of conflicting records sharing the same URL should be needed.
+    
     String docKey = makeKey(identifierClass,identifierHash);
 
     String[] outputConnectionNames = extractOutputConnectionNames(pipelineSpecificationBasic);
-    IOutputConnection[] outputConnections = connectionManager.loadMultiple(outputConnectionNames);
 
     if (Logging.ingest.isDebugEnabled())
     {
@@ -639,99 +647,10 @@ public class IncrementalIngester extends
     for (int k = 0; k < outputConnectionNames.length; k++)
     {
       String outputConnectionName = outputConnectionNames[k];
-      IOutputConnection connection = outputConnections[k];
 
-      String oldURI = null;
-      String oldURIHash = null;
-      String oldOutputVersion = null;
-
-      // Repeat if needed
-      while (true)
-      {
-        long sleepAmt = 0L;
-        try
-        {
-          // See what uri was used before for this doc, if any
-          ArrayList list = new ArrayList();
-          String query = buildConjunctionClause(list,new ClauseDescription[]{
-            new UnitaryClause(docKeyField,docKey),
-            new UnitaryClause(outputConnNameField,outputConnectionName)});
-            
-          IResultSet set = performQuery("SELECT "+docURIField+","+uriHashField+","+lastOutputVersionField+" FROM "+getTableName()+
-            " WHERE "+query,list,null,null);
-
-          if (set.getRowCount() > 0)
-          {
-            IResultRow row = set.getRow(0);
-            oldURI = (String)row.getValue(docURIField);
-            oldURIHash = (String)row.getValue(uriHashField);
-            oldOutputVersion = (String)row.getValue(lastOutputVersionField);
-          }
-          
-          break;
-        }
-        catch (ManifoldCFException e)
-        {
-          // Look for deadlock and retry if so
-          if (e.getErrorCode() == e.DATABASE_TRANSACTION_ABORT)
-          {
-            if (Logging.perf.isDebugEnabled())
-              Logging.perf.debug("Aborted select looking for status: "+e.getMessage());
-            sleepAmt = getSleepAmt();
-            continue;
-          }
-          throw e;
-        }
-        finally
-        {
-          sleepFor(sleepAmt);
-        }
-      }
-
-      // If uri hashes collide, then we must be sure to eliminate only the *correct* records from the table, or we will leave
-      // dangling documents around.  So, all uri searches and comparisons MUST compare the actual uri as well.
-
-      // But, since we need to insure that any given URI is only worked on by one thread at a time, use critical sections
-      // to block the rare case that multiple threads try to work on the same URI.
-      
-      String[] lockArray = computeLockArray(null,oldURI,outputConnectionName);
-      lockManager.enterLocks(null,null,lockArray);
-      try
-      {
-
-        ArrayList list = new ArrayList();
-        
-        if (oldURI != null)
-        {
-          IOutputConnector connector = outputConnectorPool.grab(connection);
-          if (connector == null)
-            // The connector is not installed; treat this as a service interruption.
-            throw new ServiceInterruption("Output connector not installed",0L);
-          try
-          {
-            connector.removeDocument(oldURI,oldOutputVersion,new OutputRemoveActivitiesWrapper(activities,outputConnectionName));
-          }
-          finally
-          {
-            outputConnectorPool.release(connection,connector);
-          }
-          // Delete all records from the database that match the old URI, except for THIS record.
-          list.clear();
-          String query = buildConjunctionClause(list,new ClauseDescription[]{
-            new UnitaryClause(uriHashField,"=",oldURIHash),
-            new UnitaryClause(outputConnNameField,outputConnectionName)});
-          list.add(docKey);
-          performDelete("WHERE "+query+" AND "+docKeyField+"!=?",list,null);
-        }
-
-        // If we get here, it means we are noting that the document was examined, but that no change was required.  This is signaled
-        // to noteDocumentIngest by having the null documentURI.
-        noteDocumentIngest(outputConnectionName,docKey,documentVersion,null,null,null,null,recordTime,null,null);
-      }
-      finally
-      {
-        lockManager.leaveLocks(null,null,lockArray);
-      }
+      // If we get here, it means we are noting that the document was examined, but that no change was required.  This is signaled
+      // to noteDocumentIngest by having the null documentURI.
+      noteDocumentIngest(outputConnectionName,docKey,documentVersion,null,null,null,null,recordTime,null,null);
     }
   }
 

Modified: manifoldcf/branches/CONNECTORS-990/framework/agents/src/main/java/org/apache/manifoldcf/agents/interfaces/IIncrementalIngester.java
URL: http://svn.apache.org/viewvc/manifoldcf/branches/CONNECTORS-990/framework/agents/src/main/java/org/apache/manifoldcf/agents/interfaces/IIncrementalIngester.java?rev=1610414&r1=1610413&r2=1610414&view=diff
==============================================================================
--- manifoldcf/branches/CONNECTORS-990/framework/agents/src/main/java/org/apache/manifoldcf/agents/interfaces/IIncrementalIngester.java (original)
+++ manifoldcf/branches/CONNECTORS-990/framework/agents/src/main/java/org/apache/manifoldcf/agents/interfaces/IIncrementalIngester.java Mon Jul 14 14:10:36 2014
@@ -150,21 +150,18 @@ public interface IIncrementalIngester
     String newAuthorityNameString);
 
   /** Record a document version, but don't ingest it.
-  * The purpose of this method is to keep track of the frequency at which ingestion "attempts" take place.
-  * ServiceInterruption is thrown if this action must be rescheduled.
+  * The purpose of this method is to update document version information without reindexing the document.
   *@param pipelineSpecificationBasic is the basic pipeline specification needed.
   *@param identifierClass is the name of the space in which the identifier hash should be interpreted.
   *@param identifierHash is the hashed document identifier.
   *@param documentVersion is the document version.
   *@param recordTime is the time at which the recording took place, in milliseconds since epoch.
-  *@param activities is the object used in case a document needs to be removed from the output index as the result of this operation.
   */
   public void documentRecord(
     IPipelineSpecificationBasic pipelineSpecificationBasic,
     String identifierClass, String identifierHash,
-    String documentVersion, long recordTime,
-    IOutputActivity activities)
-    throws ManifoldCFException, ServiceInterruption;
+    String documentVersion, long recordTime)
+    throws ManifoldCFException;
 
   /** Ingest a document.
   * This ingests the document, and notes it.  If this is a repeat ingestion of the document, this

Modified: manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/connectors/BaseRepositoryConnector.java
URL: http://svn.apache.org/viewvc/manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/connectors/BaseRepositoryConnector.java?rev=1610414&r1=1610413&r2=1610414&view=diff
==============================================================================
--- manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/connectors/BaseRepositoryConnector.java (original)
+++ manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/connectors/BaseRepositoryConnector.java Mon Jul 14 14:10:36 2014
@@ -324,6 +324,78 @@ public abstract class BaseRepositoryConn
     return null;
   }
 
+  /** Process a set of documents.
+  * This is the method that should cause each document to be fetched, processed, and the results either added
+  * to the queue of documents for the current job, and/or entered into the incremental ingestion manager.
+  * The document specification allows this class to filter what is done based on the job.
+  * The connector will be connected before this method can be called.
+  *@param documentIdentifiers is the set of document identifiers to process.
+  *@param statuses are the currently-stored document versions for each document in the set of document identifiers
+  * passed in above.
+  *@param activities is the interface this method should use to queue up new document references
+  * and ingest documents.
+  *@param jobMode is an integer describing how the job is being run, whether continuous or once-only.
+  *@param usesDefaultAuthority will be true only if the authority in use for these documents is the default one.
+  */
+  @Override
+  public void processDocuments(String[] documentIdentifiers, IExistingVersions statuses, Specification spec,
+    IProcessActivity activities, int jobMode, boolean usesDefaultAuthority)
+    throws ManifoldCFException, ServiceInterruption
+  {
+    // The backwards-compatible base implementation does the following:
+    // (1) Uses the deprecated methods to obtain a set of version information
+    // (2) Based on the provided version information, determines whether processing is required
+    // (3) Uses deprecated methods to process documents
+    // (4) Releases document versions
+
+    // We need to get the old version strings together in order to use the deprecated methods
+    String[] oldVersions = new String[documentIdentifiers.length];
+    for (int i = 0; i < oldVersions.length; i++)
+    {
+      oldVersions[i] = statuses.getIndexedVersionString(documentIdentifiers[i]);
+    }
+    DocumentVersions dv = new DocumentVersions();
+    getDocumentVersions(dv,documentIdentifiers,oldVersions,activities,spec,jobMode,usesDefaultAuthority);
+    try
+    {
+      // Next, we determine what part of the set of documents were unchanged, and what part we need to refetch.
+      Set<String> fetchDocuments = new HashSet<String>();
+      Set<String> scanDocuments = new HashSet<String>();
+      for (int i = 0; i < documentIdentifiers.length; i++)
+      {
+        String documentIdentifier = documentIdentifiers[i];
+        VersionContent vc = dv.getDocumentVersion(documentIdentifier);
+        if (vc != null)
+        {
+          if (dv.isAlwaysRefetch(documentIdentifier) || activities.checkDocumentNeedsReindexing(documentIdentifier,vc.getVersionString()))
+            fetchDocuments.add(documentIdentifier);
+          scanDocuments.add(documentIdentifier);
+        }
+      }
+
+      // Construct the appropriate data to call processDocuments() with
+      String[] processIDs = new String[scanDocuments.size()];
+      boolean[] scanOnly = new boolean[scanDocuments.size()];
+      int index = 0;
+      for (int i = 0; i < documentIdentifiers.length; i++)
+      {
+        String documentIdentifier = documentIdentifiers[i];
+        if (scanDocuments.contains(documentIdentifier))
+        {
+          processIDs[index] = documentIdentifier;
+          scanOnly[index] = !fetchDocuments.contains(documentIdentifier);
+          index++;
+        }
+      }
+      processDocuments(processIDs,dv,activities,scanOnly,jobMode);
+    }
+    finally
+    {
+      // Release document versions
+      releaseDocumentVersions(documentIdentifiers,dv);
+    }
+  }
+
   /** Get document versions given an array of document identifiers.
   * This method is called for EVERY document that is considered. It is therefore important to perform
   * as little work as possible here.
@@ -340,7 +412,6 @@ public abstract class BaseRepositoryConn
   *@param jobMode is an integer describing how the job is being run, whether continuous or once-only.
   *@param usesDefaultAuthority will be true only if the authority in use for these documents is the default one.
   */
-  @Override
   public void getDocumentVersions(
     DocumentVersions documentVersions,
     String[] documentIdentifiers, String[] oldVersions,
@@ -488,7 +559,6 @@ public abstract class BaseRepositoryConn
   *@param documentIdentifiers is the set of document identifiers.
   *@param versions is the corresponding set of version strings (individual identifiers may have no version).
   */
-  @Override
   public void releaseDocumentVersions(String[] documentIdentifiers, DocumentVersions versions)
     throws ManifoldCFException
   {
@@ -543,7 +613,6 @@ public abstract class BaseRepositoryConn
   * should only find other references, and should not actually call the ingestion methods.
   *@param jobMode is an integer describing how the job is being run, whether continuous or once-only.
   */
-  @Override
   public void processDocuments(String[] documentIdentifiers, DocumentVersions versions, IProcessActivity activities,
     boolean[] scanOnly, int jobMode)
     throws ManifoldCFException, ServiceInterruption

Added: manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IExistingVersions.java
URL: http://svn.apache.org/viewvc/manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IExistingVersions.java?rev=1610414&view=auto
==============================================================================
--- manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IExistingVersions.java (added)
+++ manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IExistingVersions.java Mon Jul 14 14:10:36 2014
@@ -0,0 +1,37 @@
+/* $Id$ */
+
+/**
+* Licensed to the Apache Software Foundation (ASF) under one or more
+* contributor license agreements. See the NOTICE file distributed with
+* this work for additional information regarding copyright ownership.
+* The ASF licenses this file to You under the Apache License, Version 2.0
+* (the "License"); you may not use this file except in compliance with
+* the License. You may obtain a copy of the License at
+*
+* http://www.apache.org/licenses/LICENSE-2.0
+*
+* Unless required by applicable law or agreed to in writing, software
+* distributed under the License is distributed on an "AS IS" BASIS,
+* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+* See the License for the specific language governing permissions and
+* limitations under the License.
+*/
+package org.apache.manifoldcf.crawler.interfaces;
+
+import org.apache.manifoldcf.core.interfaces.*;
+import java.util.*;
+
+/** This interface describes functionality designed to allow retrieval of existing
+* version information from previous crawls.  It is part of the IRepositoryConnector API.
+*/
+public interface IExistingVersions
+{
+  public static final String _rcsid = "@(#)$Id$";
+
+  /** Retrieve an existing version string given a document identifier.
+  *@param documentIdentifier is the document identifier.
+  *@return the document version string, or null if the document was never previously indexed.
+  */
+  public String getIndexedVersionString(String documentIdentifier);
+
+}

Propchange: manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IExistingVersions.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IExistingVersions.java
------------------------------------------------------------------------------
    svn:keywords = Id

Modified: manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IProcessActivity.java
URL: http://svn.apache.org/viewvc/manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IProcessActivity.java?rev=1610414&r1=1610413&r2=1610414&view=diff
==============================================================================
--- manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IProcessActivity.java (original)
+++ manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IProcessActivity.java Mon Jul 14 14:10:36 2014
@@ -23,15 +23,41 @@ import java.io.*;
 import org.apache.manifoldcf.core.interfaces.*;
 import org.apache.manifoldcf.agents.interfaces.*;
 
-/** This interface abstracts from the activities that a fetched document processor can do.
+/** This interface abstracts from the activities that a connector's processDocuments() method can do.
+* The processing flow for a document is expected to go something like this:
+* (1) The connector's processDocuments() method is called with a set of documents to be processed.
+* (2) The connector computes a version string for each document in the set as part of determining
+*    whether the document indeed needs to be refetched.
+* (3) For each document processed, there can be one of several dispositions:
+*   (a) There is no such document (anymore): nothing is called for the document (the framework will delete it).
+*   (b) The document is (re)indexed: ingestDocumentWithException() is called for the document.
+*   (c) The document is determined to be unchanged and no updates are needed: noteUnchangedDocument() is called
+*     for the document.
+*   (d) The document is determined to be unchanged BUT the version string needs to be updated: recordDocument()
+*     is called for the document.
+*   (e) The document is determined to be unindexable BUT it still exists in the repository: noDocument()
+*    is called for the document.
+*   (f) There was a service interruption: ServiceInterruption is thrown.
+* (4) In order to determine whether a document needs to be reindexed, the method checkDocumentNeedsReindexing()
+*    is available to return an opinion on that matter.
 */
-public interface IProcessActivity extends IHistoryActivity, IEventActivity, IAbortActivity, IFingerprintActivity,
-    ICarrydownActivity
+public interface IProcessActivity extends IVersionActivity
 {
   public static final String _rcsid = "@(#)$Id: IProcessActivity.java 988245 2010-08-23 18:39:35Z kwright $";
 
+  /** Check if a document needs to be reindexed, based on a computed version string.
+  * Call this method to determine whether reindexing is necessary.  Pass in a newly-computed version
+  * string.  This method will return "true" if the document needs to be re-indexed.
+  *@param documentIdentifier is the document identifier.
+  *@param newVersionString is the newly-computed version string.
+  *@return true if the document needs to be reindexed.
+  */
+  public boolean checkDocumentNeedsReindexing(String documentIdentifier,
+    String newVersionString)
+    throws ManifoldCFException;
+
   /** Add a document description to the current job's queue.
-  *@param localIdentifier is the local document identifier to add (for the connector that
+  *@param documentIdentifier is the local document identifier to add (for the connector that
   * fetched the document).
   *@param parentIdentifier is the document identifier that is considered to be the "parent"
   * of this identifier.  May be null, if no hopcount filtering desired for this kind of relationship.
@@ -45,12 +71,12 @@ public interface IProcessActivity extend
   *@param originationTime is the time, in ms since epoch, that the document originated.  Pass null if none or unknown.
   *@param prereqEventNames are the names of the prerequisite events which this document requires prior to processing.  Pass null if none.
   */
-  public void addDocumentReference(String localIdentifier, String parentIdentifier, String relationshipType,
+  public void addDocumentReference(String documentIdentifier, String parentIdentifier, String relationshipType,
     String[] dataNames, Object[][] dataValues, Long originationTime, String[] prereqEventNames)
     throws ManifoldCFException;
 
   /** Add a document description to the current job's queue.
-  *@param localIdentifier is the local document identifier to add (for the connector that
+  *@param documentIdentifier is the document identifier to add (for the connector that
   * fetched the document).
   *@param parentIdentifier is the document identifier that is considered to be the "parent"
   * of this identifier.  May be null, if no hopcount filtering desired for this kind of relationship.
@@ -63,13 +89,12 @@ public interface IProcessActivity extend
   *          The type of each object must either be a String, or a CharacterInput.
   *@param originationTime is the time, in ms since epoch, that the document originated.  Pass null if none or unknown.
   */
-  public void addDocumentReference(String localIdentifier, String parentIdentifier, String relationshipType,
+  public void addDocumentReference(String documentIdentifier, String parentIdentifier, String relationshipType,
     String[] dataNames, Object[][] dataValues, Long originationTime)
     throws ManifoldCFException;
 
-
   /** Add a document description to the current job's queue.
-  *@param localIdentifier is the local document identifier to add (for the connector that
+  *@param documentIdentifier is the document identifier to add (for the connector that
   * fetched the document).
   *@param parentIdentifier is the document identifier that is considered to be the "parent"
   * of this identifier.  May be null, if no hopcount filtering desired for this kind of relationship.
@@ -81,12 +106,12 @@ public interface IProcessActivity extend
   *@param dataValues are the values that correspond to the data names in the dataNames parameter.  May be null only if dataNames is null.
   *          The type of each object must either be a String, or a CharacterInput.
   */
-  public void addDocumentReference(String localIdentifier, String parentIdentifier, String relationshipType,
+  public void addDocumentReference(String documentIdentifier, String parentIdentifier, String relationshipType,
     String[] dataNames, Object[][] dataValues)
     throws ManifoldCFException;
 
   /** Add a document description to the current job's queue.
-  *@param localIdentifier is the local document identifier to add (for the connector that
+  *@param documentIdentifier is the document identifier to add (for the connector that
   * fetched the document).
   *@param parentIdentifier is the document identifier that is considered to be the "parent"
   * of this identifier.  May be null, if no hopcount filtering desired for this kind of relationship.
@@ -94,27 +119,19 @@ public interface IProcessActivity extend
   * reference.  This must be one of the strings returned by the IRepositoryConnector method
   * "getRelationshipTypes()".  May be null.
   */
-  public void addDocumentReference(String localIdentifier, String parentIdentifier, String relationshipType)
+  public void addDocumentReference(String documentIdentifier, String parentIdentifier, String relationshipType)
     throws ManifoldCFException;
 
   /** Add a document description to the current job's queue.  This method is equivalent to
   * addDocumentReference(localIdentifier,null,null).
-  *@param localIdentifier is the local document identifier to add (for the connector that
+  *@param documentIdentifier is the document identifier to add (for the connector that
   * fetched the document).
   */
-  public void addDocumentReference(String localIdentifier)
+  public void addDocumentReference(String documentIdentifier)
     throws ManifoldCFException;
 
-
-  /** Record a document version, but don't ingest it.
-  *@param localIdentifier is the document identifier.
-  *@param version is the document version.
-  */
-  public void recordDocument(String localIdentifier, String version)
-    throws ManifoldCFException, ServiceInterruption;
-
   /** Ingest the current document.
-  *@param localIdentifier is the document's local identifier.
+  *@param documentIdentifier is the document's identifier.
   *@param version is the version of the document, as reported by the getDocumentVersions() method of the
   *       corresponding repository connector.
   *@param documentURI is the URI to use to retrieve this document from the search interface (and is
@@ -122,11 +139,11 @@ public interface IProcessActivity extend
   *@param data is the document data.  The data is closed after ingestion is complete.
   *@throws IOException only when data stream reading fails.
   */
-  public void ingestDocumentWithException(String localIdentifier, String version, String documentURI, RepositoryDocument data)
+  public void ingestDocumentWithException(String documentIdentifier, String version, String documentURI, RepositoryDocument data)
     throws ManifoldCFException, ServiceInterruption, IOException;
 
   /** Ingest the current document.
-  *@param localIdentifier is the document's local identifier.
+  *@param documentIdentifier is the document's identifier.
   *@param version is the version of the document, as reported by the getDocumentVersions() method of the
   *       corresponding repository connector.
   *@param documentURI is the URI to use to retrieve this document from the search interface (and is
@@ -136,48 +153,78 @@ public interface IProcessActivity extend
   * according to standard best practices.
   */
   @Deprecated
-  public void ingestDocument(String localIdentifier, String version, String documentURI, RepositoryDocument data)
+  public void ingestDocument(String documentIdentifier, String version, String documentURI, RepositoryDocument data)
     throws ManifoldCFException, ServiceInterruption;
 
+  /** Note the fact that a document exists but is unchanged, and nothing further
+  * needs to be done to it.
+  * Call this method if it is determined that the document in question is identical to
+  * the formerly indexed document, AND when the version string for the document
+  * has not changed either.
+  */
+  public void noteUnchangedDocument(String documentIdentifier)
+    throws ManifoldCFException;
+
+  /** Remove the specified document from the search engine index, and update the
+  * recorded version information for the document.
+  *@param documentIdentifier is the document's local identifier.
+  *@param version is the version string to be recorded for the document.
+  */
+  public void noDocument(String documentIdentifier, String version)
+    throws ManifoldCFException, ServiceInterruption;
+
+  /** Record a document version, WITHOUT reindexing it, or removing it.  (Other
+  * documents with the same URL, however, will still be removed.)  This is
+  * useful if the version string changes but the document contents are known not
+  * to have changed.
+  *@param documentIdentifier is the document identifier.
+  *@param version is the document version.
+  */
+  public void recordDocument(String documentIdentifier, String version)
+    throws ManifoldCFException;
+
   /** Delete the current document from the search engine index, while keeping track of the version information
   * for it (to reduce churn).
-  *@param localIdentifier is the document's local identifier.
-  *@param version is the version of the document, as reported by the getDocumentVersions() method of the
-  *       corresponding repository connector.
+  * Deprecated; use noDocument() above instead.
+  *@param documentIdentifier is the document's local identifier.
+  *@param version is the version string to be recorded for the document.
   */
-  public void deleteDocument(String localIdentifier, String version)
+  @Deprecated
+  public void deleteDocument(String documentIdentifier, String version)
     throws ManifoldCFException, ServiceInterruption;
 
-  /** Delete the current document from the search engine index.  This method does NOT keep track of version
-  * information for the document and thus can lead to "churn", whereby the same document is queued, versioned,
-  * and removed on subsequent crawls.  It therefore should be considered to be deprecated, in favor of
-  * deleteDocument(String localIdentifier, String version).
-  *@param localIdentifier is the document's local identifier.
+  /** Delete the specified document permanently from the search engine index, and from the status table.
+  * This method does NOT keep track of any document version information for the document and thus can
+  * lead to "churn", whereby the same document is queued, processed,
+  * and removed on subsequent crawls.  It is therefore preferable to use noDocument() instead,
+  * in any case where the same decision will need to be made over and over.
+  *@param documentIdentifier is the document's identifier.
   */
-  public void deleteDocument(String localIdentifier)
+  @Deprecated
+  public void deleteDocument(String documentIdentifier)
     throws ManifoldCFException, ServiceInterruption;
 
   /** Override the schedule for the next time a document is crawled.
   * Calling this method allows you to set an upper recrawl bound, lower recrawl bound, upper expire bound, lower expire bound,
   * or a combination of these, on a specific document.  This method is only effective if the job is a continuous one, and if the
   * identifier you pass in is being processed.
-  *@param localIdentifier is the document's local identifier.
+  *@param documentIdentifier is the document's identifier.
   *@param lowerRecrawlBoundTime is the time in ms since epoch that the reschedule time should not fall BELOW, or null if none.
   *@param upperRecrawlBoundTime is the time in ms since epoch that the reschedule time should not rise ABOVE, or null if none.
   *@param lowerExpireBoundTime is the time in ms since epoch that the expire time should not fall BELOW, or null if none.
   *@param upperExpireBoundTime is the time in ms since epoch that the expire time should not rise ABOVE, or null if none.
   */
-  public void setDocumentScheduleBounds(String localIdentifier,
+  public void setDocumentScheduleBounds(String documentIdentifier,
     Long lowerRecrawlBoundTime, Long upperRecrawlBoundTime,
     Long lowerExpireBoundTime, Long upperExpireBoundTime)
     throws ManifoldCFException;
 
   /** Override a document's origination time.
   * Use this method to signal the framework that a document's origination time is something other than the first time it was crawled.
-  *@param localIdentifier is the document's local identifier.
+  *@param documentIdentifier is the document's identifier.
   *@param originationTime is the document's origination time, or null if unknown.
   */
-  public void setDocumentOriginationTime(String localIdentifier,
+  public void setDocumentOriginationTime(String documentIdentifier,
     Long originationTime)
     throws ManifoldCFException;
 

Modified: manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IRepositoryConnector.java
URL: http://svn.apache.org/viewvc/manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IRepositoryConnector.java?rev=1610414&r1=1610413&r2=1610414&view=diff
==============================================================================
--- manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IRepositoryConnector.java (original)
+++ manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/interfaces/IRepositoryConnector.java Mon Jul 14 14:10:36 2014
@@ -48,16 +48,13 @@ import java.util.*;
 * It therefore establishes a space of document identifiers.  Each connector will only ever be
 * asked to deal with identifiers that have in some way originated from the connector.
 *
-* Documents are fetched by ManifoldCF in three stages.  First, the addSeedDocuments() method is called in the connector
+* Documents are fetched by ManifoldCF in two stages.  First, the addSeedDocuments() method is called in the connector
 * implementation.  This method is meant to add a set of document identifiers to the queue.  When ManifoldCF is ready
-* to process a document, the document identifier is used to obtain a current document version string, using the
-* getDocumentVersions() method (the second stage).  This version string is used to decide whether or not the
-* third stage need be called for the document or not.  The third stage is responsible for sending document content
-* to the output, and for extracting any references to additional documents, and consists of the processDocuments() method.
+* to process a document, the document identifier is used to build a version string for the document and check whether
+* the document needs to be indexed, and index it if needed (the second stage).  The second stage
+* consists of the processDocuments() method.
 *
-* All of these methods interact with ManifoldCF by means of an "activity" interface.  For example, an IVersionActivity object
-* is passed to the getDocumentVersions() method, and that object contains methods that are necessary for getDocumentVersions()
-* to do its job.  A similar architecture is used throughout the connector framework.
+* All of these methods interact with ManifoldCF by means of an "activity" interface.
 */
 public interface IRepositoryConnector extends IConnector
 {
@@ -182,57 +179,23 @@ public interface IRepositoryConnector ex
     String lastSeedVersion, long seedTime, int jobMode)
     throws ManifoldCFException, ServiceInterruption;
 
-  /** Get document versions given an array of document identifiers.
-  * This method is called for EVERY document that is considered. It is therefore important to perform
-  * as little work as possible here.
-  * The connector will be connected before this method can be called.
-  *@param documentVersions is the versions object, to be filled in by this method.
-  *@param documentIdentifiers is the array of local document identifiers, as understood by this connector.
-  *@param oldVersions is the corresponding array of version strings that have been saved for the document identifiers.
-  *   A null value indicates that this is a first-time fetch, while an empty string indicates that the previous document
-  *   had an empty version string.
-  *@param activities is the interface this method should use to perform whatever framework actions are desired.
-  *@param spec is the current document specification for the current job.  If there is a dependency on this
-  * specification, then the version string should include the pertinent data, so that reingestion will occur
-  * when the specification changes.  This is primarily useful for metadata.
-  *@param jobMode is an integer describing how the job is being run, whether continuous or once-only.
-  *@param usesDefaultAuthority will be true only if the authority in use for these documents is the default one.
-  */
-  public void getDocumentVersions(
-    DocumentVersions documentVersions,
-    String[] documentIdentifiers, String[] oldVersions,
-    IVersionActivity activities,
-    Specification spec, int jobMode, boolean usesDefaultAuthority)
-    throws ManifoldCFException, ServiceInterruption;
-
   /** Process a set of documents.
   * This is the method that should cause each document to be fetched, processed, and the results either added
   * to the queue of documents for the current job, and/or entered into the incremental ingestion manager.
   * The document specification allows this class to filter what is done based on the job.
   * The connector will be connected before this method can be called.
   *@param documentIdentifiers is the set of document identifiers to process.
-  *@param versions are the version strings returned by getDocumentVersions() above.
+  *@param statuses are the currently-stored document versions for each document in the set of document identifiers
+  * passed in above.
   *@param activities is the interface this method should use to queue up new document references
   * and ingest documents.
-  *@param scanOnly is an array corresponding to the document identifiers.  It is set to true to indicate when the processing
-  * should only find other references, and should not actually call the ingestion methods.
   *@param jobMode is an integer describing how the job is being run, whether continuous or once-only.
+  *@param usesDefaultAuthority will be true only if the authority in use for these documents is the default one.
   */
-  public void processDocuments(String[] documentIdentifiers, DocumentVersions versions, IProcessActivity activities,
-    boolean[] scanOnly, int jobMode)
+  public void processDocuments(String[] documentIdentifiers, IExistingVersions statuses, Specification spec,
+    IProcessActivity activities, int jobMode, boolean usesDefaultAuthority)
     throws ManifoldCFException, ServiceInterruption;
 
-  /** Free a set of documents.  This method is called for all documents whose versions have been fetched using
-  * the getDocumentVersions() method, including those that returned null versions.  It may be used to free resources
-  * committed during the getDocumentVersions() method.  It is guaranteed to be called AFTER any calls to
-  * processDocuments() for the documents in question.
-  * The connector will be connected before this method can be called.
-  *@param documentIdentifiers is the set of document identifiers.
-  *@param versions is the corresponding set of version strings (individual identifiers may have no version).
-  */
-  public void releaseDocumentVersions(String[] documentIdentifiers, DocumentVersions versions)
-    throws ManifoldCFException;
-
   /** Get the maximum number of documents to amalgamate together into one batch, for this connector.
   * The connector does not need to be connected for this method to be called.
   *@return the maximum number. 0 indicates "unlimited".

Modified: manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/WorkerThread.java
URL: http://svn.apache.org/viewvc/manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/WorkerThread.java?rev=1610414&r1=1610413&r2=1610414&view=diff
==============================================================================
--- manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/WorkerThread.java (original)
+++ manifoldcf/branches/CONNECTORS-990/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/WorkerThread.java Mon Jul 14 14:10:36 2014
@@ -1642,6 +1642,22 @@ public class WorkerThread extends Thread
       existingDr.addPrerequisiteEvents(prereqEventNames);
     }
 
+    /** Check if a document needs to be reindexed, based on a computed version string.
+    * Call this method to determine whether reindexing is necessary.  Pass in a newly-computed version
+    * string.  This method will return "true" if the document needs to be re-indexed.
+    *@param documentIdentifier is the document identifier.
+    *@param newVersionString is the newly-computed version string.
+    *@return true if the document needs to be reindexed.
+    */
+    @Override
+    public boolean checkDocumentNeedsReindexing(String documentIdentifier,
+      String newVersionString)
+      throws ManifoldCFException
+    {
+      IPipelineSpecificationWithVersions spec = fetchPipelineSpecifications.get(documentIdentifier);
+      return ingester.checkFetchDocument(spec,newVersionString,parameterVersion,connection.getACLAuthority());
+    }
+
     /** Add a document description to the current job's queue.
     *@param localIdentifier is the local document identifier to add (for the connector that
     * fetched the document).
@@ -1733,20 +1749,32 @@ public class WorkerThread extends Thread
       return jobManager.retrieveParentDataAsFiles(jobID,ManifoldCF.hash(localIdentifier),dataName);
     }
 
+    /** Note the fact that a document exists but is unchanged, and nothing further
+    * needs to be done to it.
+    * Call this method if it is determined that the document in question is identical to
+    * the formerly indexed document, AND when the version string for the document
+    * has not changed either.
+    */
+    @Override
+    public void noteUnchangedDocument(String documentIdentifier)
+      throws ManifoldCFException
+    {
+      // MHL ???
+    }
+
     /** Record a document version, but don't ingest it.
-    * ServiceInterruption is thrown if this action must be rescheduled.
     *@param documentIdentifier is the document identifier.
     *@param version is the document version.
     */
     @Override
     public void recordDocument(String documentIdentifier, String version)
-      throws ManifoldCFException, ServiceInterruption
+      throws ManifoldCFException
     {
       String documentIdentifierHash = ManifoldCF.hash(documentIdentifier);
       ingester.documentRecord(
         pipelineSpecification.getBasicPipelineSpecification(),
         connectionName,documentIdentifierHash,
-        version,currentTime,ingestLogger);
+        version,currentTime);
     }
 
     /** Ingest the current document.
@@ -1823,16 +1851,15 @@ public class WorkerThread extends Thread
         ingestLogger);
     }
 
-    /** Delete the current document from the search engine index, while keeping track of the version information
+    /** Remove the specified document from the search engine index, while keeping track of the version information
     * for it (to reduce churn).
     *@param documentIdentifier is the document's local identifier.
-    *@param version is the version of the document, as reported by the getDocumentVersions() method of the
-    *       corresponding repository connector.
+    *@param version is the version string to be recorded for the document.
     */
-    @Override
-    public void deleteDocument(String documentIdentifier, String version)
+    public void noDocument(String documentIdentifier, String version)
       throws ManifoldCFException, ServiceInterruption
     {
+      // Special interpretation for empty version string
       if (version.length() == 0)
         deleteDocument(documentIdentifier);
       else
@@ -1847,15 +1874,32 @@ public class WorkerThread extends Thread
           throw new IllegalStateException("IngestDocumentWithException threw an illegal IOException: "+e.getMessage(),e);
         }
       }
+
     }
 
-    /** Delete the current document from the search engine index.  This method does NOT keep track of version
-    * information for the document and thus can lead to "churn", whereby the same document is queued, versioned,
-    * and removed on subsequent crawls.  It therefore should be considered to be deprecated, in favor of
-    * deleteDocument(String localIdentifier, String version).
+    /** Delete the current document from the search engine index, while keeping track of the version information
+    * for it (to reduce churn).
+    * Use noDocument() above instead.
     *@param documentIdentifier is the document's local identifier.
+    *@param version is the version string to be recorded for the document.
+    */
+    @Override
+    @Deprecated
+    public void deleteDocument(String documentIdentifier, String version)
+      throws ManifoldCFException, ServiceInterruption
+    {
+      noDocument(documentIdentifier,version);
+    }
+
+    /** Delete the specified document from the search engine index, and from the status table.  This
+    *  method does NOT keep track of version
+    * information for the document and thus can lead to "churn", whereby the same document is queued, processed,
+    * and removed on subsequent crawls.  It is therefore preferable to use noDocument() instead,
+    * in any case where the same decision will need to be made over and over.
+    *@param documentIdentifier is the document's identifier.
     */
     @Override
+    @Deprecated
     public void deleteDocument(String documentIdentifier)
       throws ManifoldCFException, ServiceInterruption
     {