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/12/10 14:10:50 UTC

svn commit: r1644390 - in /manifoldcf/branches/CONNECTORS-1118/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/...

Author: kwright
Date: Wed Dec 10 13:10:50 2014
New Revision: 1644390

URL: http://svn.apache.org/r1644390
Log:
Convert deletion methods to also cut back on loading of instances

Modified:
    manifoldcf/branches/CONNECTORS-1118/framework/agents/src/main/java/org/apache/manifoldcf/agents/incrementalingest/IncrementalIngester.java
    manifoldcf/branches/CONNECTORS-1118/framework/agents/src/main/java/org/apache/manifoldcf/agents/interfaces/IIncrementalIngester.java
    manifoldcf/branches/CONNECTORS-1118/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/DocumentCleanupThread.java
    manifoldcf/branches/CONNECTORS-1118/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/DocumentDeleteThread.java
    manifoldcf/branches/CONNECTORS-1118/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/ExpireThread.java

Modified: manifoldcf/branches/CONNECTORS-1118/framework/agents/src/main/java/org/apache/manifoldcf/agents/incrementalingest/IncrementalIngester.java
URL: http://svn.apache.org/viewvc/manifoldcf/branches/CONNECTORS-1118/framework/agents/src/main/java/org/apache/manifoldcf/agents/incrementalingest/IncrementalIngester.java?rev=1644390&r1=1644389&r2=1644390&view=diff
==============================================================================
--- manifoldcf/branches/CONNECTORS-1118/framework/agents/src/main/java/org/apache/manifoldcf/agents/incrementalingest/IncrementalIngester.java (original)
+++ manifoldcf/branches/CONNECTORS-1118/framework/agents/src/main/java/org/apache/manifoldcf/agents/incrementalingest/IncrementalIngester.java Wed Dec 10 13:10:50 2014
@@ -765,7 +765,7 @@ public class IncrementalIngester extends
   }
 
   /** Remove a document component from the search engine index.
-  *@param pipelineSpecificationBasic is the basic pipeline specification.
+  *@param pipelineConnections is the 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.
@@ -773,12 +773,12 @@ public class IncrementalIngester extends
   */
   @Override
   public void documentRemove(
-    IPipelineSpecificationBasic pipelineSpecificationBasic,
+    IPipelineConnections pipelineConnections,
     String identifierClass, String identifierHash, String componentHash,
     IOutputRemoveActivity activities)
     throws ManifoldCFException, ServiceInterruption
   {
-    documentRemoveMultiple(pipelineSpecificationBasic,
+    documentRemoveMultiple(pipelineConnections,
       new String[]{identifierClass},
       new String[]{identifierHash},
       componentHash,
@@ -921,24 +921,24 @@ public class IncrementalIngester extends
 
 
   /** Delete multiple documents from the search engine index.
-  *@param pipelineSpecificationBasics are the pipeline specifications associated with the documents.
+  *@param pipelineConnections 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.
   *@param activities is the object to use to log the details of the ingestion attempt.  May be null.
   */
   @Override
   public void documentDeleteMultiple(
-    IPipelineSpecificationBasic[] pipelineSpecificationBasics,
+    IPipelineConnections[] pipelineConnections,
     String[] identifierClasses, String[] identifierHashes,
     IOutputRemoveActivity activities)
     throws ManifoldCFException, ServiceInterruption
   {
     // Segregate request by pipeline spec instance address.  Not perfect but works in the
     // environment it is used it.
-    Map<IPipelineSpecificationBasic,List<Integer>> keyMap = new HashMap<IPipelineSpecificationBasic,List<Integer>>();
-    for (int i = 0; i < pipelineSpecificationBasics.length; i++)
+    Map<IPipelineConnections,List<Integer>> keyMap = new HashMap<IPipelineConnections,List<Integer>>();
+    for (int i = 0; i < pipelineConnections.length; i++)
     {
-      IPipelineSpecificationBasic spec = pipelineSpecificationBasics[i];
+      IPipelineConnections spec = pipelineConnections[i];
       List<Integer> list = keyMap.get(spec);
       if (list == null)
       {
@@ -949,10 +949,10 @@ public class IncrementalIngester extends
     }
 
     // Create the return array.
-    Iterator<IPipelineSpecificationBasic> iter = keyMap.keySet().iterator();
+    Iterator<IPipelineConnections> iter = keyMap.keySet().iterator();
     while (iter.hasNext())
     {
-      IPipelineSpecificationBasic spec = iter.next();
+      IPipelineConnections spec = iter.next();
       List<Integer> list = keyMap.get(spec);
       String[] localIdentifierClasses = new String[list.size()];
       String[] localIdentifierHashes = new String[list.size()];
@@ -967,21 +967,20 @@ public class IncrementalIngester extends
   }
 
   /** Delete multiple documents from the search engine index.
-  *@param pipelineSpecificationBasic is the basic pipeline specification.
+  *@param pipelineConnections is the 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.
   *@param activities is the object to use to log the details of the ingestion attempt.  May be null.
   */
   @Override
   public void documentDeleteMultiple(
-    IPipelineSpecificationBasic pipelineSpecificationBasic,
+    IPipelineConnections pipelineConnections,
     String[] identifierClasses, String[] identifierHashes,
     IOutputRemoveActivity activities)
     throws ManifoldCFException, ServiceInterruption
   {
-    String[] outputConnectionNames = extractOutputConnectionNames(pipelineSpecificationBasic);
-    // Load connection managers up front to save time
-    IOutputConnection[] outputConnections = connectionManager.loadMultiple(outputConnectionNames);
+    String[] outputConnectionNames = pipelineConnections.getOutputConnectionNames();
+    IOutputConnection[] outputConnections = pipelineConnections.getOutputConnections();
     
     // No transactions here, so we can cycle through the connection names one at a time
     for (int z = 0; z < outputConnectionNames.length; z++)
@@ -1182,7 +1181,7 @@ public class IncrementalIngester extends
   }
 
   /** Remove multiple document components from the search engine index.
-  *@param pipelineSpecificationBasic is the basic pipeline specification.
+  *@param pipelineConnections is the pipeline specification.
   *@param identifierClasses are the names of the spaces in which the identifier hash should be interpreted.
   *@param identifierHashes are the hashes of the ids of the documents.
   *@param componentHash is the hashed component identifier, if any.
@@ -1190,14 +1189,13 @@ public class IncrementalIngester extends
   */
   @Override
   public void documentRemoveMultiple(
-    IPipelineSpecificationBasic pipelineSpecificationBasic,
+    IPipelineConnections pipelineConnections,
     String[] identifierClasses, String[] identifierHashes, String componentHash,
     IOutputRemoveActivity activities)
     throws ManifoldCFException, ServiceInterruption
   {
-    String[] outputConnectionNames = extractOutputConnectionNames(pipelineSpecificationBasic);
-    // Load connection managers up front to save time
-    IOutputConnection[] outputConnections = connectionManager.loadMultiple(outputConnectionNames);
+    String[] outputConnectionNames = pipelineConnections.getOutputConnectionNames();
+    IOutputConnection[] outputConnections = pipelineConnections.getOutputConnections();
     
     // No transactions here, so we can cycle through the connection names one at a time
     for (int z = 0; z < outputConnectionNames.length; z++)
@@ -1545,19 +1543,19 @@ public class IncrementalIngester extends
   }
 
   /** Delete a document from the search engine index.
-  *@param pipelineSpecificationBasic is the basic pipeline specification.
+  *@param pipelineConnections is the 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 activities is the object to use to log the details of the ingestion attempt.  May be null.
   */
   @Override
   public void documentDelete(
-    IPipelineSpecificationBasic pipelineSpecificationBasic,
+    IPipelineConnections pipelineConnections,
     String identifierClass, String identifierHash,
     IOutputRemoveActivity activities)
     throws ManifoldCFException, ServiceInterruption
   {
-    documentDeleteMultiple(pipelineSpecificationBasic,new String[]{identifierClass},new String[]{identifierHash},activities);
+    documentDeleteMultiple(pipelineConnections,new String[]{identifierClass},new String[]{identifierHash},activities);
   }
 
   /** Find out what URIs a SET of document URIs are currently ingested.

Modified: manifoldcf/branches/CONNECTORS-1118/framework/agents/src/main/java/org/apache/manifoldcf/agents/interfaces/IIncrementalIngester.java
URL: http://svn.apache.org/viewvc/manifoldcf/branches/CONNECTORS-1118/framework/agents/src/main/java/org/apache/manifoldcf/agents/interfaces/IIncrementalIngester.java?rev=1644390&r1=1644389&r2=1644390&view=diff
==============================================================================
--- manifoldcf/branches/CONNECTORS-1118/framework/agents/src/main/java/org/apache/manifoldcf/agents/interfaces/IIncrementalIngester.java (original)
+++ manifoldcf/branches/CONNECTORS-1118/framework/agents/src/main/java/org/apache/manifoldcf/agents/interfaces/IIncrementalIngester.java Wed Dec 10 13:10:50 2014
@@ -233,27 +233,27 @@ public interface IIncrementalIngester
     throws ManifoldCFException, ServiceInterruption, IOException;
 
   /** Remove a document component from the search engine index.
-  *@param pipelineSpecificationBasic is the basic pipeline specification.
+  *@param pipelineConnections is the 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,
+    IPipelineConnections pipelineConnections,
     String identifierClass, String identifierHash, String componentHash,
     IOutputRemoveActivity activities)
     throws ManifoldCFException, ServiceInterruption;
 
   /** Remove multiple document components from the search engine index.
-  *@param pipelineSpecificationBasic is the basic pipeline specification.
+  *@param pipelineConnections is the pipeline specification.
   *@param identifierClasses are the names of the spaces in which the identifier hash should be interpreted.
   *@param identifierHashes are the hashes of the ids of the documents.
   *@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 documentRemoveMultiple(
-    IPipelineSpecificationBasic pipelineSpecificationBasic,
+    IPipelineConnections pipelineConnections,
     String[] identifierClasses, String[] identifierHashes, String componentHash,
     IOutputRemoveActivity activities)
     throws ManifoldCFException, ServiceInterruption;
@@ -285,37 +285,37 @@ public interface IIncrementalIngester
     throws ManifoldCFException;
 
   /** Delete multiple documents, and their components, from the search engine index.
-  *@param pipelineSpecificationBasics are the pipeline specifications associated with the documents.
+  *@param pipelineConnections 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.
   *@param activities is the object to use to log the details of the ingestion attempt.  May be null.
   */
   public void documentDeleteMultiple(
-    IPipelineSpecificationBasic[] pipelineSpecificationBasics,
+    IPipelineConnections[] pipelineConnections,
     String[] identifierClasses, String[] identifierHashes,
     IOutputRemoveActivity activities)
     throws ManifoldCFException, ServiceInterruption;
 
   /** Delete multiple documents, and their components, from the search engine index.
-  *@param pipelineSpecificationBasic is the basic pipeline specification.
+  *@param pipelineConnections is the 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.
   *@param activities is the object to use to log the details of the ingestion attempt.  May be null.
   */
   public void documentDeleteMultiple(
-    IPipelineSpecificationBasic pipelineSpecificationBasic,
+    IPipelineConnections pipelineConnections,
     String[] identifierClasses, String[] identifierHashes,
     IOutputRemoveActivity activities)
     throws ManifoldCFException, ServiceInterruption;
 
   /** Delete a document, and all its components, from the search engine index.
-  *@param pipelineSpecificationBasic is the basic pipeline specification.
+  *@param pipelineConnections is the 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 activities is the object to use to log the details of the ingestion attempt.  May be null.
   */
   public void documentDelete(
-    IPipelineSpecificationBasic pipelineSpecificationBasic,
+    IPipelineConnections pipelineConnections,
     String identifierClass, String identifierHash,
     IOutputRemoveActivity activities)
     throws ManifoldCFException, ServiceInterruption;

Modified: manifoldcf/branches/CONNECTORS-1118/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/DocumentCleanupThread.java
URL: http://svn.apache.org/viewvc/manifoldcf/branches/CONNECTORS-1118/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/DocumentCleanupThread.java?rev=1644390&r1=1644389&r2=1644390&view=diff
==============================================================================
--- manifoldcf/branches/CONNECTORS-1118/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/DocumentCleanupThread.java (original)
+++ manifoldcf/branches/CONNECTORS-1118/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/DocumentCleanupThread.java Wed Dec 10 13:10:50 2014
@@ -80,6 +80,9 @@ public class DocumentCleanupThread exten
       IIncrementalIngester ingester = IncrementalIngesterFactory.make(threadContext);
       IJobManager jobManager = JobManagerFactory.make(threadContext);
       IRepositoryConnectionManager connMgr = RepositoryConnectionManagerFactory.make(threadContext);
+      ITransformationConnectionManager transformationConnectionManager = TransformationConnectionManagerFactory.make(threadContext);
+      IOutputConnectionManager outputConnectionManager = OutputConnectionManagerFactory.make(threadContext);
+      
       IReprioritizationTracker rt = ReprioritizationTrackerFactory.make(threadContext);
 
       IRepositoryConnectorPool repositoryConnectorPool = RepositoryConnectorPoolFactory.make(threadContext);
@@ -110,7 +113,7 @@ public class DocumentCleanupThread exten
 
           IJobDescription job = dds.getJobDescription();
           String connectionName = job.getConnectionName();
-          IPipelineSpecificationBasic pipelineSpecificationBasic = new PipelineSpecificationBasic(job);
+          IPipelineConnections pipelineConnections = new PipelineConnections(new PipelineSpecificationBasic(job),transformationConnectionManager,outputConnectionManager);
 
           try
           {
@@ -186,7 +189,7 @@ public class DocumentCleanupThread exten
 
               try
               {
-                ingester.documentDeleteMultiple(pipelineSpecificationBasic,docClassesToRemove,hashedDocsToRemove,activities);
+                ingester.documentDeleteMultiple(pipelineConnections,docClassesToRemove,hashedDocsToRemove,activities);
                 // Success!  Label all these as needing deletion from queue.
                 for (int k = 0; k < arrayDocHashes.size(); k++)
                 {

Modified: manifoldcf/branches/CONNECTORS-1118/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/DocumentDeleteThread.java
URL: http://svn.apache.org/viewvc/manifoldcf/branches/CONNECTORS-1118/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/DocumentDeleteThread.java?rev=1644390&r1=1644389&r2=1644390&view=diff
==============================================================================
--- manifoldcf/branches/CONNECTORS-1118/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/DocumentDeleteThread.java (original)
+++ manifoldcf/branches/CONNECTORS-1118/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/DocumentDeleteThread.java Wed Dec 10 13:10:50 2014
@@ -79,7 +79,9 @@ public class DocumentDeleteThread extend
       IJobManager jobManager = JobManagerFactory.make(threadContext);
       IIncrementalIngester ingester = IncrementalIngesterFactory.make(threadContext);
       IRepositoryConnectionManager connMgr = RepositoryConnectionManagerFactory.make(threadContext);
-
+      ITransformationConnectionManager transformationConnectionManager = TransformationConnectionManagerFactory.make(threadContext);
+      IOutputConnectionManager outputConnectionManager = OutputConnectionManagerFactory.make(threadContext);
+      
       // Loop
       while (true)
       {
@@ -100,7 +102,7 @@ public class DocumentDeleteThread extend
           
           IJobDescription job = dds.getJobDescription();
           String connectionName = job.getConnectionName();
-          IPipelineSpecificationBasic pipelineSpecificationBasic = new PipelineSpecificationBasic(job);
+          IPipelineConnections pipelineConnections = new PipelineConnections(new PipelineSpecificationBasic(job),transformationConnectionManager,outputConnectionManager);
           
           try
           {
@@ -129,7 +131,7 @@ public class DocumentDeleteThread extend
                 
             try
             {
-              ingester.documentDeleteMultiple(pipelineSpecificationBasic,docClassesToRemove,hashedDocsToRemove,logger);
+              ingester.documentDeleteMultiple(pipelineConnections,docClassesToRemove,hashedDocsToRemove,logger);
               for (int j = 0; j < dds.getCount(); j++)
               {
                 deleteFromQueue[j] = true;

Modified: manifoldcf/branches/CONNECTORS-1118/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/ExpireThread.java
URL: http://svn.apache.org/viewvc/manifoldcf/branches/CONNECTORS-1118/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/ExpireThread.java?rev=1644390&r1=1644389&r2=1644390&view=diff
==============================================================================
--- manifoldcf/branches/CONNECTORS-1118/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/ExpireThread.java (original)
+++ manifoldcf/branches/CONNECTORS-1118/framework/pull-agent/src/main/java/org/apache/manifoldcf/crawler/system/ExpireThread.java Wed Dec 10 13:10:50 2014
@@ -70,6 +70,8 @@ public class ExpireThread extends Thread
       IIncrementalIngester ingester = IncrementalIngesterFactory.make(threadContext);
       IJobManager jobManager = JobManagerFactory.make(threadContext);
       IRepositoryConnectionManager connMgr = RepositoryConnectionManagerFactory.make(threadContext);
+      ITransformationConnectionManager transformationConnectionManager = TransformationConnectionManagerFactory.make(threadContext);
+      IOutputConnectionManager outputConnectionManager = OutputConnectionManagerFactory.make(threadContext);
       IReprioritizationTracker rt = ReprioritizationTrackerFactory.make(threadContext);
 
       IRepositoryConnectorPool repositoryConnectorPool = RepositoryConnectorPoolFactory.make(threadContext);
@@ -100,7 +102,7 @@ public class ExpireThread extends Thread
 
           IJobDescription job = dds.getJobDescription();
           String connectionName = job.getConnectionName();
-          IPipelineSpecificationBasic pipelineSpecificationBasic = new PipelineSpecificationBasic(job);
+          IPipelineConnections pipelineConnections = new PipelineConnections(new PipelineSpecificationBasic(job),transformationConnectionManager,outputConnectionManager);
           
           try
           {
@@ -180,7 +182,7 @@ public class ExpireThread extends Thread
               // If we fail, we need to put the documents back on the queue.
               try
               {
-                ingester.documentDeleteMultiple(pipelineSpecificationBasic,docClassesToRemove,hashedDocsToRemove,activities);
+                ingester.documentDeleteMultiple(pipelineConnections,docClassesToRemove,hashedDocsToRemove,activities);
                 // Success!  Label all these as needing deletion from queue.
                 for (int k = 0; k < arrayDocHashes.size(); k++)
                 {