You are viewing a plain text version of this content. The canonical link for it is here.
Posted to scm@geronimo.apache.org by am...@apache.org on 2003/11/17 11:57:40 UTC

cvs commit: incubator-geronimo/modules/kernel/src/java/org/apache/geronimo/kernel/deployment/scanner DeploymentScanner.java

ammulder    2003/11/17 02:57:40

  Modified:    .        maven.xml
               modules/core/src/test/org/apache/geronimo/deployment/goal
                        DeployURLTest.java
               modules/kernel project.xml
               modules/kernel/src/conf boot.mlet
               modules/kernel/src/java/org/apache/geronimo/kernel Main.java
               modules/kernel/src/java/org/apache/geronimo/kernel/deployment
                        DeploymentController.java
               modules/kernel/src/java/org/apache/geronimo/kernel/deployment/goal
                        DeployURL.java DeploymentGoal.java RedeployURL.java
                        UndeployURL.java
               modules/kernel/src/java/org/apache/geronimo/kernel/deployment/scanner
                        DeploymentScanner.java
  Added:       modules/kernel/src/java/org/apache/geronimo/kernel/deployment
                        ApplicationDeployer.java GeronimoTargetModule.java
                        ServerTarget.java
               modules/kernel/src/java/org/apache/geronimo/kernel/deployment/client
                        DeploymentNotification.java
               modules/kernel/src/java/org/apache/geronimo/kernel/deployment/goal
                        DistributeURL.java
  Log:
  New JSR-88 server-side support
   - ApplicationDeployer is now a GeronimoMBean
   - ApplicationDeployer is now in kernel
   - kernel now depends on JSR-88 (which is moved to bootlib)
   - All deployment goes through JSR-88 ApplicationDeployer (including
       deployment of services)
   - Every deployment job gets an ID and status notifications
   - Deployment jobs can be queued and are handled in the order received
   - Deployment goals are associated with JSR-88 TargetModuleIDs
  
  More work left (though everything deploys and runs as before)
   - command-line deployer was broken in the last week or so (classpath)
   - JSR-88 client has to be hooked up to revised server
   - Need to figure out how to associate modules with ModuleTypes (perhaps
       include ModuleType in TargetModuleID)
   - Need to figure out how to get a TargetModuleID from a deployment URL
       and/or ObjectName (include it in JSR-77 MBean?)
  
  Revision  Changes    Path
  1.44      +12 -1     incubator-geronimo/maven.xml
  
  Index: maven.xml
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/maven.xml,v
  retrieving revision 1.43
  retrieving revision 1.44
  diff -u -r1.43 -r1.44
  --- maven.xml	16 Nov 2003 05:39:54 -0000	1.43
  +++ maven.xml	17 Nov 2003 10:57:39 -0000	1.44
  @@ -389,6 +389,17 @@
               include="*.jar"/>
       </goal>
   
  +    <goal name="modules:aggregate:geronimo-spec:geronimo-spec-j2ee-deployment">
  +        <!-- Do not do the default libraries stuff -->
  +        <aggregate:default libraries="false"/>
  +
  +        <!-- Put j2ee-deployment into deploy/bootlib -->
  +        <aggregate:copy
  +            source="${module.root}/target"
  +            target="${aggregate.dir}/bootlib"
  +            include="*.jar"/>
  +    </goal>
  +
       <goal name="modules:aggregate:geronimo:geronimo-web-console">
           <!-- Do not do the default libraries stuff -->
           <aggregate:default libraries="false"/>
  
  
  
  1.4       +4 -2      incubator-geronimo/modules/core/src/test/org/apache/geronimo/deployment/goal/DeployURLTest.java
  
  Index: DeployURLTest.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/core/src/test/org/apache/geronimo/deployment/goal/DeployURLTest.java,v
  retrieving revision 1.3
  retrieving revision 1.4
  diff -u -r1.3 -r1.4
  --- DeployURLTest.java	8 Sep 2003 04:35:12 -0000	1.3
  +++ DeployURLTest.java	17 Nov 2003 10:57:40 -0000	1.4
  @@ -62,6 +62,8 @@
   
   import org.apache.geronimo.kernel.deployment.scanner.URLType;
   import org.apache.geronimo.kernel.deployment.goal.DeployURL;
  +import org.apache.geronimo.kernel.deployment.GeronimoTargetModule;
  +import org.apache.geronimo.kernel.deployment.ServerTarget;
   
   import junit.framework.TestCase;
   
  @@ -75,7 +77,7 @@
   {
       public void testFileUrlWithSpace() {
           try {
  -            DeployURL goal = new DeployURL(new URL("file:///C:/Program Files/Apache Group/Geronimo"), URLType.UNPACKED_ARCHIVE);
  +            DeployURL goal = new DeployURL(new GeronimoTargetModule(new ServerTarget("localhost"), "test"), new URL("file:///C:/Program Files/Apache Group/Geronimo"), URLType.UNPACKED_ARCHIVE);
               URL url = goal.getUrl();
               URI baseURI = URI.create(url.toString()).normalize();;        
           } catch (MalformedURLException e){
  
  
  
  1.10      +10 -1     incubator-geronimo/modules/kernel/project.xml
  
  Index: project.xml
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/kernel/project.xml,v
  retrieving revision 1.9
  retrieving revision 1.10
  diff -u -r1.9 -r1.10
  --- project.xml	11 Nov 2003 23:53:32 -0000	1.9
  +++ project.xml	17 Nov 2003 10:57:40 -0000	1.10
  @@ -80,6 +80,15 @@
               </properties>
           </dependency>
   
  +        <dependency>
  +            <groupId>geronimo-spec</groupId>
  +            <artifactId>geronimo-spec-j2ee-deployment</artifactId>
  +            <version>DEV</version>
  +            <properties>
  +                <module>true</module>
  +            </properties>
  +        </dependency>
  +
           <!-- Thirdparty -->
   
           <dependency>
  
  
  
  1.4       +7 -0      incubator-geronimo/modules/kernel/src/conf/boot.mlet
  
  Index: boot.mlet
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/kernel/src/conf/boot.mlet,v
  retrieving revision 1.3
  retrieving revision 1.4
  diff -u -r1.3 -r1.4
  --- boot.mlet	14 Nov 2003 16:27:34 -0000	1.3
  +++ boot.mlet	17 Nov 2003 10:57:40 -0000	1.4
  @@ -25,6 +25,13 @@
   </MLET>
   
   <MLET CODE="org.apache.geronimo.kernel.service.GeronimoMBean"
  +    NAME="geronimo.deployment:role=ApplicationDeployer"
  +    ARCHIVE=""
  +    >
  +    <ARG type="java.lang.String" value="org.apache.geronimo.kernel.deployment.ApplicationDeployer"/>
  +</MLET>
  +
  +<MLET CODE="org.apache.geronimo.kernel.service.GeronimoMBean"
         NAME="geronimo.deployment:role=DeploymentPlanner,type=Service"
         ARCHIVE=""
         >
  
  
  
  1.2       +14 -5     incubator-geronimo/modules/kernel/src/java/org/apache/geronimo/kernel/Main.java
  
  Index: Main.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/kernel/src/java/org/apache/geronimo/kernel/Main.java,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- Main.java	8 Sep 2003 04:38:33 -0000	1.1
  +++ Main.java	17 Nov 2003 10:57:40 -0000	1.2
  @@ -109,7 +109,7 @@
       private final URL mletURL;
       private final URL bootURL;
   
  -    private ObjectName controllerName;
  +    private ObjectName deployerName;
   
       public Main(String domainName, URL mletURL, URL bootURL) {
           this.domainName = domainName;
  @@ -126,6 +126,7 @@
           Object[] deployArgs = {bootURL};
           JMXKernel kernel = null;
           ShutdownThread hook = new ShutdownThread("Shutdown-Thread", Thread.currentThread());
  +        ObjectName controllerName = null;
           try {
               Runtime.getRuntime().addShutdownHook(hook);
               try {
  @@ -137,6 +138,7 @@
   
                   // check they all started OK and it included a controller and service planner
                   ObjectName controllerPattern = new ObjectName("*:role=DeploymentController,*");
  +                ObjectName deployerPattern = new ObjectName("*:role=ApplicationDeployer,*");
                   ObjectName plannerPattern = new ObjectName("*:role=DeploymentPlanner,type=Service,*");
                   boolean planner = false;
                   for (Iterator i = bootedMBeans.iterator(); i.hasNext();) {
  @@ -152,10 +154,17 @@
                           controllerName = mletName;
                       } else if (plannerPattern.apply(mletName)) {
                           planner = true;
  +                    } else if(deployerPattern.apply(mletName)) {
  +                        if(deployerName != null) {
  +                            throw new DeploymentException("Multiple ApplicationDeployers specified in boot mlet");
  +                        }
  +                        deployerName = mletName;
                       }
                   }
                   if (controllerName == null) {
                       throw new DeploymentException("Boot mlet did not load a DeploymentController");
  +                } else if (deployerName == null) {
  +                        throw new DeploymentException("Boot mlet did not load an ApplicationDeployer");
                   } else if (!planner) {
                       throw new DeploymentException("Boot mlet did not load a DeploymentPlanner for type=Service");
                   }
  @@ -163,7 +172,7 @@
                   // start her up
                   log.info("Deploying Bootstrap Services from " + bootURL);
                   MBeanServer mbServer = kernel.getMBeanServer();
  -                mbServer.invoke(controllerName, "deploy", deployArgs, DEPLOY_ARG_TYPES);
  +                mbServer.invoke(deployerName, "deploy", deployArgs, DEPLOY_ARG_TYPES);
   
                   // Booted... print the startup time
                   time = (System.currentTimeMillis() - time) / 1000;
  @@ -197,11 +206,11 @@
                   // we were in the process of shutting down - ignore
               }
               if (kernel != null) {
  -                if (controllerName != null) {
  +                if (deployerName != null) {
                       try {
                           log.info("Undeploy Bootstrap Services");
                           MBeanServer mbServer = kernel.getMBeanServer();
  -                        mbServer.invoke(controllerName, "undeploy", deployArgs, DEPLOY_ARG_TYPES);
  +                        mbServer.invoke(deployerName, "undeploy", deployArgs, DEPLOY_ARG_TYPES);
                       } catch (Throwable e) {
                           log.error("Error stopping Server", e);
                       }
  
  
  
  1.6       +163 -109  incubator-geronimo/modules/kernel/src/java/org/apache/geronimo/kernel/deployment/DeploymentController.java
  
  Index: DeploymentController.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/kernel/src/java/org/apache/geronimo/kernel/deployment/DeploymentController.java,v
  retrieving revision 1.5
  retrieving revision 1.6
  diff -u -r1.5 -r1.6
  --- DeploymentController.java	17 Nov 2003 00:17:25 -0000	1.5
  +++ DeploymentController.java	17 Nov 2003 10:57:40 -0000	1.6
  @@ -55,29 +55,24 @@
    */
   package org.apache.geronimo.kernel.deployment;
   
  -import java.io.IOException;
  -import java.net.URL;
   import java.util.ArrayList;
  +import java.util.Arrays;
   import java.util.Collection;
  +import java.util.Collections;
   import java.util.HashMap;
   import java.util.HashSet;
   import java.util.Iterator;
   import java.util.LinkedHashSet;
  +import java.util.LinkedList;
   import java.util.List;
   import java.util.Map;
   import java.util.Set;
  -import java.util.Collections;
  -
  -import javax.management.MalformedObjectNameException;
  +import javax.enterprise.deploy.spi.TargetModuleID;
   import javax.management.ObjectName;
  -
   import org.apache.commons.logging.Log;
   import org.apache.commons.logging.LogFactory;
  -import org.apache.geronimo.kernel.deployment.goal.DeployURL;
  -import org.apache.geronimo.kernel.deployment.goal.RedeployURL;
  -import org.apache.geronimo.kernel.deployment.goal.UndeployURL;
  -import org.apache.geronimo.kernel.deployment.scanner.URLInfo;
  -import org.apache.geronimo.kernel.deployment.scanner.URLType;
  +import org.apache.geronimo.kernel.deployment.client.DeploymentNotification;
  +import org.apache.geronimo.kernel.deployment.goal.DeploymentGoal;
   import org.apache.geronimo.kernel.service.GeronimoMBeanContext;
   import org.apache.geronimo.kernel.service.GeronimoMBeanEndpoint;
   import org.apache.geronimo.kernel.service.GeronimoMBeanInfo;
  @@ -86,7 +81,8 @@
   import org.apache.geronimo.kernel.service.GeronimoParameterInfo;
   
   /**
  - *
  + * Handles the nuts & bolts of deployment -- acting on a set of goals, and
  + * providing status notifications along the way.
    *
    * @version $Revision$ $Date$
    */
  @@ -96,153 +92,129 @@
   
       private GeronimoMBeanContext context;
       private Collection planners = Collections.EMPTY_LIST;
  -    private final Map scanResults = new HashMap();
       private final Set goals = new HashSet();
       private final LinkedHashSet plans = new LinkedHashSet();
  +    private final DeploymentWaiter waiter = new DeploymentWaiter();
  +    private final DeploymentIDGenerator ids = new DeploymentIDGenerator();
  +    private long notificationSequence = 0;
  +    private Object notificationLock = new Object();
   
   
       public static GeronimoMBeanInfo getGeronimoMBeanInfo() throws Exception {
           GeronimoMBeanInfo mbeanInfo = new GeronimoMBeanInfo();
           mbeanInfo.setAutostart(true);
           mbeanInfo.setTargetClass(DeploymentController.class.getName());
  -        mbeanInfo.addOperationInfo(new GeronimoOperationInfo("planDeployment",
  +        mbeanInfo.addOperationInfo(new GeronimoOperationInfo("runDeploymentJob",
                   new GeronimoParameterInfo[] {
  -                    new GeronimoParameterInfo("Source", ObjectName.class.getName(), "Good question!"),
  -                    new GeronimoParameterInfo("URLInfos", Set.class.getName(), "Set of URLs to plan deployments for")
  +                    new GeronimoParameterInfo("goals", "[L"+DeploymentGoal.class.getName()+";", "Goals to achieve as part of this job")
                   },
                   0,
  -                "plan the set of deployments"));
  -        mbeanInfo.addOperationInfo(new GeronimoOperationInfo("isDeployed",
  +                "Execute a number of deployment goals together"));
  +        mbeanInfo.addOperationInfo(new GeronimoOperationInfo("prepareDeploymentJob",
                   new GeronimoParameterInfo[] {
  -                    new GeronimoParameterInfo("URL", URL.class.getName(), "URL to test")
  +                    new GeronimoParameterInfo("goals", "[L"+DeploymentGoal.class.getName()+";", "Goals to achieve as part of this job")
                   },
                   0,
  -                "Determine if the supplied URL is deployed"));
  -        mbeanInfo.addOperationInfo(new GeronimoOperationInfo("deploy",
  +                "Execute a number of deployment goals together, but don't start until instructed to"));
  +        mbeanInfo.addOperationInfo(new GeronimoOperationInfo("startDeploymentJob",
                   new GeronimoParameterInfo[] {
  -                    new GeronimoParameterInfo("URL", URL.class.getName(), "URL to deploy")
  +                    new GeronimoParameterInfo("jobID", Integer.TYPE, "ID of the deployment job to start")
                   },
                   0,
  -                "Deploy the URL"));
  -        mbeanInfo.addOperationInfo(new GeronimoOperationInfo("undeploy",
  +                "Indicates that the specified (prepared) deployment job can be run"));
  +        mbeanInfo.addOperationInfo(new GeronimoOperationInfo("updateDeploymentStatus",
                   new GeronimoParameterInfo[] {
  -                    new GeronimoParameterInfo("URL", URL.class.getName(), "URL to undeploy")
  +                    new GeronimoParameterInfo("jobID", Integer.TYPE.getName(), "The job whose status you're updating"),
  +                    new GeronimoParameterInfo("message", String.class.getName(), "A text description of or note on the deployment status"),
  +                    new GeronimoParameterInfo("type", String.class.getName(), "The type of deployment message (see DeploymentNotification.java)"),
  +                    new GeronimoParameterInfo("tmID", TargetModuleID.class.getName(), "The module whose status is being updated")
                   },
                   0,
  -                "Undeploy the URL"));
  +                "Updates the status of a deployment job"));
           mbeanInfo.addEndpoint(new GeronimoMBeanEndpoint("Planners", DeploymentPlanner.class.getName(),
                   ObjectName.getInstance("geronimo.deployment:role=DeploymentPlanner,*")));
           return mbeanInfo;
       }
   
       /**
  -     * @jmx:managed-operation
  +     * Registers a new deployment job.  This method will return immediately,
  +     * even if the job will take a while to run.  An ID will be assigned to
  +     * the deployment, and notifications will be sent, but it is assumed the
  +     * caller is not interested in tracking the notifications for the job.
        */
  -    public synchronized void planDeployment(ObjectName source, Set urlInfos) {
  -
  -
  -        Set lastScan = (Set) scanResults.get(source);
  -
  -        // find new and existing urlInfos
  -        for (Iterator i = urlInfos.iterator(); i.hasNext();) {
  -            URLInfo urlInfo = (URLInfo) i.next();
  -            URL url = urlInfo.getUrl();
  -
  -
  -            if (!isDeployed(url)) {
  -                //only add a new deployment goal if we don't already have one. One can already exist if
  -                //there was no deployer available when the url was scanned
  -                if ((lastScan == null) || ((lastScan != null) &&!lastScan.contains (urlInfo))){
  -//                    log.info("Adding url goal for " + url);
  -                    goals.add(new DeployURL(url, urlInfo.getType()));
  -                }
  -            } else {
  -//                log.info("Redeploying url " + url);
  -                goals.add(new RedeployURL(url));
  -            }
  -        }
  -
  -        // create remove goals for all urlInfos that were found last time but not now
  -        if (lastScan != null) {
  -            for (Iterator i = lastScan.iterator(); i.hasNext();) {
  -                URLInfo urlInfo = (URLInfo) i.next();
  -                URL url = urlInfo.getUrl();
  -
  -                if (!urlInfos.contains(urlInfo) && isDeployed(url)) {
  -                    goals.add(new UndeployURL(url));
  -                }
  -            }
  -        }
  -        scanResults.put(source, urlInfos);
  -
  -        try {
  -            generatePlans();
  -        } catch (DeploymentException e) {
  -            log.warn("Unable to plan deployment", e);
  +    public void runDeploymentJob(DeploymentGoal[] goals) {
  +        if(goals.length == 0) {
               return;
           }
  -
  -        try {
  -            executePlans();
  -        } catch (DeploymentException e) {
  -            log.warn("Unable to execute deployment plan", e);
  -            return;
  +        int id = ids.nextID();
  +        for(int i=0; i<goals.length; i++) {
  +            goals[i].getTargetModule().setDeploymentID(id);
           }
  +        waiter.addJob(goals);
       }
   
       /**
  -     * @jmx:managed-operation
  +     * Registers a new deployment job.  This method will return immediately,
  +     * and the job will be registered but not executed.  An ID will be
  +     * assigned to the deployment, and the caller can use this ID to start
  +     * the job and monitor it via the notifications emitted for the job.
  +     * You must call startDeploymentJob with this ID or the job will never be
  +     * run.  The purpose of this is that the caller can add a notification
  +     * listener before actually starting the job and ensure that all
  +     * notifications for the job will be received.
        */
  -    public boolean isDeployed(URL url) {
  -        try {
  -            ObjectName pattern = new ObjectName("*:role=DeploymentUnit,url=" + ObjectName.quote(url.toString()) + ",*");
  -            return !context.getServer().queryNames(pattern, null).isEmpty();
  -        } catch (MalformedObjectNameException e) {
  -            throw new AssertionError();
  +    public int prepareDeploymentJob(DeploymentGoal[] goals) {
  +        if(goals.length == 0) { //todo: throws exception or send immediate success notification?
  +            return -1;
  +        }
  +        int id = ids.nextID();
  +        for(int i=0; i<goals.length; i++) {
  +            goals[i].getTargetModule().setDeploymentID(id);
           }
  +        waiter.queueJob(id, goals);
  +        return id;
       }
   
       /**
  -     * @jmx:managed-operation
  +     * Indicates that the specified job can begin execution.  The status of
  +     * the job can be monitored via notifications for its deployment ID.
        */
  -    public synchronized void deploy(URL url) throws DeploymentException {
  -        if (isDeployed(url)) {
  -            return;
  -        }
  +    public void startDeploymentJob(int deploymentID) {
  +        waiter.startJob(deploymentID);
  +    }
   
  -        URLType type = null;
  -        try {
  -            type = URLType.getType(url);
  -        } catch (IOException e) {
  -            throw new DeploymentException(e);
  +    private synchronized void executeJob(DeploymentGoal[] job) {
  +        if(job.length == 0) {
  +            return;
           }
  -
  +        int id = job[0].getTargetModule().getDeploymentID();
           try {
  -            DeployURL goal = new DeployURL(url, type);
  -            goals.add(goal);
  +            goals.addAll(Arrays.asList(job));
  +            for(int i=0; i<job.length; i++) {
  +                updateDeploymentStatus(id, "Starting deployment job.", DeploymentNotification.DEPLOYMENT_UPDATE, job[i].getTargetModule());
  +            }
               generatePlans();
               executePlans();
  +            for(int i=0; i<job.length; i++) {
  +                updateDeploymentStatus(id, "Successfully completed deployment job.", DeploymentNotification.DEPLOYMENT_COMPLETED, job[i].getTargetModule());
  +            }
           } catch (DeploymentException e) {
  -            log.warn("Unable to deploy URL " + url, e);
  -            throw e;
  +            for(int i=0; i<job.length; i++) { //todo: send success for the goals that succeeded, only failure for ones that failed
  +                updateDeploymentStatus(id, "Deployment job failed.", DeploymentNotification.DEPLOYMENT_FAILED, job[i].getTargetModule());
  +            }
  +            log.warn("Unable to complete deployment job " + id, e);
           }
       }
   
       /**
  -     * @jmx:managed-operation
  +     *
        */
  -    public synchronized void undeploy(URL url) {
  -        if (!isDeployed(url)) {
  -            return;
  -        }
  -        try {
  -            UndeployURL goal = new UndeployURL(url);
  -            goals.add(goal);
  -            generatePlans();
  -            executePlans();
  -        } catch (DeploymentException e) {
  -            log.warn("Unable to undeploy URL " + url, e);
  +    public void updateDeploymentStatus(int jobID, String message, String type, TargetModuleID tmID) {
  +        long seq;
  +        synchronized(notificationLock) {
  +            seq = ++notificationSequence;
           }
  +        context.sendNotification(new DeploymentNotification(type, this, seq, message, jobID, tmID));
       }
   
       private void generatePlans() throws DeploymentException {
  @@ -318,6 +290,9 @@
       }
   
       public void doStart() {
  +        Thread t = new Thread(waiter, "Geronimo Deployment Queue");
  +        waiter.setRunner(t);
  +        t.start();
       }
   
       public boolean canStop() {
  @@ -328,5 +303,84 @@
       }
   
       public void doFail() {
  +    }
  +
  +    private class DeploymentWaiter implements Runnable {
  +        private LinkedList work = new LinkedList();
  +        private boolean finished = false;
  +        private Thread runner;
  +        private Map waiting = new HashMap();
  +
  +        public void run() {
  +            DeploymentGoal[] job;
  +            while(!finished) {
  +                job = null;
  +                synchronized(work) {
  +                    if(work.isEmpty()) {
  +                        try {
  +                            work.wait();
  +                        } catch(InterruptedException e) {}
  +                    }
  +                    job = (DeploymentGoal[])work.removeFirst();
  +                }
  +                if(job != null) {
  +                    executeJob(job); // needs to acquire lock on DeploymentController, may take a while
  +                }
  +            }
  +        }
  +
  +        public void queueJob(int jobID, DeploymentGoal[] job) {
  +            waiting.put(new Integer(jobID), job);
  +        }
  +
  +        public void startJob(int jobID) {
  +            DeploymentGoal[] job = (DeploymentGoal[])waiting.remove(new Integer(jobID));
  +            if(job != null) {
  +                addJob(job);
  +            }
  +        }
  +
  +        public void addJob(DeploymentGoal[] job) {
  +            if(job == null) {
  +                log.error("Job should not be null", new RuntimeException());
  +                return;
  +            }
  +            synchronized(work) {
  +                work.addLast(job);
  +                work.notify();
  +            }
  +        }
  +
  +        public void setRunner(Thread t) {
  +            runner = t;
  +        }
  +
  +        /**
  +         * Prevents any future work from being done.  Does not stop a
  +         * deployment job that is currently in progress.
  +         */
  +        public void stop() {
  +            finished = true;
  +            if(runner != null) {
  +                runner.interrupt();
  +                runner = null;
  +            }
  +        }
  +    }
  +
  +    private static class DeploymentIDGenerator {
  +        int counter = 0;
  +
  +        public void start() {
  +            //todo: load the saved job ID?
  +        }
  +
  +        public synchronized int nextID() {
  +            return ++counter;
  +        }
  +
  +        public void stop() {
  +            //todo: persist the current job ID?
  +        }
       }
   }
  
  
  
  1.1                  incubator-geronimo/modules/kernel/src/java/org/apache/geronimo/kernel/deployment/ApplicationDeployer.java
  
  Index: ApplicationDeployer.java
  ===================================================================
  /* ====================================================================
   * The Apache Software License, Version 1.1
   *
   * Copyright (c) 2003 The Apache Software Foundation.  All rights
   * reserved.
   *
   * Redistribution and use in source and binary forms, with or without
   * modification, are permitted provided that the following conditions
   * are met:
   *
   * 1. Redistributions of source code must retain the above copyright
   *    notice, this list of conditions and the following disclaimer.
   *
   * 2. Redistributions in binary form must reproduce the above copyright
   *    notice, this list of conditions and the following disclaimer in
   *    the documentation and/or other materials provided with the
   *    distribution.
   *
   * 3. The end-user documentation included with the redistribution,
   *    if any, must include the following acknowledgment:
   *       "This product includes software developed by the
   *        Apache Software Foundation (http://www.apache.org/)."
   *    Alternately, this acknowledgment may appear in the software itself,
   *    if and wherever such third-party acknowledgments normally appear.
   *
   * 4. The names "Apache" and "Apache Software Foundation" and
   *    "Apache Geronimo" must not be used to endorse or promote products
   *    derived from this software without prior written permission. For
   *    written permission, please contact apache@apache.org.
   *
   * 5. Products derived from this software may not be called "Apache",
   *    "Apache Geronimo", nor may "Apache" appear in their name, without
   *    prior written permission of the Apache Software Foundation.
   *
   * THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED
   * WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
   * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
   * DISCLAIMED.  IN NO EVENT SHALL THE APACHE SOFTWARE FOUNDATION OR
   * ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
   * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
   * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF
   * USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
   * ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
   * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT
   * OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF
   * SUCH DAMAGE.
   * ====================================================================
   *
   * This software consists of voluntary contributions made by many
   * individuals on behalf of the Apache Software Foundation.  For more
   * information on the Apache Software Foundation, please see
   * <http://www.apache.org/>.
   *
   * ====================================================================
   */
  package org.apache.geronimo.kernel.deployment;
  
  import java.net.InetAddress;
  import java.net.UnknownHostException;
  import java.net.URL;
  import java.io.File;
  import java.io.FileOutputStream;
  import java.io.OutputStream;
  import java.io.IOException;
  import java.util.ArrayList;
  import java.util.List;
  import java.util.Set;
  import java.util.Iterator;
  import java.util.Map;
  import java.util.HashMap;
  import javax.enterprise.deploy.spi.Target;
  import javax.enterprise.deploy.spi.TargetModuleID;
  import javax.enterprise.deploy.spi.exceptions.TargetException;
  import javax.enterprise.deploy.shared.ModuleType;
  import javax.management.ObjectName;
  import javax.management.MalformedObjectNameException;
  import javax.management.InstanceNotFoundException;
  import javax.management.MBeanException;
  import javax.management.ReflectionException;
  import org.apache.commons.logging.Log;
  import org.apache.commons.logging.LogFactory;
  import org.apache.geronimo.kernel.service.GeronimoMBeanTarget;
  import org.apache.geronimo.kernel.service.GeronimoMBeanContext;
  import org.apache.geronimo.kernel.service.GeronimoMBeanInfo;
  import org.apache.geronimo.kernel.service.GeronimoOperationInfo;
  import org.apache.geronimo.kernel.service.GeronimoParameterInfo;
  import org.apache.geronimo.kernel.jmx.JMXUtil;
  import org.apache.geronimo.kernel.deployment.scanner.URLInfo;
  import org.apache.geronimo.kernel.deployment.scanner.URLType;
  import org.apache.geronimo.kernel.deployment.goal.DeployURL;
  import org.apache.geronimo.kernel.deployment.goal.RedeployURL;
  import org.apache.geronimo.kernel.deployment.goal.UndeployURL;
  import org.apache.geronimo.kernel.deployment.goal.DeploymentGoal;
  import org.apache.geronimo.kernel.deployment.goal.DistributeURL;
  
  /**
   * This is the server-side back end of the JSR-88 DeploymentManager.
   * Presumably, it will also be invoked by the local directory scanner
   * when a deployable J2EE module is encountered.
   *
   * @version $Revision: 1.1 $ $Date: 2003/11/17 10:57:40 $
   */
  public class ApplicationDeployer implements GeronimoMBeanTarget {
      private final static Log log = LogFactory.getLog(ApplicationDeployer.class);
      private final static ObjectName DEPLOYER_NAME = JMXUtil.getObjectName("geronimo.deployment:role=DeploymentController");
      private ServerTarget localServerTarget;
      private File saveDir;
      private GeronimoMBeanContext context;
  
      public static GeronimoMBeanInfo getGeronimoMBeanInfo() throws Exception {
          GeronimoMBeanInfo mbeanInfo = new GeronimoMBeanInfo();
          mbeanInfo.setAutostart(true);
          mbeanInfo.setTargetClass(ApplicationDeployer.class.getName());
          mbeanInfo.addOperationInfo(new GeronimoOperationInfo("planDeployment",
                  new GeronimoParameterInfo[] {
                      new GeronimoParameterInfo("Source", ObjectName.class.getName(), "Good question!"),
                      new GeronimoParameterInfo("URLInfos", Set.class.getName(), "Set of URLs to plan deployments for")
                  },
                  0,
                  "plan the set of deployments"));
          mbeanInfo.addOperationInfo(new GeronimoOperationInfo("isDeployed",
                  new GeronimoParameterInfo[] {
                      new GeronimoParameterInfo("URL", URL.class.getName(), "URL to test")
                  },
                  0,
                  "Determine if the supplied URL is deployed"));
          mbeanInfo.addOperationInfo(new GeronimoOperationInfo("deploy",
                  new GeronimoParameterInfo[] {
                      new GeronimoParameterInfo("URL", URL.class.getName(), "URL to deploy")
                  },
                  0,
                  "Deploy the URL"));
          mbeanInfo.addOperationInfo(new GeronimoOperationInfo("undeploy",
                  new GeronimoParameterInfo[] {
                      new GeronimoParameterInfo("URL", URL.class.getName(), "URL to undeploy")
                  },
                  0,
                  "Undeploy the URL"));
          //todo: add the rest of the operation methods to support the JSR-88 client
          return mbeanInfo;
      }
  
      /**
       * Creates a new deployer
       */
      public ApplicationDeployer() {
          try {
              localServerTarget = new ServerTarget(InetAddress.getLocalHost().getHostName());
              localServerTarget.setHomeDir(System.getProperty("geronimo.home"));
          } catch(UnknownHostException e) {
              throw new RuntimeException("Unable to look up local hostname", e);
          }
      }
  
      /**
       * Sets the GeronimoMBeanContext.  This is called before doStart and with a null context after stop.
       *
       * @param context the new context; will be null after stop
       */
      public void setMBeanContext(GeronimoMBeanContext context) {
          this.context = context;
      }
  
      /**
       * Checks if the target is ready to start.  A target can delay the start of the GeronimoBean by returning
       * false from this method.
       *
       * @return true if the target is ready to start; false otherwise
       */
      public boolean canStart() {
          return true;
      }
  
      /**
       * Starts the target.  This method is called by the GeronimoMBean to inform the target that the MBean is about to
       * start.  This is called immediately before moving to the running state.
       */
      public void doStart() {
      }
  
      /**
       * Checks if the target is ready to stop.  A target can delay the stopping of the GeronimoBean by returning
       * false from this method.
       *
       * @return true if the target is ready to stop; false otherwise
       */
      public boolean canStop() {
          return true;
      }
  
      /**
       * Stops the target.  This method is called by the GeronimoMBean to inform the target that the MBean is about to
       * stop.  This is called immediately before moving to the stopped state.
       */
      public void doStop() {
      }
  
      /**
       * Fails the MBean.  This method is called by the GeronimoMBean to inform the target that the MBean is about to
       * fail.  This is called immediately before moving to the failed state.
       */
      public void doFail() {
      }
  
      /**
       */
      public Target[] getTargets() { // this should logically be an operation, but it seems that operations must have arguments
          return new Target[]{localServerTarget};
      }
  
      /**
       */
      public TargetModuleID[] getRunningModules(int moduleTypeCode, Target[] targetList) {
          ModuleType moduleType = ModuleType.getModuleType(moduleTypeCode);
          return new TargetModuleID[0]; //todo: implement me
      }
  
      /**
       */
      public TargetModuleID[] getNonRunningModules(int moduleTypeCode, Target[] targetList) {
          ModuleType moduleType = ModuleType.getModuleType(moduleTypeCode);
          return new TargetModuleID[0]; //todo: implement me
      }
  
      /**
       */
      public TargetModuleID[] getAvailableModules(int moduleTypeCode, Target[] targetList) {
          ModuleType moduleType = ModuleType.getModuleType(moduleTypeCode);
          return new TargetModuleID[0]; //todo: implement me
      }
  
      /**
       */
      public int prepareDistribute(Target[] targets, URL moduleArchive, URL deploymentPlan) throws TargetException {
          if(targets.length != 1 || !targets[0].equals(localServerTarget)) {
              throw new TargetException("The deployer can only distribute to the local application server ("+localServerTarget+")");
          }
          //todo: implement me
          return -1;
      }
  
      /**
       */
      public int prepareDistribute(Target[] targets, String name, byte[] moduleArchive, byte[] deploymentPlan) throws TargetException {
          if(targets.length != 1 || !targets[0].equals(localServerTarget)) {
              throw new TargetException("The deployer can only distribute to the local application server ("+localServerTarget+")");
          }
          File module = saveFile(name, moduleArchive);
          File dd = saveFile("geronimo-deployment-"+name, deploymentPlan);
          ModuleType type = verifyDeployment(module, dd);
          //todo: Create and start an MBean for the deployment, use that later to check status of the deployment
          GeronimoTargetModule tm = new GeronimoTargetModule(localServerTarget, name); //todo: specify URL for web apps
          try {
              Integer id = (Integer)context.getServer().invoke(DEPLOYER_NAME, "prepareDeploymentJob", new Object[]{
                  new DistributeURL(tm, module.toURL(), URLType.PACKED_ARCHIVE)}, new String[]{
                      "[L"+DeploymentGoal.class.getName()+";"});
              return id.intValue();
  //            if(!deployments.contains(tm)) {
  //                deployments.add(tm);
  //            }
          } catch(Exception e) {
              log.error("Unable to prepare a deployment job", e);
              return -1;
          }
      }
  
      /**
       */
      public int start(TargetModuleID[] modules) {
          validateModules(modules, false);
          //todo: implement me
          return -1;
      }
  
      /**
       */
      public int stop(TargetModuleID[] modules) {
          validateModules(modules, true);
          //todo: implement me
          return -1;
      }
  
      /**
       */
      public int undeploy(TargetModuleID[] modules) {
          validateModules(modules, false);
          //todo: implement me
          return -1;
      }
  
      /**
       */
      public int redeploy(TargetModuleID[] moduleIDList, URL moduleArchive, URL deploymentPlan) {
          //todo: implement me
          return -1;
      }
  
      /**
       */
      public int redeploy(TargetModuleID[] moduleIDList, byte[] moduleArchive, byte[] deploymentPlan) {
          //todo: implement me
          return -1;
      }
  
      // ---------------------- Methods required to populate Property Editors ----------------------
  
      /**
       * Used to provide a list of security users/groups/roles that the deployer
       * can map a J2EE security role to.
       *
       * @param securityRealm The security realm in use by the application
       *
       * @return A list of security mapping options, or null if the current user
       *         is not authorized to retrieve that information, or the
       *         information is not available.
       */
      public String[] getSecurityRoleOptions(String securityRealm) {
          return new String[0]; //todo: implement me
      }
  
      /**
       * Gets a list of the JNDI names of global resources of a particular type
       * defined in the server.  For example, a list of all javax.sql.DataSource
       * resources.  Note that any resources tied to a particular application
       * will not be included.
       *
       * @param resourceClassName The name of the interface that the resource
       *                          should implement (e.g. javax.sql.DataSource).
       *
       * @return A list of the JNDI names of the available resources.  Returns
       *         null of no such resources are available, the current user is
       *         not authorized to retrieve the list, etc.
       */
      public String[] getResourceJndiNames(String resourceClassName) {
          return new String[0]; //todo: implement me
      }
  
      // ---------------------- Helper Methods ----------------------
  
      private File saveFile(String name, byte[] bytes) {
          if(saveDir == null) {
              String home = System.getProperty("geronimo.home");
              if(home.startsWith("file:")) {
                  home = home.substring(5);
              }
              saveDir = new File(home, "working");
              if(!saveDir.exists()) {
                  log.warn("Geronimo working directory ("+saveDir.getAbsolutePath()+") does not exist!");
                  if(!saveDir.mkdir()) {
                      throw new RuntimeException("Unable to create working directory "+saveDir.getAbsolutePath());
                  }
              }
          }
          try {
              File target = new File(saveDir, name);
              log.info("Preparing to save file to "+target.getAbsolutePath());
              OutputStream out = new FileOutputStream(target, false);
              out.write(bytes);
              out.flush();
              out.close();
              return target;
          } catch(IOException e) {
              throw new RuntimeException("Unable to save file locally");
          }
      }
  
      private ModuleType verifyDeployment(File module, File dd) {
          //todo: validation
          if(module.getName().toLowerCase().endsWith(".war")) {
              return ModuleType.WAR;
          } else if(module.getName().toLowerCase().endsWith(".jar")) {
              return ModuleType.EJB;
          } else {
              log.error("Validation Error: cannot determine the J2EE module type for "+module.getName());
              return null;
          }
      }
  
      private void validateModules(TargetModuleID[] modules, boolean running) {
          for(int i = 0; i < modules.length; i++) {
              TargetModuleID module = modules[i];
              if(!module.getTarget().equals(localServerTarget)) {
                  throw new IllegalArgumentException("Cannot affect modules for target "+module.getTarget().getName());
              }
              //todo: validate whether module is running according to the running argument
          }
      }
  
      private void runDeployment(DeploymentGoal[] goals) {
          try {
              context.getServer().invoke(DEPLOYER_NAME, "runDeploymentJob", new Object[]{goals},
                      new String[]{"[L"+DeploymentGoal.class.getName()+";"});
          } catch(InstanceNotFoundException e) {
              e.printStackTrace();
              throw new RuntimeException("Unable to execute deployment", e);
          } catch(MBeanException e) {
              e.printStackTrace();
              throw new RuntimeException("Unable to execute deployment", e);
          } catch(ReflectionException e) {
              e.printStackTrace();
              throw new RuntimeException("Unable to execute deployment", e);
          }
      }
  
      // ------------------------ Moved from DeploymentController ------------------------------
      private final Map scanResults = new HashMap();
  
      public synchronized void planDeployment(ObjectName source, Set urlInfos) {
          Set lastScan = (Set) scanResults.get(source);
          List goals = new ArrayList();
  
          // find new and existing urlInfos
          for (Iterator i = urlInfos.iterator(); i.hasNext();) {
              URLInfo urlInfo = (URLInfo) i.next();
              URL url = urlInfo.getUrl();
  
  
              if (!isDeployed(url)) {
                  //only add a new deployment goal if we don't already have one. One can already exist if
                  //there was no deployer available when the url was scanned
                  if ((lastScan == null) || ((lastScan != null) &&!lastScan.contains (urlInfo))){
  //                    log.info("Adding url goal for " + url);
                      GeronimoTargetModule tm = new GeronimoTargetModule(localServerTarget, url.getPath()); //todo: specify URL for web apps
                      goals.add(new DeployURL(tm, url, urlInfo.getType()));
                  }
              } else {
  //                log.info("Redeploying url " + url);
                  //todo: look up old TargetModuleID
                  GeronimoTargetModule tm = new GeronimoTargetModule(localServerTarget, url.getPath()); //todo: specify URL for web apps
                  goals.add(new RedeployURL(tm, url));
              }
          }
  
          // create remove goals for all urlInfos that were found last time but not now
          if (lastScan != null) {
              for (Iterator i = lastScan.iterator(); i.hasNext();) {
                  URLInfo urlInfo = (URLInfo) i.next();
                  URL url = urlInfo.getUrl();
  
                  if (!urlInfos.contains(urlInfo) && isDeployed(url)) {
                      //todo: look up old TargetModuleID
                      GeronimoTargetModule tm = new GeronimoTargetModule(localServerTarget, url.getPath()); //todo: specify URL for web apps
                      goals.add(new UndeployURL(tm, url));
                  }
              }
          }
          scanResults.put(source, urlInfos);
          runDeployment((DeploymentGoal[])goals.toArray(new DeploymentGoal[goals.size()]));
      }
  
      /**
       */
      public boolean isDeployed(URL url) {
          try {
              ObjectName pattern = new ObjectName("*:role=DeploymentUnit,url=" + ObjectName.quote(url.toString()) + ",*");
              return !context.getServer().queryNames(pattern, null).isEmpty();
          } catch (MalformedObjectNameException e) {
              throw new AssertionError();
          }
      }
  
      /**
       */
      public synchronized void deploy(URL url) throws DeploymentException {
          if (isDeployed(url)) {
              return;
          }
  
          URLType type = null;
          try {
              type = URLType.getType(url);
          } catch (IOException e) {
              throw new DeploymentException(e);
          }
  
          GeronimoTargetModule tm = new GeronimoTargetModule(localServerTarget, url.getPath()); //todo: specify URL for web apps
          runDeployment(new DeploymentGoal[]{new DeployURL(tm, url, type)});
      }
  
      /**
       */
      public synchronized void undeploy(URL url) {
          if (!isDeployed(url)) {
              return;
          }
          GeronimoTargetModule tm = new GeronimoTargetModule(localServerTarget, url.getPath()); //todo: specify URL for web apps
          //todo: look up old TargetModuleID
          runDeployment(new DeploymentGoal[]{new UndeployURL(tm, url)});
      }
  }
  
  
  
  1.1                  incubator-geronimo/modules/kernel/src/java/org/apache/geronimo/kernel/deployment/GeronimoTargetModule.java
  
  Index: GeronimoTargetModule.java
  ===================================================================
  /* ====================================================================
   * The Apache Software License, Version 1.1
   *
   * Copyright (c) 2003 The Apache Software Foundation.  All rights
   * reserved.
   *
   * Redistribution and use in source and binary forms, with or without
   * modification, are permitted provided that the following conditions
   * are met:
   *
   * 1. Redistributions of source code must retain the above copyright
   *    notice, this list of conditions and the following disclaimer.
   *
   * 2. Redistributions in binary form must reproduce the above copyright
   *    notice, this list of conditions and the following disclaimer in
   *    the documentation and/or other materials provided with the
   *    distribution.
   *
   * 3. The end-user documentation included with the redistribution,
   *    if any, must include the following acknowledgment:
   *       "This product includes software developed by the
   *        Apache Software Foundation (http://www.apache.org/)."
   *    Alternately, this acknowledgment may appear in the software itself,
   *    if and wherever such third-party acknowledgments normally appear.
   *
   * 4. The names "Apache" and "Apache Software Foundation" and
   *    "Apache Geronimo" must not be used to endorse or promote products
   *    derived from this software without prior written permission. For
   *    written permission, please contact apache@apache.org.
   *
   * 5. Products derived from this software may not be called "Apache",
   *    "Apache Geronimo", nor may "Apache" appear in their name, without
   *    prior written permission of the Apache Software Foundation.
   *
   * THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED
   * WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
   * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
   * DISCLAIMED.  IN NO EVENT SHALL THE APACHE SOFTWARE FOUNDATION OR
   * ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
   * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
   * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF
   * USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
   * ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
   * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT
   * OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF
   * SUCH DAMAGE.
   * ====================================================================
   *
   * This software consists of voluntary contributions made by many
   * individuals on behalf of the Apache Software Foundation.  For more
   * information on the Apache Software Foundation, please see
   * <http://www.apache.org/>.
   *
   * ====================================================================
   */
  package org.apache.geronimo.kernel.deployment;
  
  import java.util.List;
  import java.util.ArrayList;
  import java.io.Serializable;
  import javax.enterprise.deploy.spi.TargetModuleID;
  import javax.enterprise.deploy.spi.Target;
  
  /**
   * The Geronimo implementation of TargetModuleID.  In addition to the basic
   * properties required by the interface, this implementation tracks the
   * deployment job ID for when this module was originally distributed, as
   * well as the ObjectName of the MBean for the specified Target where this
   * module can be found.
   *
   * @version $Revision: 1.1 $ $Date: 2003/11/17 10:57:40 $
   */
  public class GeronimoTargetModule implements TargetModuleID, Serializable {
      private Target target;
      private String moduleID;
      private int deploymentID;
      private String webURL;
      private TargetModuleID parent;
      private List children = new ArrayList();
      private String mbeanObjectName;
  
      public GeronimoTargetModule(Target target, String moduleID) {
          this.target = target;
          this.moduleID = moduleID;
          this.deploymentID = deploymentID;
      }
  
      public GeronimoTargetModule(Target target, String moduleID, String webURL) {
          this.target = target;
          this.moduleID = moduleID;
          this.deploymentID = deploymentID;
          this.webURL = webURL;
      }
  
      public GeronimoTargetModule(Target target, String moduleID, GeronimoTargetModule parent) {
          this.target = target;
          this.moduleID = moduleID;
          this.deploymentID = deploymentID;
          this.parent = parent;
          parent.children.add(this);
      }
  
      public GeronimoTargetModule(Target target, String moduleID, String webURL, GeronimoTargetModule parent) {
          this.target = target;
          this.moduleID = moduleID;
          this.deploymentID = deploymentID;
          this.webURL = webURL;
          this.parent = parent;
          parent.children.add(this);
      }
  
      /**
       * Retrieve the target server that this module was deployed to.
       *
       * @return an object representing a server target.
       */
      public Target getTarget() {
          return target;
      }
  
      /**
       * Retrieve the id assigned to represent the deployed module.
       */
      public String getModuleID() {
          return moduleID+"."+deploymentID;
      }
  
      /**
       * If this TargetModulID represents a web module retrieve the URL for it.
       *
       * @return the URL of a web module or null if the module is not a web module.
       */
      public String getWebURL() {
          return webURL;
      }
  
      /**
       * Retrieve the identifier of the parent object of this deployed module.  If
       * there is no parent then this is the root object deployed.  The root could
       * represent an EAR file or it could be a stand alone module that was deployed.
       *
       * @return the TargetModuleID of the parent of this object. A <code>null</code>
       *         value means this module is the root object deployed.
       */
      public TargetModuleID getParentTargetModuleID() {
          return parent;
      }
  
      /**
       * Retrieve a list of identifiers of the children of this deployed module.
       *
       * @return a list of TargetModuleIDs identifying the childern of this object.
       *         A <code>null</code> value means this module has no children
       */
      public TargetModuleID[] getChildTargetModuleID() {
          return (TargetModuleID[])children.toArray(new TargetModuleID[children.size()]);
      }
  
      public int getDeploymentID() {
          return deploymentID;
      }
  
      public void setDeploymentID(int deploymentID) {
          this.deploymentID = deploymentID;
      }
  
      public String getMbeanObjectName() {
          return mbeanObjectName;
      }
  
      public void setMbeanObjectName(String mbeanObjectName) {
          this.mbeanObjectName = mbeanObjectName;
      }
  
      public boolean equals(Object o) {
          if(this == o) return true;
          if(!(o instanceof GeronimoTargetModule)) return false;
  
          final GeronimoTargetModule geronimoTargetModule = (GeronimoTargetModule)o;
  
          if(deploymentID != geronimoTargetModule.deploymentID) return false;
          if(!moduleID.equals(geronimoTargetModule.moduleID)) return false;
          if(parent != null ? !parent.equals(geronimoTargetModule.parent) : geronimoTargetModule.parent != null) return false;
          if(!target.equals(geronimoTargetModule.target)) return false;
  
          return true;
      }
  
      public int hashCode() {
          int result;
          result = target.hashCode();
          result = 29 * result + moduleID.hashCode();
          result = 29 * result + deploymentID;
          return result;
      }
  }
  
  
  
  1.1                  incubator-geronimo/modules/kernel/src/java/org/apache/geronimo/kernel/deployment/ServerTarget.java
  
  Index: ServerTarget.java
  ===================================================================
  /* ====================================================================
   * The Apache Software License, Version 1.1
   *
   * Copyright (c) 2003 The Apache Software Foundation.  All rights
   * reserved.
   *
   * Redistribution and use in source and binary forms, with or without
   * modification, are permitted provided that the following conditions
   * are met:
   *
   * 1. Redistributions of source code must retain the above copyright
   *    notice, this list of conditions and the following disclaimer.
   *
   * 2. Redistributions in binary form must reproduce the above copyright
   *    notice, this list of conditions and the following disclaimer in
   *    the documentation and/or other materials provided with the
   *    distribution.
   *
   * 3. The end-user documentation included with the redistribution,
   *    if any, must include the following acknowledgment:
   *       "This product includes software developed by the
   *        Apache Software Foundation (http://www.apache.org/)."
   *    Alternately, this acknowledgment may appear in the software itself,
   *    if and wherever such third-party acknowledgments normally appear.
   *
   * 4. The names "Apache" and "Apache Software Foundation" and
   *    "Apache Geronimo" must not be used to endorse or promote products
   *    derived from this software without prior written permission. For
   *    written permission, please contact apache@apache.org.
   *
   * 5. Products derived from this software may not be called "Apache",
   *    "Apache Geronimo", nor may "Apache" appear in their name, without
   *    prior written permission of the Apache Software Foundation.
   *
   * THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED
   * WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
   * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
   * DISCLAIMED.  IN NO EVENT SHALL THE APACHE SOFTWARE FOUNDATION OR
   * ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
   * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
   * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF
   * USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
   * ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
   * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT
   * OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF
   * SUCH DAMAGE.
   * ====================================================================
   *
   * This software consists of voluntary contributions made by many
   * individuals on behalf of the Apache Software Foundation.  For more
   * information on the Apache Software Foundation, please see
   * <http://www.apache.org/>.
   *
   * ====================================================================
   */
  package org.apache.geronimo.kernel.deployment;
  
  import java.io.Serializable;
  import javax.enterprise.deploy.spi.Target;
  
  /**
   * A target representing a single (non-clustered) Geronimo server.
   *
   * @version $Revision: 1.1 $ $Date: 2003/11/17 10:57:40 $
   */
  public class ServerTarget implements Target, Serializable {
      private String hostname;
      private String homeDir;
  
      public ServerTarget(String hostname) {
          this.hostname = hostname;
      }
  
      public String getName() {
          return hostname;
      }
  
      public String getHostname() {
          return hostname;
      }
  
      public void setHostname(String hostname) {
          this.hostname = hostname;
      }
  
      public String getHomeDir() {
          return homeDir;
      }
  
      public void setHomeDir(String homeDir) {
          this.homeDir = homeDir;
      }
  
      public String getDescription() {
          return "Geronimo Server"+(homeDir == null ? "" : " at "+homeDir);
      }
  
      public boolean equals(Object o) {
          if(this == o) return true;
          if(!(o instanceof ServerTarget)) return false;
  
          final ServerTarget serverTarget = (ServerTarget)o;
  
          if(!homeDir.equals(serverTarget.homeDir)) return false;
          if(!hostname.equals(serverTarget.hostname)) return false;
  
          return true;
      }
  
      public int hashCode() {
          int result;
          result = hostname.hashCode();
          result = 29 * result + homeDir.hashCode();
          return result;
      }
  }
  
  
  
  1.1                  incubator-geronimo/modules/kernel/src/java/org/apache/geronimo/kernel/deployment/client/DeploymentNotification.java
  
  Index: DeploymentNotification.java
  ===================================================================
  /* ====================================================================
   * The Apache Software License, Version 1.1
   *
   * Copyright (c) 2003 The Apache Software Foundation.  All rights
   * reserved.
   *
   * Redistribution and use in source and binary forms, with or without
   * modification, are permitted provided that the following conditions
   * are met:
   *
   * 1. Redistributions of source code must retain the above copyright
   *    notice, this list of conditions and the following disclaimer.
   *
   * 2. Redistributions in binary form must reproduce the above copyright
   *    notice, this list of conditions and the following disclaimer in
   *    the documentation and/or other materials provided with the
   *    distribution.
   *
   * 3. The end-user documentation included with the redistribution,
   *    if any, must include the following acknowledgment:
   *       "This product includes software developed by the
   *        Apache Software Foundation (http://www.apache.org/)."
   *    Alternately, this acknowledgment may appear in the software itself,
   *    if and wherever such third-party acknowledgments normally appear.
   *
   * 4. The names "Apache" and "Apache Software Foundation" and
   *    "Apache Geronimo" must not be used to endorse or promote products
   *    derived from this software without prior written permission. For
   *    written permission, please contact apache@apache.org.
   *
   * 5. Products derived from this software may not be called "Apache",
   *    "Apache Geronimo", nor may "Apache" appear in their name, without
   *    prior written permission of the Apache Software Foundation.
   *
   * THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED
   * WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
   * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
   * DISCLAIMED.  IN NO EVENT SHALL THE APACHE SOFTWARE FOUNDATION OR
   * ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
   * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
   * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF
   * USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
   * ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
   * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT
   * OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF
   * SUCH DAMAGE.
   * ====================================================================
   *
   * This software consists of voluntary contributions made by many
   * individuals on behalf of the Apache Software Foundation.  For more
   * information on the Apache Software Foundation, please see
   * <http://www.apache.org/>.
   *
   * ====================================================================
   */
  package org.apache.geronimo.kernel.deployment.client;
  
  import javax.management.Notification;
  import javax.enterprise.deploy.spi.TargetModuleID;
  
  /**
   * A notification for a deployment
   *
   * @version $Revision: 1.1 $ $Date: 2003/11/17 10:57:40 $
   */
  public class DeploymentNotification extends Notification {
      public final static String DEPLOYMENT_COMPLETED = "app.deploy.completed";
      public final static String DEPLOYMENT_FAILED = "app.deploy.failure";
      public final static String DEPLOYMENT_UPDATE = "app.deploy.update";
      private int deploymentID;
      private TargetModuleID targetModuleID;
  
      public DeploymentNotification(String type, Object source, long sequenceNumber, int deploymentID, TargetModuleID targetModuleID) {
          super(type, source, sequenceNumber);
          this.deploymentID = deploymentID;
          this.targetModuleID = targetModuleID;
      }
  
      public DeploymentNotification(String type, Object source, long sequenceNumber, long timeStamp, int deploymentID, TargetModuleID targetModuleID) {
          super(type, source, sequenceNumber, timeStamp);
          this.deploymentID = deploymentID;
          this.targetModuleID = targetModuleID;
      }
  
      public DeploymentNotification(String type, Object source, long sequenceNumber, String message, int deploymentID, TargetModuleID targetModuleID) {
          super(type, source, sequenceNumber, message);
          this.deploymentID = deploymentID;
          this.targetModuleID = targetModuleID;
      }
  
      public DeploymentNotification(String type, Object source, long sequenceNumber, long timeStamp, String message, int deploymentID, TargetModuleID targetModuleID) {
          super(type, source, sequenceNumber, timeStamp, message);
          this.deploymentID = deploymentID;
          this.targetModuleID = targetModuleID;
      }
  
      public int getDeploymentID() {
          return deploymentID;
      }
  
      public void setDeploymentID(int deploymentID) {
          this.deploymentID = deploymentID;
      }
  
      public TargetModuleID getTargetModuleID() {
          return targetModuleID;
      }
  
      public void setTargetModuleID(TargetModuleID targetModuleID) {
          this.targetModuleID = targetModuleID;
      }
  }
  
  
  
  1.2       +4 -2      incubator-geronimo/modules/kernel/src/java/org/apache/geronimo/kernel/deployment/goal/DeployURL.java
  
  Index: DeployURL.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/kernel/src/java/org/apache/geronimo/kernel/deployment/goal/DeployURL.java,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- DeployURL.java	8 Sep 2003 04:38:33 -0000	1.1
  +++ DeployURL.java	17 Nov 2003 10:57:40 -0000	1.2
  @@ -60,6 +60,7 @@
   import java.net.URL;
   
   import org.apache.geronimo.kernel.deployment.scanner.URLType;
  +import org.apache.geronimo.kernel.deployment.GeronimoTargetModule;
   
   /**
    *
  @@ -70,7 +71,8 @@
       private final URL url;
       private final URLType type;
   
  -    public DeployURL(final URL url, final URLType type) {
  +    public DeployURL(final GeronimoTargetModule targetModule, final URL url, final URLType type) {
  +        super(targetModule);
           if (url == null) {
               throw new IllegalArgumentException("URL is null");
           }
  
  
  
  1.2       +13 -1     incubator-geronimo/modules/kernel/src/java/org/apache/geronimo/kernel/deployment/goal/DeploymentGoal.java
  
  Index: DeploymentGoal.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/kernel/src/java/org/apache/geronimo/kernel/deployment/goal/DeploymentGoal.java,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- DeploymentGoal.java	8 Sep 2003 04:38:33 -0000	1.1
  +++ DeploymentGoal.java	17 Nov 2003 10:57:40 -0000	1.2
  @@ -55,6 +55,8 @@
    */
   package org.apache.geronimo.kernel.deployment.goal;
   
  +import org.apache.geronimo.kernel.deployment.GeronimoTargetModule;
  +
   /**
    *
    *
  @@ -62,6 +64,12 @@
    */
   public abstract class DeploymentGoal {
       private boolean attained;
  +    private GeronimoTargetModule targetModule;
  +
  +    public DeploymentGoal(GeronimoTargetModule targetModule) {
  +        this.targetModule = targetModule;
  +        attained = false;
  +    }
   
       public boolean isAttained() {
           return attained;
  @@ -69,5 +77,9 @@
   
       public void setAttained(boolean attained) {
           this.attained = attained;
  +    }
  +
  +    public GeronimoTargetModule getTargetModule() {
  +        return targetModule;
       }
   }
  
  
  
  1.2       +4 -2      incubator-geronimo/modules/kernel/src/java/org/apache/geronimo/kernel/deployment/goal/RedeployURL.java
  
  Index: RedeployURL.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/kernel/src/java/org/apache/geronimo/kernel/deployment/goal/RedeployURL.java,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- RedeployURL.java	8 Sep 2003 04:38:33 -0000	1.1
  +++ RedeployURL.java	17 Nov 2003 10:57:40 -0000	1.2
  @@ -56,6 +56,7 @@
   package org.apache.geronimo.kernel.deployment.goal;
   
   import java.net.URL;
  +import org.apache.geronimo.kernel.deployment.GeronimoTargetModule;
   
   /**
    *
  @@ -65,7 +66,8 @@
   public class RedeployURL extends DeploymentGoal {
       private final URL url;
   
  -    public RedeployURL(URL url) {
  +    public RedeployURL(final GeronimoTargetModule targetModule, URL url) {
  +        super(targetModule);
           this.url = url;
       }
   
  
  
  
  1.2       +4 -2      incubator-geronimo/modules/kernel/src/java/org/apache/geronimo/kernel/deployment/goal/UndeployURL.java
  
  Index: UndeployURL.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/kernel/src/java/org/apache/geronimo/kernel/deployment/goal/UndeployURL.java,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- UndeployURL.java	8 Sep 2003 04:38:33 -0000	1.1
  +++ UndeployURL.java	17 Nov 2003 10:57:40 -0000	1.2
  @@ -56,6 +56,7 @@
   package org.apache.geronimo.kernel.deployment.goal;
   
   import java.net.URL;
  +import org.apache.geronimo.kernel.deployment.GeronimoTargetModule;
   
   /**
    *
  @@ -65,7 +66,8 @@
   public class UndeployURL extends DeploymentGoal {
       private final URL url;
   
  -    public UndeployURL(URL url) {
  +    public UndeployURL(final GeronimoTargetModule targetModule, URL url) {
  +        super(targetModule);
           this.url = url;
       }
   
  
  
  
  1.1                  incubator-geronimo/modules/kernel/src/java/org/apache/geronimo/kernel/deployment/goal/DistributeURL.java
  
  Index: DistributeURL.java
  ===================================================================
  /* ====================================================================
   * The Apache Software License, Version 1.1
   *
   * Copyright (c) 2003 The Apache Software Foundation.  All rights
   * reserved.
   *
   * Redistribution and use in source and binary forms, with or without
   * modification, are permitted provided that the following conditions
   * are met:
   *
   * 1. Redistributions of source code must retain the above copyright
   *    notice, this list of conditions and the following disclaimer.
   *
   * 2. Redistributions in binary form must reproduce the above copyright
   *    notice, this list of conditions and the following disclaimer in
   *    the documentation and/or other materials provided with the
   *    distribution.
   *
   * 3. The end-user documentation included with the redistribution,
   *    if any, must include the following acknowledgment:
   *       "This product includes software developed by the
   *        Apache Software Foundation (http://www.apache.org/)."
   *    Alternately, this acknowledgment may appear in the software itself,
   *    if and wherever such third-party acknowledgments normally appear.
   *
   * 4. The names "Apache" and "Apache Software Foundation" and
   *    "Apache Geronimo" must not be used to endorse or promote products
   *    derived from this software without prior written permission. For
   *    written permission, please contact apache@apache.org.
   *
   * 5. Products derived from this software may not be called "Apache",
   *    "Apache Geronimo", nor may "Apache" appear in their name, without
   *    prior written permission of the Apache Software Foundation.
   *
   * THIS SOFTWARE IS PROVIDED ``AS IS'' AND ANY EXPRESSED OR IMPLIED
   * WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE IMPLIED WARRANTIES
   * OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE ARE
   * DISCLAIMED.  IN NO EVENT SHALL THE APACHE SOFTWARE FOUNDATION OR
   * ITS CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
   * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
   * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF
   * USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND
   * ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY,
   * OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT
   * OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF
   * SUCH DAMAGE.
   * ====================================================================
   *
   * This software consists of voluntary contributions made by many
   * individuals on behalf of the Apache Software Foundation.  For more
   * information on the Apache Software Foundation, please see
   * <http://www.apache.org/>.
   *
   * ====================================================================
   */
  package org.apache.geronimo.kernel.deployment.goal;
  
  import java.io.File;
  import java.net.MalformedURLException;
  import java.net.URL;
  
  import org.apache.geronimo.kernel.deployment.scanner.URLType;
  import org.apache.geronimo.kernel.deployment.GeronimoTargetModule;
  
  /**
   * A goal for a module that needs to be distributed (according to the JSR-88
   * definition).  This means the modules must be physically distributed to each
   * target (if the server requires it), it must be validated, and any server-
   * specific code must be generated.  In other words, once a module is
   * successfully distributed, it should be able to be started with no errors.
   *
   * @version $Revision: 1.1 $ $Date: 2003/11/17 10:57:40 $
   */
  public class DistributeURL extends DeploymentGoal {
      private final URL url;
      private final URLType type;
  
      public DistributeURL(final GeronimoTargetModule targetModule, final URL url, final URLType type) {
          super(targetModule);
          if (url == null) {
              throw new IllegalArgumentException("URL is null");
          }
          if (type == null) {
              throw new IllegalArgumentException("URLType is null");
          }
  
          this.url = normalizeURL(url);
          this.type = type;
      }
  
      public URL getUrl() {
          return url;
      }
  
      public URLType getType() {
          return type;
      }
  
      private static URL normalizeURL(URL url) {
          assert url != null;
  
          if (url.getProtocol().equals("file")) {
              String filename = url.getFile().replace('/', File.separatorChar);
              File file = new File(filename);
              try {
                  url = file.toURI().toURL();
              } catch (MalformedURLException ignore) {
              }
          }
  
          return url;
      }
  }
  
  
  
  1.4       +13 -20    incubator-geronimo/modules/kernel/src/java/org/apache/geronimo/kernel/deployment/scanner/DeploymentScanner.java
  
  Index: DeploymentScanner.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/kernel/src/java/org/apache/geronimo/kernel/deployment/scanner/DeploymentScanner.java,v
  retrieving revision 1.3
  retrieving revision 1.4
  diff -u -r1.3 -r1.4
  --- DeploymentScanner.java	16 Nov 2003 00:52:22 -0000	1.3
  +++ DeploymentScanner.java	17 Nov 2003 10:57:40 -0000	1.4
  @@ -57,7 +57,6 @@
   
   import java.io.File;
   import java.io.IOException;
  -import java.net.MalformedURLException;
   import java.net.URL;
   import java.util.Collections;
   import java.util.HashMap;
  @@ -65,23 +64,17 @@
   import java.util.Iterator;
   import java.util.Map;
   import java.util.Set;
  -
  -import javax.management.MBeanServer;
   import javax.management.ObjectName;
  -import javax.management.relation.RelationServiceMBean;
  -
   import org.apache.commons.logging.Log;
   import org.apache.commons.logging.LogFactory;
  -import org.apache.geronimo.kernel.jmx.JMXUtil;
  -import org.apache.geronimo.kernel.service.AbstractManagedObject;
  -import org.apache.geronimo.kernel.service.GeronimoMBeanInfo;
  +import org.apache.geronimo.kernel.deployment.ApplicationDeployer;
   import org.apache.geronimo.kernel.service.GeronimoAttributeInfo;
  -import org.apache.geronimo.kernel.service.GeronimoOperationInfo;
  -import org.apache.geronimo.kernel.service.GeronimoParameterInfo;
  +import org.apache.geronimo.kernel.service.GeronimoMBeanContext;
   import org.apache.geronimo.kernel.service.GeronimoMBeanEndpoint;
  +import org.apache.geronimo.kernel.service.GeronimoMBeanInfo;
   import org.apache.geronimo.kernel.service.GeronimoMBeanTarget;
  -import org.apache.geronimo.kernel.service.GeronimoMBeanContext;
  -import org.apache.geronimo.kernel.deployment.DeploymentController;
  +import org.apache.geronimo.kernel.service.GeronimoOperationInfo;
  +import org.apache.geronimo.kernel.service.GeronimoParameterInfo;
   
   /**
    * An MBean that maintains a list of URLs and periodically invokes a Scanner
  @@ -99,7 +92,7 @@
       private long scanInterval;
       private boolean run;
       private Thread scanThread;
  -    private DeploymentController deploymentController;
  +    private ApplicationDeployer applicationDeployer;
   
       public static GeronimoMBeanInfo getGeronimoMBeanInfo() throws Exception {
           GeronimoMBeanInfo mbeanInfo = new GeronimoMBeanInfo();
  @@ -123,15 +116,15 @@
                   new GeronimoParameterInfo[]{},
                   1,
                   "Scan all URLs now"));
  -        mbeanInfo.addEndpoint(new GeronimoMBeanEndpoint("DeploymentController",
  -                DeploymentController.class.getName(),
  -                ObjectName.getInstance("geronimo.deployment:role=DeploymentController"),
  +        mbeanInfo.addEndpoint(new GeronimoMBeanEndpoint("ApplicationDeployer",
  +                ApplicationDeployer.class.getName(),
  +                ObjectName.getInstance("geronimo.deployment:role=ApplicationDeployer"),
                   true));
           return mbeanInfo;
       }
   
  -    public void setDeploymentController(DeploymentController deploymentController) {
  -        this.deploymentController = deploymentController;
  +    public void setApplicationDeployer(ApplicationDeployer applicationDeployer) {
  +        this.applicationDeployer = applicationDeployer;
       }
   
       public DeploymentScanner() {
  @@ -252,7 +245,7 @@
           }
   
           try {
  -            deploymentController.planDeployment(context.getObjectName(), results);
  +            applicationDeployer.planDeployment(context.getObjectName(), results);
           } catch (Exception e) {
               log.error(e.getMessage(), e);
           }