You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@continuum.apache.org by oc...@apache.org on 2008/12/04 11:32:05 UTC

svn commit: r723279 - in /continuum/branches/continuum-parallel-builds: continuum-api/src/main/java/org/apache/continuum/buildmanager/ continuum-api/src/main/java/org/apache/continuum/configuration/ continuum-api/src/main/java/org/apache/continuum/task...

Author: oching
Date: Thu Dec  4 02:32:04 2008
New Revision: 723279

URL: http://svn.apache.org/viewvc?rev=723279&view=rev
Log:
- added builds manager
- updated "overall" build queue

Added:
    continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/buildmanager/
    continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/buildmanager/BuildManager.java
    continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/buildmanager/BuildManagerException.java
    continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/taskqueue/OverallBuildQueue.java
    continuum/branches/continuum-parallel-builds/continuum-core/src/main/java/org/apache/continuum/buildmanager/
    continuum/branches/continuum-parallel-builds/continuum-core/src/main/java/org/apache/continuum/buildmanager/DefaultBuildManager.java
    continuum/branches/continuum-parallel-builds/continuum-core/src/main/java/org/apache/continuum/taskqueue/DefaultOverallBuildQueue.java
    continuum/branches/continuum-parallel-builds/continuum-core/src/test/java/org/apache/continuum/buildmanager/
    continuum/branches/continuum-parallel-builds/continuum-core/src/test/java/org/apache/continuum/buildmanager/DefaultBuildManagerTest.java
    continuum/branches/continuum-parallel-builds/continuum-core/src/test/java/org/apache/continuum/taskqueue/DefaultOverallBuildQueueTest.java
Removed:
    continuum/branches/continuum-parallel-builds/continuum-core/src/main/java/org/apache/continuum/taskqueue/DefaultOverallQueue.java
    continuum/branches/continuum-parallel-builds/continuum-core/src/main/java/org/apache/continuum/taskqueue/OverallQueue.java
    continuum/branches/continuum-parallel-builds/continuum-core/src/test/java/org/apache/continuum/taskqueue/DefaultOverallQueueTest.java
Modified:
    continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/configuration/GeneralConfiguration.java
    continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/taskqueue/manager/TaskQueueManager.java
    continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/maven/continuum/configuration/ConfigurationService.java
    continuum/branches/continuum-parallel-builds/continuum-base/continuum-configuration/pom.xml
    continuum/branches/continuum-parallel-builds/continuum-base/continuum-configuration/src/main/mdo/continuum-configuration.xml
    continuum/branches/continuum-parallel-builds/continuum-commons/src/main/java/org/apache/maven/continuum/configuration/DefaultConfigurationService.java
    continuum/branches/continuum-parallel-builds/continuum-core/src/main/java/org/apache/continuum/taskqueue/manager/DefaultTaskQueueManager.java
    continuum/branches/continuum-parallel-builds/continuum-core/src/main/resources/META-INF/plexus/components.xml
    continuum/branches/continuum-parallel-builds/continuum-test/src/main/java/org/apache/maven/continuum/configuration/ConfigurationServiceMock.java

Added: continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/buildmanager/BuildManager.java
URL: http://svn.apache.org/viewvc/continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/buildmanager/BuildManager.java?rev=723279&view=auto
==============================================================================
--- continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/buildmanager/BuildManager.java (added)
+++ continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/buildmanager/BuildManager.java Thu Dec  4 02:32:04 2008
@@ -0,0 +1,98 @@
+package org.apache.continuum.buildmanager;
+
+/*
+ * 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.
+ */
+
+import java.io.File;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.continuum.taskqueue.OverallBuildQueue;
+import org.apache.maven.continuum.model.project.BuildDefinition;
+
+public interface BuildManager
+{    
+    // TODO: 
+    // - move all creation of tasks inside the build manager!
+    // - in the implementation, make sure that the number of overall build queue instances are
+    //      read from the config file (continuum.xml)!
+    
+    OverallBuildQueue getOverallBuildQueueWhereProjectIsQueued( int project )
+        throws BuildManagerException;
+    
+    List<OverallBuildQueue> getOverallBuildQueuesInUse();
+    
+    /* prepare-build-projects queue */
+        
+    void addProjectToPrepareBuildQueue( Map<Integer, Integer> projectsBuildDefinitionsMap, int trigger )
+        throws BuildManagerException;
+
+    /**
+     * Add projects to prepare-build-projects queue. 
+     * Used when build is triggered at the group level. 
+     * 
+     * @param projectsBuildDefinitions
+     * @param trigger
+     * @throws BuildManagerException
+     */
+    void addProjectsToPrepareBuildQueue( Collection<Map<Integer, Integer>> projectsBuildDefinitions, int trigger )
+        throws BuildManagerException;
+
+    /* checkout queue */
+    
+    void addProjectToCheckoutQueue( int id, File workingDirectory, String projectName, String projectScmUsername,
+                                    String projectScmPassword )
+        throws BuildManagerException;
+
+    //void addProjectsToCheckoutQueue()
+    //    throws BuildManagerException;
+
+    void cancelProjectCheckout( int projectId )
+        throws BuildManagerException;
+    
+    void cancelAllCheckouts()
+        throws BuildManagerException;
+    
+    void removeProjectFromCheckoutQueue( int projectId )
+        throws BuildManagerException;
+
+    void removeProjectsFromCheckoutQueue( int[] projectIds )
+        throws BuildManagerException;
+    
+    /* build queue */
+    
+    void addProjectToBuildQueue( int projectId, BuildDefinition buildDefinition, int trigger, String projectName, String buildDefLabel )
+        throws BuildManagerException;
+
+    //void addProjectsToBuildQueue()
+    //    throws BuildManagerException;
+
+    void cancelProjectBuild( int projectId )
+        throws BuildManagerException;
+    
+    void cancelAllBuilds()
+        throws BuildManagerException;
+    
+    void removeProjectFromBuildQueue( int projectId, int buildDefinitionId, int trigger, String projectName )
+        throws BuildManagerException;
+
+    void removeProjectsFromBuildQueue( int[] projectIds )
+        throws BuildManagerException;
+}

Added: continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/buildmanager/BuildManagerException.java
URL: http://svn.apache.org/viewvc/continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/buildmanager/BuildManagerException.java?rev=723279&view=auto
==============================================================================
--- continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/buildmanager/BuildManagerException.java (added)
+++ continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/buildmanager/BuildManagerException.java Thu Dec  4 02:32:04 2008
@@ -0,0 +1,43 @@
+package org.apache.continuum.buildmanager;
+
+/*
+ * 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.
+ */
+
+/**
+ * @author <a href="mailto:oching@apache.org">Maria Odea Ching</a>
+ * @version
+ */
+public class BuildManagerException
+    extends Exception
+{
+    public BuildManagerException( String message )
+    {
+        super( message );
+    }
+
+    public BuildManagerException( Throwable cause )
+    {
+        super( cause );
+    }
+
+    public BuildManagerException( String message, Throwable cause )
+    {
+        super( message, cause );
+    }
+}

Modified: continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/configuration/GeneralConfiguration.java
URL: http://svn.apache.org/viewvc/continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/configuration/GeneralConfiguration.java?rev=723279&r1=723278&r2=723279&view=diff
==============================================================================
--- continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/configuration/GeneralConfiguration.java (original)
+++ continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/configuration/GeneralConfiguration.java Thu Dec  4 02:32:04 2008
@@ -41,6 +41,8 @@
     private ProxyConfiguration proxyConfiguration;
     
     private File releaseOutputDirectory;
+    
+    private int numberOfBuildsInParallel;
 
     public GeneralConfiguration()
     {
@@ -113,4 +115,13 @@
         this.releaseOutputDirectory = releaseOutputDirectory;
     }
 
+    public int getNumberOfBuildsInParallel()
+    {
+        return numberOfBuildsInParallel;
+    }
+
+    public void setNumberOfBuildsInParallel( int numberOfBuildsInParallel )
+    {
+        this.numberOfBuildsInParallel = numberOfBuildsInParallel;
+    }
 }

Added: continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/taskqueue/OverallBuildQueue.java
URL: http://svn.apache.org/viewvc/continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/taskqueue/OverallBuildQueue.java?rev=723279&view=auto
==============================================================================
--- continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/taskqueue/OverallBuildQueue.java (added)
+++ continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/taskqueue/OverallBuildQueue.java Thu Dec  4 02:32:04 2008
@@ -0,0 +1,315 @@
+package org.apache.continuum.taskqueue;
+
+/*
+ * 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.
+ */
+
+import java.util.List;
+
+import org.apache.maven.continuum.buildqueue.BuildProjectTask;
+import org.codehaus.plexus.component.repository.exception.ComponentLookupException;
+import org.codehaus.plexus.taskqueue.Task;
+import org.codehaus.plexus.taskqueue.TaskQueue;
+import org.codehaus.plexus.taskqueue.TaskQueueException;
+import org.codehaus.plexus.taskqueue.execution.TaskQueueExecutor;
+
+public interface OverallBuildQueue
+{       
+    /**
+     * Returns the id of the "overall" build queue
+     * 
+     * @return
+     */
+    int getId();
+    
+    void setId( int id );
+    
+    /* Checkout Queue */
+    
+    /**
+     * Returns the checkout queue.
+     * 
+     * @return
+     */
+    TaskQueue getCheckoutQueue();
+
+    /**
+     * Add checkout task to checkout queue.
+     * 
+     * @param checkoutTask
+     * @throws TaskQueueException TODO
+     */
+    void addToCheckoutQueue( Task checkoutTask )
+        throws TaskQueueException;
+    
+   /**
+    * Add checkout tasks to checkout queue.
+    * 
+    * @param checkoutTasks
+ * @throws TaskQueueException TODO
+    */
+    void addToCheckoutQueue( List<Task> checkoutTasks )
+        throws TaskQueueException;
+
+    /**
+     * Get all checkout tasks in checkout queue.
+     * 
+     * @return
+     * @throws TaskQueueException TODO
+     */
+    List /* CheckOutTask */getCheckOutTasksInQueue()
+        throws TaskQueueException;
+
+    /**
+     * Check if the project is in the checkout queue.
+     * 
+     * @param projectId
+     * @return
+     * @throws TaskQueueException TODO
+     */
+    boolean isInCheckoutQueue( int projectId )
+        throws TaskQueueException;
+
+    /**
+     * Remove project from checkout queue.
+     * 
+     * @param projectId
+     * @return
+     * @throws TaskQueueException TODO
+     */
+    boolean removeProjectFromCheckoutQueue( int projectId )
+        throws TaskQueueException;
+
+    /**
+     * Remove the specified projects in the checkout queue.
+     * 
+     * @param projectId
+     * @return
+     * @throws TaskQueueException TODO
+     */
+    boolean removeProjectsFromCheckoutQueue( int[] projectId )
+        throws TaskQueueException;
+
+    /**
+     * 
+     * @param hashCodes
+     * @throws TaskQueueException TODO
+     */
+    void removeTasksFromCheckoutQueueWithHashCodes( int[] hashCodes )
+        throws TaskQueueException;
+
+    /* Prepare Build Projects Queue */
+    
+    /**
+     * Returns the prepare-build-projects queue.
+     * 
+     * @return
+     */
+    TaskQueue getPrepareBuildQueue();
+
+    /**
+     * Returns the task queue executor of the prepare-build-projects queue.
+     * 
+     * @return
+     * @throws ComponentLookupException TODO
+     */
+    TaskQueueExecutor getPrepareBuildTaskQueueExecutor()
+        throws ComponentLookupException;
+
+    /**
+     * Add prepare build task to prepare-build-project queue.
+     * 
+     * @param prepareBuildTask
+     * @throws TaskQueueException TODO
+     */
+    void addToPrepareBuildQueue( Task prepareBuildTask )
+        throws TaskQueueException;
+    
+    /**
+     * Add prepare build tasks to prepare-build-project queue
+     * 
+     * @param prepareBuildTasks
+     * @throws TaskQueueException TODO
+     */
+    void addToPrepareBuildQueue( List<Task> prepareBuildTasks )
+        throws TaskQueueException;
+
+    /**
+     * Checks if the project is in the prepare-build-projects queue.
+     * 
+     * @param projectId
+     * @return
+     * @throws TaskQueueException TODO
+     */
+    boolean isInPrepareBuildQueue( int projectId )
+        throws TaskQueueException;
+
+    /**
+     * Checks if the current prepare build task being executed is the specified project.
+     * 
+     * @param projectId
+     * @return
+     * @throws TaskQueueException TODO
+     */
+    boolean isCurrentPrepareBuildTaskInExecution( int projectId )
+        throws TaskQueueException;
+    
+    /* Build Queue */
+
+    /**
+     * Returns the build queue.
+     * 
+     * @return
+     */
+    TaskQueue getBuildQueue();
+
+    /**
+     * Returns the task queue executor for the build queue.
+     * 
+     * @return
+     * @throws ComponentLookupException TODO
+     */
+    TaskQueueExecutor getBuildTaskQueueExecutor()
+        throws ComponentLookupException;
+
+    /**
+     * Add the build task to the build queue.
+     * 
+     * @param buildTask
+     * @throws Exception
+     */
+    void addToBuildQueue( Task buildTask )
+        throws TaskQueueException;
+    
+    /**
+     * Add the build tasks to the build queue.
+     * 
+     * @param buildTasks
+     * @throws TaskQueueException TODO
+     */
+    void addToBuildQueue( List<Task> buildTasks )
+        throws TaskQueueException;
+
+    /**
+     * Returns the project id of the project currently being built.
+     * 
+     * @return
+     * @throws TaskQueueException TODO
+     */
+    int getProjectIdInCurrentBuild()
+        throws TaskQueueException;
+
+    /**
+     * Returns the build tasks in the build queue.
+     * 
+     * @return
+     * @throws TaskQueueException TODO
+     */
+    List<BuildProjectTask> getProjectsInBuildQueue()
+        throws TaskQueueException;
+
+    /**
+     * Checks if there is a build in progress.
+     * 
+     * @return
+     * @throws TaskQueueException TODO
+     */
+    boolean isBuildInProgress()
+        throws TaskQueueException;
+
+    /**
+     * Checks if the specified project is in the build queue.
+     * 
+     * @param projectId
+     * @return
+     * @throws TaskQueueException TODO
+     */
+    boolean isInBuildQueue( int projectId )
+        throws TaskQueueException;
+
+    /**
+     * Checks if the specified project with the specified build definition is in the build queue.
+     * 
+     * @param projectId
+     * @param buildDefinitionId
+     * @return
+     * @throws TaskQueueException TODO
+     */
+    boolean isInBuildQueue( int projectId, int buildDefinitionId )
+        throws TaskQueueException;
+
+    /**
+     * Cancel the build task of the corresponding project.
+     * 
+     * @param projectId
+     * @throws TaskQueueException TODO
+     */
+    void cancelBuildTask( int projectId )
+        throws TaskQueueException;
+
+    /**
+     * Cancel the current build.
+     * 
+     * @return
+     * @throws TaskQueueException TODO
+     */
+    boolean cancelCurrentBuild()
+        throws TaskQueueException;
+
+    /**
+     * Remove the project matching the specified id, name, build definition and trigger from the build queue.
+     * 
+     * @param projectId
+     * @param buildDefinitionId
+     * @param trigger
+     * @param projectName
+     * @return
+     * @throws TaskQueueException TODO
+     */
+    boolean removeProjectFromBuildQueue( int projectId, int buildDefinitionId, int trigger, String projectName )
+        throws TaskQueueException;
+
+    /**
+     * Remove the specified project from the build queue.
+     * 
+     * @param projectId
+     * @return
+     * @throws TaskQueueException TODO
+     */
+    boolean removeProjectFromBuildQueue( int projectId )
+        throws TaskQueueException;
+
+    /**
+     * Remove the specified projects from the build queue.
+     * 
+     * @param projectIds
+     * @return
+     * @throws TaskQueueException TODO
+     */
+    boolean removeProjectsFromBuildQueue( int[] projectIds )
+        throws TaskQueueException;
+
+    /**
+     * Remove the projects matching the specified hashcodes from the build queue.
+     * 
+     * @param hashCodes
+     * @throws TaskQueueException TODO
+     */
+    void removeProjectsFromBuildQueueWithHashCodes( int[] hashCodes )
+        throws TaskQueueException;
+}

Modified: continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/taskqueue/manager/TaskQueueManager.java
URL: http://svn.apache.org/viewvc/continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/taskqueue/manager/TaskQueueManager.java?rev=723279&r1=723278&r2=723279&view=diff
==============================================================================
--- continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/taskqueue/manager/TaskQueueManager.java (original)
+++ continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/continuum/taskqueue/manager/TaskQueueManager.java Thu Dec  4 02:32:04 2008
@@ -10,7 +10,11 @@
  * @author <a href="mailto:ctan@apache.org">Maria Catherine Tan</a>
  */
 public interface TaskQueueManager
-{
+{    
+    //TODO: 
+    // - Remove all methods concerning the checkout, prepare build and build queues!!!
+    // - Point all objects using these methods to the "overall" build queue!    
+    
     String ROLE = TaskQueueManager.class.getName();
 
     boolean buildInProgress()

Modified: continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/maven/continuum/configuration/ConfigurationService.java
URL: http://svn.apache.org/viewvc/continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/maven/continuum/configuration/ConfigurationService.java?rev=723279&r1=723278&r2=723279&view=diff
==============================================================================
--- continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/maven/continuum/configuration/ConfigurationService.java (original)
+++ continuum/branches/continuum-parallel-builds/continuum-api/src/main/java/org/apache/maven/continuum/configuration/ConfigurationService.java Thu Dec  4 02:32:04 2008
@@ -83,6 +83,10 @@
 
     String getReleaseOutput( int projectGroupId, String releaseName )
         throws ConfigurationException;
+    
+    int getNumberOfBuildsInParallel();
+    
+    void setNumberOfBuildsInParallel( int num );
 
     // ----------------------------------------------------------------------
     //

Modified: continuum/branches/continuum-parallel-builds/continuum-base/continuum-configuration/pom.xml
URL: http://svn.apache.org/viewvc/continuum/branches/continuum-parallel-builds/continuum-base/continuum-configuration/pom.xml?rev=723279&r1=723278&r2=723279&view=diff
==============================================================================
--- continuum/branches/continuum-parallel-builds/continuum-base/continuum-configuration/pom.xml (original)
+++ continuum/branches/continuum-parallel-builds/continuum-base/continuum-configuration/pom.xml Thu Dec  4 02:32:04 2008
@@ -35,7 +35,7 @@
           <models>
             <model>src/main/mdo/continuum-configuration.xml</model>
           </models>
-          <version>1.0.0</version>
+          <version>1.1.0</version>
         </configuration>
         <executions>
           <execution>

Modified: continuum/branches/continuum-parallel-builds/continuum-base/continuum-configuration/src/main/mdo/continuum-configuration.xml
URL: http://svn.apache.org/viewvc/continuum/branches/continuum-parallel-builds/continuum-base/continuum-configuration/src/main/mdo/continuum-configuration.xml?rev=723279&r1=723278&r2=723279&view=diff
==============================================================================
--- continuum/branches/continuum-parallel-builds/continuum-base/continuum-configuration/src/main/mdo/continuum-configuration.xml (original)
+++ continuum/branches/continuum-parallel-builds/continuum-base/continuum-configuration/src/main/mdo/continuum-configuration.xml Thu Dec  4 02:32:04 2008
@@ -31,31 +31,31 @@
   <classes>
     <class rootElement="true" xml.tagName="continuum-configuration">
       <name>ContinuumConfigurationModel</name>
-      <version>1.0.0+</version>
+      <version>1.1.0+</version>
       <fields>
         <field>
           <name>baseUrl</name>
-          <version>1.0.0+</version>
+          <version>1.1.0+</version>
           <type>String</type>
         </field>
         <field>
           <name>buildOutputDirectory</name>
-          <version>1.0.0+</version>
+          <version>1.1.0+</version>
           <type>String</type>
         </field>
         <field>
           <name>deploymentRepositoryDirectory</name>
-          <version>1.0.0+</version>
+          <version>1.1.0+</version>
           <type>String</type>
         </field>   
         <field>
           <name>workingDirectory</name>
-          <version>1.0.0+</version>
+          <version>1.1.0+</version>
           <type>String</type>
         </field>
         <field>
           <name>proxyConfiguration</name>
-          <version>1.0.0+</version>
+          <version>1.1.0+</version>
           <association>
             <type>ProxyConfiguration</type>
             <multiplicity>1</multiplicity>
@@ -63,33 +63,38 @@
         </field>     
         <field>
           <name>releaseOutputDirectory</name>
-          <version>1.0.0+</version>
+          <version>1.1.0+</version>
           <type>String</type>
-        </field>                
+        </field>    
+        <field>
+          <name>numberOfBuildsInParallel</name>
+          <version>1.1.0+</version>
+          <type>int</type>
+        </field>            
       </fields>
     </class>
     <class xml.tagName="proxy-configuration">
       <name>ProxyConfiguration</name>
-      <version>1.0.0+</version>
+      <version>1.1.0+</version>
       <fields>
         <field>
           <name>proxyHost</name>
-          <version>1.0.0+</version>
+          <version>1.1.0+</version>
           <type>String</type>
         </field>
         <field>
           <name>proxyPort</name>
-          <version>1.0.0+</version>
+          <version>1.1.0+</version>
           <type>int</type>
         </field>
         <field>
           <name>proxyUser</name>
-          <version>1.0.0+</version>
+          <version>1.1.0+</version>
           <type>String</type>
         </field>   
         <field>
           <name>proxyPassword</name>
-          <version>1.0.0+</version>
+          <version>1.1.0+</version>
           <type>String</type>
         </field>                     
       </fields>

Modified: continuum/branches/continuum-parallel-builds/continuum-commons/src/main/java/org/apache/maven/continuum/configuration/DefaultConfigurationService.java
URL: http://svn.apache.org/viewvc/continuum/branches/continuum-parallel-builds/continuum-commons/src/main/java/org/apache/maven/continuum/configuration/DefaultConfigurationService.java?rev=723279&r1=723278&r2=723279&view=diff
==============================================================================
--- continuum/branches/continuum-parallel-builds/continuum-commons/src/main/java/org/apache/maven/continuum/configuration/DefaultConfigurationService.java (original)
+++ continuum/branches/continuum-parallel-builds/continuum-commons/src/main/java/org/apache/maven/continuum/configuration/DefaultConfigurationService.java Thu Dec  4 02:32:04 2008
@@ -380,6 +380,16 @@
             return null;
         }
     }
+    
+    public int getNumberOfBuildsInParallel()
+    {
+        return generalConfiguration.getNumberOfBuildsInParallel();
+    }
+    
+    public void setNumberOfBuildsInParallel( int num )
+    {
+        generalConfiguration.setNumberOfBuildsInParallel( num );
+    }
 
     // ----------------------------------------------------------------------
     //

Added: continuum/branches/continuum-parallel-builds/continuum-core/src/main/java/org/apache/continuum/buildmanager/DefaultBuildManager.java
URL: http://svn.apache.org/viewvc/continuum/branches/continuum-parallel-builds/continuum-core/src/main/java/org/apache/continuum/buildmanager/DefaultBuildManager.java?rev=723279&view=auto
==============================================================================
--- continuum/branches/continuum-parallel-builds/continuum-core/src/main/java/org/apache/continuum/buildmanager/DefaultBuildManager.java (added)
+++ continuum/branches/continuum-parallel-builds/continuum-core/src/main/java/org/apache/continuum/buildmanager/DefaultBuildManager.java Thu Dec  4 02:32:04 2008
@@ -0,0 +1,246 @@
+package org.apache.continuum.buildmanager;
+
+/*
+ * 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.
+ */
+
+import java.io.File;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.continuum.taskqueue.OverallBuildQueue;
+import org.apache.maven.continuum.buildqueue.BuildProjectTask;
+import org.apache.maven.continuum.configuration.ConfigurationService;
+import org.apache.maven.continuum.model.project.BuildDefinition;
+import org.codehaus.plexus.PlexusConstants;
+import org.codehaus.plexus.PlexusContainer;
+import org.codehaus.plexus.component.repository.exception.ComponentLookupException;
+import org.codehaus.plexus.context.Context;
+import org.codehaus.plexus.context.ContextException;
+import org.codehaus.plexus.logging.AbstractLogEnabled;
+import org.codehaus.plexus.personality.plexus.lifecycle.phase.Contextualizable;
+import org.codehaus.plexus.taskqueue.TaskQueueException;
+
+/**
+ * DefaultBuildManager
+ * 
+ * Manages the overall build queues and designates to which "overall" build queue a task will be 
+ * queued. When parallel builds are enabled, a number of "overall" build queues (depending on what is
+ * set in the configuration) will be utilized for the builds. Otherwise, only one "overall" build 
+ * queue will be utilized.
+ * 
+ * @author <a href="mailto:oching@apache.org">Maria Odea Ching</a>
+ * @plexus.component role="org.apache.continuum.buildmanager.BuildManager"
+ */
+public class DefaultBuildManager
+    extends AbstractLogEnabled
+    implements BuildManager, Contextualizable
+{   
+    // TODO: the number of "overall" build queues should be dynamic & sensitive to the
+    //          changes in configuration!
+    
+    private List<OverallBuildQueue> overallBuildQueuesInUse;
+    
+    /**
+     * @plexus.requirement
+     */
+    private ConfigurationService configurationService;
+    
+    private PlexusContainer container;
+    
+    public DefaultBuildManager()
+    {
+        overallBuildQueuesInUse = new ArrayList<OverallBuildQueue>();        
+    }
+
+    // should be invoked before adding anything in any of the queues!
+    private synchronized OverallBuildQueue getOverallBuildQueue()
+        throws ComponentLookupException
+    {
+        OverallBuildQueue overallBuildQueue = null;
+                
+        int parallelBuildsNum = configurationService.getNumberOfBuildsInParallel();
+        if( overallBuildQueuesInUse.size() < parallelBuildsNum )
+        {   
+            overallBuildQueue = ( OverallBuildQueue ) container.lookup( OverallBuildQueue.class );            
+            overallBuildQueue.setId( overallBuildQueuesInUse.size() + 1 );
+            
+            overallBuildQueuesInUse.add( overallBuildQueue );            
+        }       
+        else
+        {
+            // add logic for distributing work among the queues            
+            // - check whether the project is already in any of the queues
+            // - if not, always get the overall build queue with the least tasks queued to it
+        }
+        
+        return overallBuildQueue;
+    }
+    
+    public OverallBuildQueue getOverallBuildQueueWhereProjectIsQueued( int projectId )
+        throws BuildManagerException
+    {
+        try
+        {
+            for( OverallBuildQueue overallBuildQueueInUse : overallBuildQueuesInUse )
+            {                   
+                if( overallBuildQueueInUse.isInBuildQueue( projectId ) )
+                {   
+                    return overallBuildQueueInUse;                    
+                }
+            }
+        }
+        catch ( TaskQueueException e )
+        {
+            throw new BuildManagerException( "Error occurred while retrieving project in build queue: " +
+                e.getMessage() );
+        }
+        
+        return null;
+    }
+    
+    public List<OverallBuildQueue> getOverallBuildQueuesInUse()
+    {
+        return overallBuildQueuesInUse;
+    }
+    
+    public void addProjectToBuildQueue( int projectId, BuildDefinition buildDefinition, int trigger, String projectName,
+                                        String buildDefLabel )
+        throws BuildManagerException
+    {   
+        if( getOverallBuildQueueWhereProjectIsQueued( projectId ) == null )
+        {
+            BuildProjectTask task =
+                new BuildProjectTask( projectId, buildDefinition.getId(), trigger, projectName, buildDefLabel );
+            
+            task.setMaxExecutionTime( buildDefinition.getSchedule().getMaxJobExecutionTime() * 1000 );
+         
+            try
+            {
+                OverallBuildQueue overallBuildQueue = getOverallBuildQueue();
+                overallBuildQueue.addToBuildQueue( task );
+            }
+            catch ( ComponentLookupException e )
+            {
+                throw new BuildManagerException( e.getMessage() );
+            }        
+            catch ( TaskQueueException e )
+            {
+                throw new BuildManagerException( e.getMessage() );
+            }
+        }
+        else
+        {
+            getLogger().warn( "Project '" + projectName + "' is already queued." );
+        }
+    }
+
+    public void addProjectToCheckoutQueue( int id, File workingDirectory, String projectName,
+                                           String projectScmUsername, String projectScmPassword )
+        throws BuildManagerException
+    {
+        
+    }
+
+    public void addProjectToPrepareBuildQueue( Map<Integer, Integer> projectsBuildDefinitionsMap, int trigger )
+        throws BuildManagerException
+    {
+        
+    }
+
+    public void addProjectsToPrepareBuildQueue( Collection<Map<Integer, Integer>> projectsBuildDefinitions, int trigger )
+        throws BuildManagerException
+    {
+        // TODO Auto-generated method stub
+
+    }
+
+    public void cancelAllBuilds()
+        throws BuildManagerException
+    {
+        // TODO Auto-generated method stub
+
+    }
+
+    public void cancelAllCheckouts()
+        throws BuildManagerException
+    {
+        // TODO Auto-generated method stub
+
+    }
+
+    public void cancelProjectBuild( int projectId )
+        throws BuildManagerException
+    {
+        // TODO Auto-generated method stub
+
+    }
+
+    public void cancelProjectCheckout( int projectId )
+        throws BuildManagerException
+    {
+        // TODO Auto-generated method stub
+
+    }
+
+    public void removeProjectFromBuildQueue( int projectId, int buildDefinitionId, int trigger, String projectName )
+        throws BuildManagerException
+    {
+        // TODO Auto-generated method stub
+
+    }
+
+    public void removeProjectFromCheckoutQueue( int projectId )
+        throws BuildManagerException
+    {
+        // TODO Auto-generated method stub
+
+    }
+
+    public void removeProjectsFromBuildQueue( int[] projectIds )
+        throws BuildManagerException
+    {
+        // TODO Auto-generated method stub
+
+    }
+
+    public void removeProjectsFromCheckoutQueue( int[] projectIds )
+        throws BuildManagerException
+    {
+        // TODO Auto-generated method stub
+
+    }
+
+    public void contextualize( Context context )
+        throws ContextException
+    {
+        container = (PlexusContainer) context.get( PlexusConstants.PLEXUS_KEY );
+    }
+    
+    public void setContainer( PlexusContainer container )
+    {
+        this.container = container;
+    }
+    
+    public void setConfigurationService( ConfigurationService configurationService )
+    {
+        this.configurationService = configurationService;
+    }
+}

Added: continuum/branches/continuum-parallel-builds/continuum-core/src/main/java/org/apache/continuum/taskqueue/DefaultOverallBuildQueue.java
URL: http://svn.apache.org/viewvc/continuum/branches/continuum-parallel-builds/continuum-core/src/main/java/org/apache/continuum/taskqueue/DefaultOverallBuildQueue.java?rev=723279&view=auto
==============================================================================
--- continuum/branches/continuum-parallel-builds/continuum-core/src/main/java/org/apache/continuum/taskqueue/DefaultOverallBuildQueue.java (added)
+++ continuum/branches/continuum-parallel-builds/continuum-core/src/main/java/org/apache/continuum/taskqueue/DefaultOverallBuildQueue.java Thu Dec  4 02:32:04 2008
@@ -0,0 +1,571 @@
+package org.apache.continuum.taskqueue;
+
+/*
+ * 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.
+ */
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import org.apache.commons.lang.ArrayUtils;
+import org.apache.continuum.dao.BuildDefinitionDao;
+import org.apache.continuum.taskqueue.manager.TaskQueueManagerException;
+import org.apache.maven.continuum.buildqueue.BuildProjectTask;
+import org.apache.maven.continuum.model.project.BuildDefinition;
+import org.apache.maven.continuum.scm.queue.CheckOutTask;
+import org.apache.maven.continuum.scm.queue.PrepareBuildProjectsTask;
+import org.apache.maven.continuum.store.ContinuumStoreException;
+import org.codehaus.plexus.PlexusConstants;
+import org.codehaus.plexus.PlexusContainer;
+import org.codehaus.plexus.component.repository.exception.ComponentLookupException;
+import org.codehaus.plexus.context.Context;
+import org.codehaus.plexus.context.ContextException;
+import org.codehaus.plexus.logging.AbstractLogEnabled;
+import org.codehaus.plexus.personality.plexus.lifecycle.phase.Contextualizable;
+import org.codehaus.plexus.taskqueue.Task;
+import org.codehaus.plexus.taskqueue.TaskQueue;
+import org.codehaus.plexus.taskqueue.TaskQueueException;
+import org.codehaus.plexus.taskqueue.execution.TaskQueueExecutor;
+import org.codehaus.plexus.util.StringUtils;
+
+/**
+ * "Overall" build queue which has a checkout queue, a prepare-build queue, and a build queue. All builds whether forced
+ * or triggered will go through (or have to be added through) the "overall" build queue.
+ * 
+ * @author <a href="mailto:oching@apache.org">Maria Odea Ching</a>
+ * @plexus.component role="org.apache.continuum.taskqueue.OverallBuildQueue" instantiation-strategy="per-lookup"
+ */
+public class DefaultOverallBuildQueue
+    extends AbstractLogEnabled
+    implements OverallBuildQueue, Contextualizable
+{
+    // TODO:
+    // - need to specify each task queue to be instantiated each time it is looked up!!!
+
+    /**
+     * @plexus.requirement role-hint="build-project"
+     */
+    private TaskQueue buildQueue;
+
+    /**
+     * @plexus.requirement role-hint="check-out-project"
+     */
+    private TaskQueue checkoutQueue;
+
+    /**
+     * @plexus.requirement role-hint="prepare-build-project"
+     */
+    private TaskQueue prepareBuildQueue;
+
+    /**
+     * @plexus.requirement
+     */
+    private BuildDefinitionDao buildDefinitionDao;
+
+    private PlexusContainer container;
+    
+    private int id;
+    
+    public int getId()
+    {
+        return id;
+    }
+
+    public void setId( int id )
+    {
+        this.id = id;
+    }
+
+    /* Checkout Queue */
+
+    public TaskQueueExecutor getCheckoutTaskQueueExecutor()
+        throws TaskQueueManagerException
+    {
+        try
+        {
+            return (TaskQueueExecutor) container.lookup( TaskQueueExecutor.class, "check-out-project" );
+        }
+        catch ( ComponentLookupException e )
+        {
+            throw new TaskQueueManagerException( e.getMessage(), e );
+        }
+    }
+
+    public void addToCheckoutQueue( Task checkoutTask )
+        throws TaskQueueException
+    {
+        checkoutQueue.put( checkoutTask );
+    }
+
+    public void addToCheckoutQueue( List<Task> checkoutTasks )
+        throws TaskQueueException
+    {
+        for ( Task checkoutTask : checkoutTasks )
+        {
+            checkoutQueue.put( checkoutTask );
+        }
+    }
+
+    public List<CheckOutTask> getCheckOutTasksInQueue()
+        throws TaskQueueException
+    {
+        return checkoutQueue.getQueueSnapshot();       
+    }
+
+    public boolean isInCheckoutQueue( int projectId )
+        throws TaskQueueException
+    {
+        List<CheckOutTask> queue = getCheckOutTasksInQueue();
+
+        for ( CheckOutTask task : queue )
+        {
+            if ( task != null && task.getProjectId() == projectId )
+            {
+                return true;
+            }
+        }
+
+        return false;
+    }
+
+    public boolean removeProjectFromCheckoutQueue( int projectId )
+        throws TaskQueueException
+    {
+        List<CheckOutTask> queue = getCheckOutTasksInQueue();
+
+        for ( CheckOutTask task : queue )
+        {
+            if ( task != null && task.getProjectId() == projectId )
+            {
+                return checkoutQueue.remove( task );
+            }
+        }
+
+        return false;
+    }
+
+    public boolean removeProjectsFromCheckoutQueue( int[] projectsId )
+        throws TaskQueueException
+    {
+        if ( projectsId == null )
+        {
+            return false;
+        }
+        if ( projectsId.length < 1 )
+        {
+            return false;
+        }
+        List<CheckOutTask> queue = getCheckOutTasksInQueue();
+
+        List<CheckOutTask> tasks = new ArrayList<CheckOutTask>();
+
+        for ( CheckOutTask task : queue )
+        {
+            if ( task != null )
+            {
+                if ( ArrayUtils.contains( projectsId, task.getProjectId() ) )
+                {
+                    tasks.add( task );
+                }
+            }
+        }
+        if ( !tasks.isEmpty() )
+        {
+            return checkoutQueue.removeAll( tasks );
+        }
+        return false;
+    }
+
+    public void removeTasksFromCheckoutQueueWithHashCodes( int[] hashCodes )
+        throws TaskQueueException
+    {
+        List<CheckOutTask> queue = getCheckOutTasksInQueue();
+
+        for ( CheckOutTask task : queue )
+        {
+            if ( ArrayUtils.contains( hashCodes, task.hashCode() ) )
+            {
+                checkoutQueue.remove( task );
+            }
+        }
+    }
+
+    /* Prepare-build-projects Queue */
+
+    public TaskQueueExecutor getPrepareBuildTaskQueueExecutor()
+        throws ComponentLookupException
+    {        
+        return (TaskQueueExecutor) container.lookup( TaskQueueExecutor.class, "prepare-build-project" );        
+    }
+
+    public void addToPrepareBuildQueue( Task prepareBuildTask )
+        throws TaskQueueException
+    {
+        prepareBuildQueue.put( prepareBuildTask );
+    }
+
+    public void addToPrepareBuildQueue( List<Task> prepareBuildTasks )
+        throws TaskQueueException
+    {
+        for ( Task prepareBuildTask : prepareBuildTasks )
+        {
+            prepareBuildQueue.put( prepareBuildTask );
+        }
+    }
+
+    public boolean isInPrepareBuildQueue( int projectId )
+        throws TaskQueueException
+    {        
+        List<PrepareBuildProjectsTask> queue = prepareBuildQueue.getQueueSnapshot();
+
+        for ( PrepareBuildProjectsTask task : queue )
+        {
+            if ( task != null )
+            {
+                Map<Integer, Integer> map = ( (PrepareBuildProjectsTask) task ).getProjectsBuildDefinitionsMap();
+
+                if ( map.size() > 0 )
+                {
+                    Set<Integer> projectIds = map.keySet();
+
+                    if ( projectIds.contains( new Integer( projectId ) ) )
+                    {
+                        return true;
+                    }
+                }
+            }
+        }
+
+        return false;        
+    }
+
+    public boolean isCurrentPrepareBuildTaskInExecution( int projectId )
+        throws TaskQueueException
+    {
+        try
+        {
+            Task task = getPrepareBuildTaskQueueExecutor().getCurrentTask();
+    
+            if ( task != null && task instanceof PrepareBuildProjectsTask )
+            {
+                Map<Integer, Integer> map = ( (PrepareBuildProjectsTask) task ).getProjectsBuildDefinitionsMap();
+    
+                if ( map.size() > 0 )
+                {
+                    Set<Integer> projectIds = map.keySet();
+    
+                    if ( projectIds.contains( new Integer( projectId ) ) )
+                    {
+                        return true;
+                    }
+                }
+            }
+        }
+        catch ( ComponentLookupException e )
+        {
+            throw new TaskQueueException( "Error looking up prepare-build-project task queue executor." );
+        }
+        
+        return false;
+    }
+
+    /* Build Queue */
+
+    public TaskQueueExecutor getBuildTaskQueueExecutor()
+        throws ComponentLookupException
+    {        
+        return (TaskQueueExecutor) container.lookup( TaskQueueExecutor.class, "build-project" );        
+    }
+
+    public void addToBuildQueue( Task buildTask )
+        throws TaskQueueException
+    {
+        buildQueue.put( buildTask );
+    }
+
+    public void addToBuildQueue( List<Task> buildTasks )
+        throws TaskQueueException
+    {
+        for ( Task buildTask : buildTasks )
+        {
+            buildQueue.put( buildTask );
+        }
+    }
+
+    public int getProjectIdInCurrentBuild()
+        throws TaskQueueException
+    {
+        try
+        {
+            Task task = getBuildTaskQueueExecutor().getCurrentTask();
+            if ( task != null )
+            {
+                if ( task instanceof BuildProjectTask )
+                {
+                    return ( (BuildProjectTask) task ).getProjectId();
+                }
+            }
+        }
+        catch ( ComponentLookupException e )
+        {
+            throw new TaskQueueException( "Error occurred while looking up the build task queue executor. " );
+        }
+        
+        return -1;
+    }
+
+    public List<BuildProjectTask> getProjectsInBuildQueue()
+        throws TaskQueueException
+    {   
+        return buildQueue.getQueueSnapshot();        
+    }
+
+    public boolean isBuildInProgress()
+        throws TaskQueueException
+    {
+        try
+        {
+            Task task = getCurrentTask( "build-project" );
+    
+            if ( task != null && task instanceof BuildProjectTask )
+            {
+                return true;
+            }
+        }
+        catch( ComponentLookupException e )
+        {
+            // should we wrap this in a different exception instead of a TaskQueueException
+            throw new TaskQueueException( e.getMessage() );
+        }
+        
+        return false;
+    }
+
+    public boolean isInBuildQueue( int projectId )
+        throws TaskQueueException
+    {
+        return isInBuildQueue( projectId, -1 );
+    }
+
+    public boolean isInBuildQueue( int projectId, int buildDefinitionId )
+        throws TaskQueueException
+    {
+        List<BuildProjectTask> queue = getProjectsInBuildQueue();
+
+        for ( BuildProjectTask task : queue )
+        {
+            if ( task != null )
+            {
+                if ( buildDefinitionId < 0 )
+                {
+                    if ( task.getProjectId() == projectId )
+                    {
+                        return true;
+                    }
+                }
+                else
+                {
+                    if ( task.getProjectId() == projectId && task.getBuildDefinitionId() == buildDefinitionId )
+                    {
+                        return true;
+                    }
+                }
+            }
+        }
+
+        return false;
+    }
+
+    public void cancelBuildTask( int projectId )
+        throws TaskQueueException
+    {
+        try
+        {
+            Task currentTask = getBuildTaskQueueExecutor().getCurrentTask();
+    
+            if ( currentTask instanceof BuildProjectTask )
+            {
+                if ( ( (BuildProjectTask) currentTask ).getProjectId() == projectId )
+                {
+                    getLogger().info( "Cancelling task for project " + projectId );
+                    getBuildTaskQueueExecutor().cancelTask( currentTask );
+                }
+            }
+        }
+        catch ( ComponentLookupException e )
+        {
+            throw new TaskQueueException( e.getMessage() );
+        }
+    }
+
+    public boolean cancelCurrentBuild()
+        throws TaskQueueException
+    {
+        try
+        {
+            Task task = getBuildTaskQueueExecutor().getCurrentTask();
+    
+            if ( task != null )
+            {
+                if ( task instanceof BuildProjectTask )
+                {
+                    getLogger().info( "Cancelling current build task" );
+                    return getBuildTaskQueueExecutor().cancelTask( task );
+                }
+                else
+                {
+                    getLogger().warn( "Current task not a BuildProjectTask - not cancelling" );
+                }
+            }
+            else
+            {
+                getLogger().warn( "No task running - not cancelling" );
+            }
+        }
+        catch ( ComponentLookupException e )
+        {
+            throw new TaskQueueException( e.getMessage() );
+        }
+        
+        return false;
+    }
+
+    public boolean removeProjectFromBuildQueue( int projectId, int buildDefinitionId, int trigger, String projectName )
+        throws TaskQueueException
+    {
+        BuildDefinition buildDefinition;
+
+        try
+        {
+            buildDefinition = buildDefinitionDao.getBuildDefinition( buildDefinitionId );
+        }
+        catch ( ContinuumStoreException e )
+        {
+            throw new TaskQueueException( "Error while removing project from build queue: " + projectName, e );
+        }
+
+        String buildDefinitionLabel = buildDefinition.getDescription();
+        if ( StringUtils.isEmpty( buildDefinitionLabel ) )
+        {
+            buildDefinitionLabel = buildDefinition.getGoals();
+        }
+        
+        BuildProjectTask buildProjectTask =
+            new BuildProjectTask( projectId, buildDefinitionId, trigger, projectName, buildDefinitionLabel );
+        
+        return this.buildQueue.remove( buildProjectTask );
+    }
+
+    public boolean removeProjectsFromBuildQueue( int[] projectIds )
+        throws TaskQueueException
+    {
+        if ( projectIds == null )
+        {
+            return false;
+        }
+        if ( projectIds.length < 1 )
+        {
+            return false;
+        }
+        List<BuildProjectTask> queue = getProjectsInBuildQueue();
+
+        List<BuildProjectTask> tasks = new ArrayList<BuildProjectTask>();
+
+        for ( BuildProjectTask task : queue )
+        {
+            if ( task != null )
+            {
+                if ( ArrayUtils.contains( projectIds, task.getProjectId() ) )
+                {
+                    tasks.add( task );
+                }
+            }
+        }
+
+        for ( BuildProjectTask buildProjectTask : tasks )
+        {
+            getLogger().info( "cancel build for project " + buildProjectTask.getProjectId() );
+        }
+        if ( !tasks.isEmpty() )
+        {
+            return buildQueue.removeAll( tasks );
+        }
+
+        return false;
+    }
+
+    public boolean removeProjectFromBuildQueue( int projectId )
+        throws TaskQueueException
+    {
+        List<BuildProjectTask> queue = getProjectsInBuildQueue();
+
+        for ( BuildProjectTask task : queue )
+        {
+            if ( task != null && task.getProjectId() == projectId )
+            {
+                return buildQueue.remove( task );
+            }
+        }
+
+        return false;
+    }
+
+    public void removeProjectsFromBuildQueueWithHashCodes( int[] hashCodes )
+        throws TaskQueueException
+    {
+        List<BuildProjectTask> queue = getProjectsInBuildQueue();
+
+        for ( BuildProjectTask task : queue )
+        {
+            if ( ArrayUtils.contains( hashCodes, task.hashCode() ) )
+            {
+                buildQueue.remove( task );
+            }
+        }
+    }
+
+    public void contextualize( Context context )
+        throws ContextException
+    {
+        container = (PlexusContainer) context.get( PlexusConstants.PLEXUS_KEY );
+    }
+
+    public TaskQueue getCheckoutQueue()
+    {
+        return checkoutQueue;
+    }
+
+    public TaskQueue getPrepareBuildQueue()
+    {
+        return prepareBuildQueue;
+    }
+
+    public TaskQueue getBuildQueue()
+    {
+        return buildQueue;
+    }
+
+    private Task getCurrentTask( String task )
+        throws ComponentLookupException
+    {
+        TaskQueueExecutor executor = (TaskQueueExecutor) container.lookup( TaskQueueExecutor.class, task );
+        return executor.getCurrentTask();       
+    }    
+}

Modified: continuum/branches/continuum-parallel-builds/continuum-core/src/main/java/org/apache/continuum/taskqueue/manager/DefaultTaskQueueManager.java
URL: http://svn.apache.org/viewvc/continuum/branches/continuum-parallel-builds/continuum-core/src/main/java/org/apache/continuum/taskqueue/manager/DefaultTaskQueueManager.java?rev=723279&r1=723278&r2=723279&view=diff
==============================================================================
--- continuum/branches/continuum-parallel-builds/continuum-core/src/main/java/org/apache/continuum/taskqueue/manager/DefaultTaskQueueManager.java (original)
+++ continuum/branches/continuum-parallel-builds/continuum-core/src/main/java/org/apache/continuum/taskqueue/manager/DefaultTaskQueueManager.java Thu Dec  4 02:32:04 2008
@@ -34,8 +34,7 @@
 
 /**
  * @author <a href="mailto:ctan@apache.org">Maria Catherine Tan</a>
- * @plexus.component role="org.apache.continuum.taskqueue.manager.TaskQueueManager" role-hint="default" 
- *          instantiation-strategy="per-lookup"
+ * @plexus.component role="org.apache.continuum.taskqueue.manager.TaskQueueManager" role-hint="default"
  */
 public class DefaultTaskQueueManager
     extends AbstractLogEnabled

Modified: continuum/branches/continuum-parallel-builds/continuum-core/src/main/resources/META-INF/plexus/components.xml
URL: http://svn.apache.org/viewvc/continuum/branches/continuum-parallel-builds/continuum-core/src/main/resources/META-INF/plexus/components.xml?rev=723279&r1=723278&r2=723279&view=diff
==============================================================================
--- continuum/branches/continuum-parallel-builds/continuum-core/src/main/resources/META-INF/plexus/components.xml (original)
+++ continuum/branches/continuum-parallel-builds/continuum-core/src/main/resources/META-INF/plexus/components.xml Thu Dec  4 02:32:04 2008
@@ -30,6 +30,7 @@
       <role>org.codehaus.plexus.taskqueue.TaskQueue</role>
       <role-hint>build-project</role-hint>
       <implementation>org.codehaus.plexus.taskqueue.DefaultTaskQueue</implementation>
+      <instantiation-strategy>per-lookup</instantiation-strategy>
       <lifecycle-handler>plexus-configurable</lifecycle-handler>
       <configuration>
         <task-entry-evaluators>
@@ -45,6 +46,7 @@
     <component>
       <role>org.codehaus.plexus.taskqueue.TaskViabilityEvaluator</role>
       <role-hint>build-project</role-hint>
+      <instantiation-strategy>per-lookup</instantiation-strategy>
       <implementation>
         org.apache.maven.continuum.buildqueue.evaluator.BuildProjectTaskViabilityEvaluator</implementation>
       <configuration>

Added: continuum/branches/continuum-parallel-builds/continuum-core/src/test/java/org/apache/continuum/buildmanager/DefaultBuildManagerTest.java
URL: http://svn.apache.org/viewvc/continuum/branches/continuum-parallel-builds/continuum-core/src/test/java/org/apache/continuum/buildmanager/DefaultBuildManagerTest.java?rev=723279&view=auto
==============================================================================
--- continuum/branches/continuum-parallel-builds/continuum-core/src/test/java/org/apache/continuum/buildmanager/DefaultBuildManagerTest.java (added)
+++ continuum/branches/continuum-parallel-builds/continuum-core/src/test/java/org/apache/continuum/buildmanager/DefaultBuildManagerTest.java Thu Dec  4 02:32:04 2008
@@ -0,0 +1,157 @@
+package org.apache.continuum.buildmanager;
+
+/*
+ * 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.
+ */
+
+import org.apache.continuum.taskqueue.OverallBuildQueue;
+import org.apache.maven.continuum.configuration.ConfigurationService;
+import org.apache.maven.continuum.model.project.BuildDefinition;
+import org.apache.maven.continuum.model.project.Schedule;
+import org.codehaus.plexus.spring.PlexusInSpringTestCase;
+import org.jmock.Expectations;
+import org.jmock.Mockery;
+import org.jmock.integration.junit3.JUnit3Mockery;
+
+/**
+ * 
+ * @author <a href="mailto:oching@apache.org">Maria Odea Ching</a>
+ */
+public class DefaultBuildManagerTest
+    extends PlexusInSpringTestCase
+{
+    private DefaultBuildManager buildManager;
+    
+    private Mockery context;
+    
+    private ConfigurationService configurationService;
+    
+    public void setUp() 
+        throws Exception
+    {
+        super.setUp();
+        
+        context = new JUnit3Mockery();
+        
+        configurationService = context.mock( ConfigurationService.class );
+        
+        buildManager = ( DefaultBuildManager ) lookup( BuildManager.class );
+        
+        buildManager.setContainer( getContainer() ); 
+        
+        buildManager.setConfigurationService( configurationService );
+    }
+    
+    public void testParallelBuilds()
+        throws Exception
+    {   
+        // set expectations
+        context.checking(new Expectations() {{
+            exactly( 2 ).of(configurationService).getNumberOfBuildsInParallel(); will( returnValue( 2 ) );
+        }});
+        
+        Schedule schedule = new Schedule();
+        schedule.setMaxJobExecutionTime( 100 );
+        
+        BuildDefinition buildDef = new BuildDefinition();
+        buildDef.setId( 1 );
+        buildDef.setSchedule( schedule );
+        
+        buildManager.addProjectToBuildQueue( 1, buildDef, 1, "continuum-test-project-1", "build-def-label" );
+        
+        buildManager.addProjectToBuildQueue( 2, buildDef, 1, "continuum-test-project-2", "build-def-label" );
+        
+        // verify
+        context.assertIsSatisfied();
+        
+        // assert if the project is in the correct build queue!
+        OverallBuildQueue overallBuildQueue = buildManager.getOverallBuildQueueWhereProjectIsQueued( 1 );
+        assertEquals( 1, overallBuildQueue.getId() );
+        
+        overallBuildQueue = buildManager.getOverallBuildQueueWhereProjectIsQueued( 2 );
+        assertEquals( 2, overallBuildQueue.getId() );
+        
+        assertEquals( 2, buildManager.getOverallBuildQueuesInUse().size() );
+    }
+    
+    public void testParallelBuildsLimitIsMaximized()
+        throws Exception
+    {
+        // assert size of the overallBuildQueuesInUse!
+    }
+    
+    public void testParallelBuildsBuildIsCancelled()
+        throws Exception
+    {
+    
+    }
+    
+    public void testParallelBuildsProjectAlreadyInBuildQueue()
+        throws Exception
+    {
+    
+    }
+    
+    public void testParallelBuildsRemoveFromBuildQueue()
+        throws Exception
+    {
+    
+    }
+    
+    public void testParallelBuildsAddToCheckoutQueue()
+        throws Exception
+    {
+    
+    }
+        
+    public void testParallelBuildsAddToPrepareBuildQueue()
+        throws Exception
+    {
+    
+    }
+    
+    public void testParallelBuildsProjectAlreadyInCheckoutQueue()
+        throws Exception
+    {
+    
+    }
+    
+    public void testParallelBuildsProjectAlreadyInPrepareBuildQueue()
+        throws Exception
+    {
+    
+    }
+    
+    public void testParallelBuildsRemoveFromCheckoutQueue()
+        throws Exception
+    {
+    
+    }
+        
+    public void testParallelBuildsRemoveFromPrepareBuildQueue()
+        throws Exception
+    {
+    
+    }
+        
+    public void testSingleBuild()
+        throws Exception
+    {
+    
+    }
+}

Added: continuum/branches/continuum-parallel-builds/continuum-core/src/test/java/org/apache/continuum/taskqueue/DefaultOverallBuildQueueTest.java
URL: http://svn.apache.org/viewvc/continuum/branches/continuum-parallel-builds/continuum-core/src/test/java/org/apache/continuum/taskqueue/DefaultOverallBuildQueueTest.java?rev=723279&view=auto
==============================================================================
--- continuum/branches/continuum-parallel-builds/continuum-core/src/test/java/org/apache/continuum/taskqueue/DefaultOverallBuildQueueTest.java (added)
+++ continuum/branches/continuum-parallel-builds/continuum-core/src/test/java/org/apache/continuum/taskqueue/DefaultOverallBuildQueueTest.java Thu Dec  4 02:32:04 2008
@@ -0,0 +1,84 @@
+package org.apache.continuum.taskqueue;
+
+/*
+ * 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.
+ */
+
+import java.io.File;
+import java.util.HashMap;
+import java.util.Map;
+
+import org.apache.maven.continuum.AbstractContinuumTest;
+import org.apache.maven.continuum.buildqueue.BuildProjectTask;
+import org.apache.maven.continuum.scm.queue.CheckOutTask;
+import org.apache.maven.continuum.scm.queue.PrepareBuildProjectsTask;
+
+public class DefaultOverallBuildQueueTest
+    extends AbstractContinuumTest
+{    
+    private OverallBuildQueue overallQueue;
+    
+    @Override
+    protected void setUp()
+        throws Exception
+    {
+        super.setUp();
+        
+        overallQueue = ( OverallBuildQueue ) lookup( OverallBuildQueue.class );
+    }
+    
+    public void testAddToCheckoutQueue()
+        throws Exception
+    {
+        File workingDir = new File( getBasedir(), "target/working-dir" );
+        
+        CheckOutTask task = new CheckOutTask( 1, workingDir, "continuum-test-project", "username", "password" );
+        overallQueue.addToCheckoutQueue( task );
+        
+        CheckOutTask queuedTask = ( CheckOutTask ) overallQueue.getCheckoutQueue().take();
+        assertNotNull( queuedTask );
+        assertEquals( 1, queuedTask.getProjectId() );
+        assertEquals( "continuum-test-project", queuedTask.getProjectName() );
+    }
+    
+    public void testAddToBuildQueue()
+        throws Exception
+    {   
+        BuildProjectTask buildTask = new BuildProjectTask( 1, 1, 1, "continuum-test-project", "build-def-label" );
+        overallQueue.addToBuildQueue( buildTask );
+        
+        BuildProjectTask queuedTask = ( BuildProjectTask ) overallQueue.getBuildQueue().take();
+        assertNotNull( queuedTask );
+        assertEquals( 1, queuedTask.getProjectId() );
+        assertEquals( "continuum-test-project", queuedTask.getProjectName() );
+    }
+    
+    public void testAddToPrepareBuildQueue()
+        throws Exception
+    {
+        Map<Integer, Integer> projectsBuildDefMap = new HashMap<Integer, Integer>();
+        projectsBuildDefMap.put( new Integer( 1 ), new Integer( 1 ) ); 
+        
+        PrepareBuildProjectsTask prepareBuildTask = new PrepareBuildProjectsTask( projectsBuildDefMap, 1 );
+        overallQueue.addToPrepareBuildQueue( prepareBuildTask );
+        
+        PrepareBuildProjectsTask queuedTask = ( PrepareBuildProjectsTask ) overallQueue.getPrepareBuildQueue().take();
+        assertNotNull( queuedTask );
+        assertEquals( 1, ( ( Integer )queuedTask.getProjectsBuildDefinitionsMap().get( new Integer( 1 ) ) ).intValue() );        
+    }
+}

Modified: continuum/branches/continuum-parallel-builds/continuum-test/src/main/java/org/apache/maven/continuum/configuration/ConfigurationServiceMock.java
URL: http://svn.apache.org/viewvc/continuum/branches/continuum-parallel-builds/continuum-test/src/main/java/org/apache/maven/continuum/configuration/ConfigurationServiceMock.java?rev=723279&r1=723278&r2=723279&view=diff
==============================================================================
--- continuum/branches/continuum-parallel-builds/continuum-test/src/main/java/org/apache/maven/continuum/configuration/ConfigurationServiceMock.java (original)
+++ continuum/branches/continuum-parallel-builds/continuum-test/src/main/java/org/apache/maven/continuum/configuration/ConfigurationServiceMock.java Thu Dec  4 02:32:04 2008
@@ -240,4 +240,13 @@
         return null;
     }
 
+    public int getNumberOfBuildsInParallel()
+    {
+        return 1;
+    }
+    
+    public void setNumberOfBuildsInParallel( int num )
+    {
+        
+    }
 }