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/18 19:55:46 UTC

svn commit: r1611735 - in /manifoldcf/branches/CONNECTORS-989/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: Fri Jul 18 17:55:46 2014
New Revision: 1611735

URL: http://svn.apache.org/r1611735
Log:
First batch of work changing incremental ingester API

Modified:
    manifoldcf/branches/CONNECTORS-989/framework/agents/src/main/java/org/apache/manifoldcf/agents/incrementalingest/IncrementalIngester.java
    manifoldcf/branches/CONNECTORS-989/framework/agents/src/main/java/org/apache/manifoldcf/agents/interfaces/IIncrementalIngester.java
    manifoldcf/branches/CONNECTORS-989/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/WorkerThread.java

Modified: manifoldcf/branches/CONNECTORS-989/framework/agents/src/main/java/org/apache/manifoldcf/agents/incrementalingest/IncrementalIngester.java
URL: http://svn.apache.org/viewvc/manifoldcf/branches/CONNECTORS-989/framework/agents/src/main/java/org/apache/manifoldcf/agents/incrementalingest/IncrementalIngester.java?rev=1611735&r1=1611734&r2=1611735&view=diff
==============================================================================
--- manifoldcf/branches/CONNECTORS-989/framework/agents/src/main/java/org/apache/manifoldcf/agents/incrementalingest/IncrementalIngester.java (original)
+++ manifoldcf/branches/CONNECTORS-989/framework/agents/src/main/java/org/apache/manifoldcf/agents/incrementalingest/IncrementalIngester.java Fri Jul 18 17:55:46 2014
@@ -612,16 +612,18 @@ public class IncrementalIngester extends
   *@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 componentHash is the hashed component identifier, if any.
   *@param documentVersion is the document version.
   *@param recordTime is the time at which the recording took place, in milliseconds since epoch.
   */
   @Override
   public void documentRecord(
     IPipelineSpecificationBasic pipelineSpecificationBasic,
-    String identifierClass, String identifierHash,
+    String identifierClass, String identifierHash, String componentHash,
     String documentVersion, long recordTime)
     throws ManifoldCFException
   {
+    // MHL
     // 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
@@ -661,6 +663,7 @@ public class IncrementalIngester extends
   *@param pipelineSpecificationWithVersions is the pipeline specification with already-fetched output versioning information.
   *@param identifierClass is the name of the space in which the identifier hash should be interpreted.
   *@param identifierHash is the hashed document identifier.
+  *@param componentHash is the hashed component identifier, if any.
   *@param documentVersion is the document version.
   *@param parameterVersion is the version string for the forced parameters.
   *@param authorityName is the name of the authority associated with the document, if any.
@@ -670,7 +673,7 @@ public class IncrementalIngester extends
   @Override
   public void documentNoData(
     IPipelineSpecificationWithVersions pipelineSpecificationWithVersions,
-    String identifierClass, String identifierHash,
+    String identifierClass, String identifierHash, String componentHash,
     String documentVersion,
     String parameterVersion,
     String authorityName,
@@ -678,6 +681,7 @@ public class IncrementalIngester extends
     IOutputActivity activities)
     throws ManifoldCFException, ServiceInterruption
   {
+    // MHL
     PipelineConnectionsWithVersions pipelineConnectionsWithVersions = new PipelineConnectionsWithVersions(pipelineSpecificationWithVersions);
     
     String docKey = makeKey(identifierClass,identifierHash);
@@ -710,6 +714,7 @@ public class IncrementalIngester extends
   *@param pipelineSpecificationWithVersions is the pipeline specification with already-fetched output versioning information.
   *@param identifierClass is the name of the space in which the identifier hash should be interpreted.
   *@param identifierHash is the hashed document identifier.
+  *@param componentHash is the hashed component identifier, if any.
   *@param documentVersion is the document version.
   *@param parameterVersion is the version string for the forced parameters.
   *@param authorityName is the name of the authority associated with the document, if any.
@@ -723,15 +728,16 @@ public class IncrementalIngester extends
   @Override
   public boolean documentIngest(
     IPipelineSpecificationWithVersions pipelineSpecificationWithVersions,
-    String identifierClass, String identifierHash,
+    String identifierClass, String identifierHash, String componentHash,
     String documentVersion,
     String parameterVersion,
     String authorityName,
-    RepositoryDocument document,
+    RepositoryDocument data,
     long ingestTime, String documentURI,
     IOutputActivity activities)
     throws ManifoldCFException, ServiceInterruption, IOException
   {
+    // MHL
     PipelineConnectionsWithVersions pipelineConnectionsWithVersions = new PipelineConnectionsWithVersions(pipelineSpecificationWithVersions);
     
     String docKey = makeKey(identifierClass,identifierHash);
@@ -742,7 +748,7 @@ public class IncrementalIngester extends
     }
 
     // Set indexing date
-    document.setIndexingDate(new Date());
+    data.setIndexingDate(new Date());
     
     // Set up a pipeline
     PipelineObjectWithVersions pipeline = pipelineGrabWithVersions(pipelineConnectionsWithVersions);
@@ -751,7 +757,7 @@ public class IncrementalIngester extends
       throw new ServiceInterruption("Pipeline connector not installed",0L);
     try
     {
-      return pipeline.addOrReplaceDocumentWithException(docKey,documentURI,document,documentVersion,parameterVersion,authorityName,activities,ingestTime) == IPipelineConnector.DOCUMENTSTATUS_ACCEPTED;
+      return pipeline.addOrReplaceDocumentWithException(docKey,documentURI,data,documentVersion,parameterVersion,authorityName,activities,ingestTime) == IPipelineConnector.DOCUMENTSTATUS_ACCEPTED;
     }
     finally
     {
@@ -759,6 +765,25 @@ public class IncrementalIngester extends
     }
   }
 
+  /** Remove a document component from the search engine index.
+  *@param pipelineSpecificationBasic is the basic pipeline specification.
+  *@param identifierClass is the name of the space in which the identifier hash should be interpreted.
+  *@param identifierHash is the hash of the id of the document.
+  *@param componentHash is the hashed component identifier, if any.
+  *@param activities is the object to use to log the details of the ingestion attempt.  May be null.
+  */
+  @Override
+  public void documentRemove(
+    IPipelineSpecificationBasic pipelineSpecificationBasic,
+    String identifierClass, String identifierHash, String componentHash,
+    IOutputRemoveActivity activities)
+    throws ManifoldCFException, ServiceInterruption
+  {
+    // MHL
+    documentDelete(pipelineSpecificationBasic,
+      identifierClass,identifierHash,activities);
+  }
+
   protected static String[] extractOutputConnectionNames(IPipelineSpecificationBasic pipelineSpecificationBasic)
   {
     String[] rval = new String[pipelineSpecificationBasic.getOutputCount()];

Modified: manifoldcf/branches/CONNECTORS-989/framework/agents/src/main/java/org/apache/manifoldcf/agents/interfaces/IIncrementalIngester.java
URL: http://svn.apache.org/viewvc/manifoldcf/branches/CONNECTORS-989/framework/agents/src/main/java/org/apache/manifoldcf/agents/interfaces/IIncrementalIngester.java?rev=1611735&r1=1611734&r2=1611735&view=diff
==============================================================================
--- manifoldcf/branches/CONNECTORS-989/framework/agents/src/main/java/org/apache/manifoldcf/agents/interfaces/IIncrementalIngester.java (original)
+++ manifoldcf/branches/CONNECTORS-989/framework/agents/src/main/java/org/apache/manifoldcf/agents/interfaces/IIncrementalIngester.java Fri Jul 18 17:55:46 2014
@@ -154,12 +154,13 @@ public interface IIncrementalIngester
   *@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 componentHash is the hashed component identifier, if any.
   *@param documentVersion is the document version.
   *@param recordTime is the time at which the recording took place, in milliseconds since epoch.
   */
   public void documentRecord(
     IPipelineSpecificationBasic pipelineSpecificationBasic,
-    String identifierClass, String identifierHash,
+    String identifierClass, String identifierHash, String componentHash,
     String documentVersion, long recordTime)
     throws ManifoldCFException;
 
@@ -171,6 +172,7 @@ public interface IIncrementalIngester
   *@param pipelineSpecificationWithVersions is the pipeline specification with already-fetched output versioning information.
   *@param identifierClass is the name of the space in which the identifier hash should be interpreted.
   *@param identifierHash is the hashed document identifier.
+  *@param componentHash is the hashed component identifier, if any.
   *@param documentVersion is the document version.
   *@param parameterVersion is the version string for the forced parameters.
   *@param authorityName is the name of the authority associated with the document, if any.
@@ -179,7 +181,7 @@ public interface IIncrementalIngester
   */
   public void documentNoData(
     IPipelineSpecificationWithVersions pipelineSpecificationWithVersions,
-    String identifierClass, String identifierHash,
+    String identifierClass, String identifierHash, String componentHash,
     String documentVersion,
     String parameterVersion,
     String authorityName,
@@ -195,6 +197,7 @@ public interface IIncrementalIngester
   *@param pipelineSpecificationWithVersions is the pipeline specification with already-fetched output versioning information.
   *@param identifierClass is the name of the space in which the identifier hash should be interpreted.
   *@param identifierHash is the hashed document identifier.
+  *@param componentHash is the hashed component identifier, if any.
   *@param documentVersion is the document version.
   *@param parameterVersion is the version string for the forced parameters.
   *@param authorityName is the name of the authority associated with the document, if any.
@@ -207,7 +210,7 @@ public interface IIncrementalIngester
   */
   public boolean documentIngest(
     IPipelineSpecificationWithVersions pipelineSpecificationWithVersions,
-    String identifierClass, String identifierHash,
+    String identifierClass, String identifierHash, String componentHash,
     String documentVersion,
     String parameterVersion,
     String authorityName,
@@ -216,6 +219,19 @@ public interface IIncrementalIngester
     IOutputActivity activities)
     throws ManifoldCFException, ServiceInterruption, IOException;
 
+  /** Remove a document component from the search engine index.
+  *@param pipelineSpecificationBasic is the basic pipeline specification.
+  *@param identifierClass is the name of the space in which the identifier hash should be interpreted.
+  *@param identifierHash is the hash of the id of the document.
+  *@param componentHash is the hashed component identifier, if any.
+  *@param activities is the object to use to log the details of the ingestion attempt.  May be null.
+  */
+  public void documentRemove(
+    IPipelineSpecificationBasic pipelineSpecificationBasic,
+    String identifierClass, String identifierHash, String componentHash,
+    IOutputRemoveActivity activities)
+    throws ManifoldCFException, ServiceInterruption;
+
   /** Note the fact that we checked a document (and found that it did not need to be ingested, because the
   * versions agreed).
   *@param pipelineSpecificationBasic is a pipeline specification.
@@ -242,7 +258,7 @@ public interface IIncrementalIngester
     long checkTime)
     throws ManifoldCFException;
 
-  /** Delete multiple documents from the search engine index.
+  /** Delete multiple documents, and their components, from the search engine index.
   *@param pipelineSpecificationBasics are the pipeline specifications associated with the documents.
   *@param identifierClasses are the names of the spaces in which the identifier hashes should be interpreted.
   *@param identifierHashes is tha array of document identifier hashes if the documents.
@@ -254,7 +270,7 @@ public interface IIncrementalIngester
     IOutputRemoveActivity activities)
     throws ManifoldCFException, ServiceInterruption;
 
-  /** Delete multiple documents from the search engine index.
+  /** Delete multiple documents, and their components, from the search engine index.
   *@param pipelineSpecificationBasic is the basic pipeline specification.
   *@param identifierClasses are the names of the spaces in which the identifier hashes should be interpreted.
   *@param identifierHashes is tha array of document identifier hashes if the documents.
@@ -266,7 +282,7 @@ public interface IIncrementalIngester
     IOutputRemoveActivity activities)
     throws ManifoldCFException, ServiceInterruption;
 
-  /** Delete a document from the search engine index.
+  /** Delete a document, and all its components, from the search engine index.
   *@param pipelineSpecificationBasic is the basic pipeline specification.
   *@param identifierClass is the name of the space in which the identifier hash should be interpreted.
   *@param identifierHash is the hash of the id of the document.

Modified: manifoldcf/branches/CONNECTORS-989/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/WorkerThread.java
URL: http://svn.apache.org/viewvc/manifoldcf/branches/CONNECTORS-989/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/WorkerThread.java?rev=1611735&r1=1611734&r2=1611735&view=diff
==============================================================================
--- manifoldcf/branches/CONNECTORS-989/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/WorkerThread.java (original)
+++ manifoldcf/branches/CONNECTORS-989/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/WorkerThread.java Fri Jul 18 17:55:46 2014
@@ -1433,12 +1433,7 @@ public class WorkerThread extends Thread
     public void recordDocument(String documentIdentifier, String version)
       throws ManifoldCFException
     {
-      String documentIdentifierHash = ManifoldCF.hash(documentIdentifier);
-      ingester.documentRecord(
-        pipelineSpecification.getBasicPipelineSpecification(),
-        connectionName,documentIdentifierHash,
-        version,currentTime);
-      touchedSet.add(documentIdentifier);
+      recordDocument(documentIdentifier,null,version);
     }
 
     /** Record a document version, WITHOUT reindexing it, or removing it.  (Other
@@ -1455,7 +1450,17 @@ public class WorkerThread extends Thread
       String version)
       throws ManifoldCFException
     {
-      // MHL
+      String documentIdentifierHash = ManifoldCF.hash(documentIdentifier);
+      String componentIdentifierHash;
+      if (componentIdentifier != null)
+        componentIdentifierHash = ManifoldCF.hash(componentIdentifier);
+      else
+        componentIdentifierHash = null;
+      ingester.documentRecord(
+        pipelineSpecification.getBasicPipelineSpecification(),
+        connectionName,documentIdentifierHash,componentIdentifierHash,
+        version,currentTime);
+      touchedSet.add(documentIdentifier);
     }
 
     /** Ingest the current document.
@@ -1497,11 +1502,34 @@ public class WorkerThread extends Thread
     public void ingestDocumentWithException(String documentIdentifier, String version, String documentURI, RepositoryDocument data)
       throws ManifoldCFException, ServiceInterruption, IOException
     {
+    }
+
+    /** Ingest the current document.
+    *@param documentIdentifier is the document's identifier.
+    *@param componentIdentifier is the component document identifier, if any.
+    *@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
+    *       also the unique key in the index).
+    *@param data is the document data.  The data is closed after ingestion is complete.
+    *@throws IOException only when data stream reading fails.
+    */
+    @Override
+    public void ingestDocumentWithException(String documentIdentifier,
+      String componentIdentifier,
+      String version, String documentURI, RepositoryDocument data)
+      throws ManifoldCFException, ServiceInterruption, IOException
+    {
       // We should not get called here if versions agree, unless the repository
       // connector cannot distinguish between versions - in which case it must
       // always ingest (essentially)
 
       String documentIdentifierHash = ManifoldCF.hash(documentIdentifier);
+      String componentIdentifierHash;
+      if (componentIdentifier != null)
+        componentIdentifierHash = ManifoldCF.hash(componentIdentifier);
+      else
+        componentIdentifierHash = null;
 
       if (data != null)
       {
@@ -1524,7 +1552,7 @@ public class WorkerThread extends Thread
       // First, we need to add into the metadata the stuff from the job description.
       ingester.documentIngest(
         fetchPipelineSpecifications.get(documentIdentifierHash),
-        connectionName,documentIdentifierHash,
+        connectionName,documentIdentifierHash,componentIdentifierHash,
         version,parameterVersion,
         connection.getACLAuthority(),
         data,currentTime,
@@ -1534,45 +1562,16 @@ public class WorkerThread extends Thread
       touchedSet.add(documentIdentifier);
     }
 
-    /** Ingest the current document.
-    *@param documentIdentifier is the document's identifier.
-    *@param componentIdentifier is the component document identifier, if any.
-    *@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
-    *       also the unique key in the index).
-    *@param data is the document data.  The data is closed after ingestion is complete.
-    *@throws IOException only when data stream reading fails.
-    */
-    @Override
-    public void ingestDocumentWithException(String documentIdentifier,
-      String componentIdentifier,
-      String version, String documentURI, RepositoryDocument data)
-      throws ManifoldCFException, ServiceInterruption, IOException
-    {
-      // MHL
-    }
-
     /** 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 string to be recorded for the document.
     */
+    @Override
     public void noDocument(String documentIdentifier, String version)
       throws ManifoldCFException, ServiceInterruption
     {
-      // Special interpretation for empty version string; treat as if the document doesn't exist
-      // (by ignoring it and allowing it to be deleted later)
-      String documentIdentifierHash = ManifoldCF.hash(documentIdentifier);
-      ingester.documentNoData(
-        fetchPipelineSpecifications.get(documentIdentifierHash),
-        connectionName,documentIdentifierHash,
-        version,parameterVersion,
-        connection.getACLAuthority(),
-        currentTime,
-        ingestLogger);
-      
-      touchedSet.add(documentIdentifier);
+      noDocument(documentIdentifier,null,version);
     }
 
     /** Remove the specified document from the search engine index, and update the
@@ -1587,7 +1586,23 @@ public class WorkerThread extends Thread
       String version)
       throws ManifoldCFException, ServiceInterruption
     {
-      // MHL
+      // Special interpretation for empty version string; treat as if the document doesn't exist
+      // (by ignoring it and allowing it to be deleted later)
+      String documentIdentifierHash = ManifoldCF.hash(documentIdentifier);
+      String componentIdentifierHash;
+      if (componentIdentifier != null)
+        componentIdentifierHash = ManifoldCF.hash(componentIdentifier);
+      else
+        componentIdentifierHash = null;
+      ingester.documentNoData(
+        fetchPipelineSpecifications.get(documentIdentifierHash),
+        connectionName,documentIdentifierHash,componentIdentifierHash,
+        version,parameterVersion,
+        connection.getACLAuthority(),
+        currentTime,
+        ingestLogger);
+      
+      touchedSet.add(documentIdentifier);
     }
 
     /** Remove the specified document component permanently from the search engine index, and from the status table.
@@ -1602,15 +1617,7 @@ public class WorkerThread extends Thread
     public void removeDocument(String documentIdentifier)
       throws ManifoldCFException, ServiceInterruption
     {
-      // Remove from incremental ingester ONLY.
-      String documentIdentifierHash = ManifoldCF.hash(documentIdentifier);
-      ingester.documentDelete(
-        pipelineSpecification.getBasicPipelineSpecification(),
-        connectionName,documentIdentifierHash,
-        ingestLogger);
-      
-      // Note that we touched it, so it won't get checked
-      touchedSet.add(documentIdentifier);
+      removeDocument(documentIdentifier,null);
     }
 
     /** Remove the specified document component permanently from the search engine index, and from the status table.
@@ -1626,7 +1633,21 @@ public class WorkerThread extends Thread
       String componentIdentifier)
       throws ManifoldCFException, ServiceInterruption
     {
-      // MHL
+      // Remove from incremental ingester ONLY.
+      String documentIdentifierHash = ManifoldCF.hash(documentIdentifier);
+      String componentIdentifierHash;
+      if (componentIdentifier != null)
+        componentIdentifierHash = ManifoldCF.hash(componentIdentifier);
+      else
+        componentIdentifierHash = null;
+
+      ingester.documentRemove(
+        pipelineSpecification.getBasicPipelineSpecification(),
+        connectionName,documentIdentifierHash,componentIdentifierHash,
+        ingestLogger);
+      
+      // Note that we touched it, so it won't get checked
+      touchedSet.add(documentIdentifier);
     }