You are viewing a plain text version of this content. The canonical link for it is here.
Posted to scm@geronimo.apache.org by jb...@apache.org on 2004/02/12 19:27:39 UTC

cvs commit: incubator-geronimo/modules/deployment/src/test/org/apache/geronimo/deployment/service ParentDeployerTest.java ServiceDeployerTest.java

jboynes     2004/02/12 10:27:39

  Modified:    modules/deployment project.xml
               modules/deployment/src/java/org/apache/geronimo/deployment
                        Bootstrap.java
               modules/deployment/src/schema geronimo-common.xsd
                        geronimo-config.xsd
               modules/deployment/src/test-resources/services plan1.xml
  Added:       modules/deployment/src/java/org/apache/geronimo/deployment
                        ConfigurationBuilder.java Deployer.java
                        DeploymentContext.java
               modules/deployment/src/java/org/apache/geronimo/deployment/service
                        ServiceConfigBuilder.java
  Removed:     modules/deployment/src/java/org/apache/geronimo/deployment
                        BatchDeployerFactory.java JARDeployer.java
               modules/deployment/src/java/org/apache/geronimo/deployment/tools
                        DeployCommand.java
               modules/deployment/src/java/org/apache/geronimo/deployment/util
                        ExplicitDeployment.java
               modules/deployment/src/test/org/apache/geronimo/deployment
                        JARDeployerTest.java
               modules/deployment/src/test/org/apache/geronimo/deployment/service
                        ParentDeployerTest.java ServiceDeployerTest.java
  Log:
  new GBean based service deployer
  deprecate old DeploymentModule API
    remove old service deployer using it
    todo: remove DeploymentModule API when war and connector deployers are updated
  new Bootstrap class which is invoked from maven during the build to intialize the deployers
  
  Revision  Changes    Path
  1.10      +9 -1      incubator-geronimo/modules/deployment/project.xml
  
  Index: project.xml
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/deployment/project.xml,v
  retrieving revision 1.9
  retrieving revision 1.10
  diff -u -r1.9 -r1.10
  --- project.xml	9 Feb 2004 07:17:31 -0000	1.9
  +++ project.xml	12 Feb 2004 18:27:39 -0000	1.10
  @@ -52,6 +52,14 @@
           </dependency>
   
           <dependency>
  +            <groupId>geronimo</groupId>
  +            <artifactId>geronimo-system</artifactId>
  +            <version>DEV</version>
  +            <properties>
  +            </properties>
  +        </dependency>
  +
  +        <dependency>
               <groupId>geronimo-spec</groupId>
               <artifactId>geronimo-spec-j2ee-deployment</artifactId>
               <version>DEV</version>
  
  
  
  1.6       +92 -74    incubator-geronimo/modules/deployment/src/java/org/apache/geronimo/deployment/Bootstrap.java
  
  Index: Bootstrap.java
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/deployment/src/java/org/apache/geronimo/deployment/Bootstrap.java,v
  retrieving revision 1.5
  retrieving revision 1.6
  diff -u -r1.5 -r1.6
  --- Bootstrap.java	4 Feb 2004 05:43:31 -0000	1.5
  +++ Bootstrap.java	12 Feb 2004 18:27:39 -0000	1.6
  @@ -55,108 +55,126 @@
    */
   package org.apache.geronimo.deployment;
   
  +import java.io.BufferedOutputStream;
   import java.io.File;
  -import java.net.MalformedURLException;
  +import java.io.FileInputStream;
  +import java.io.FileOutputStream;
  +import java.io.IOException;
  +import java.io.ObjectOutputStream;
   import java.net.URI;
  -import java.net.URL;
   import java.util.ArrayList;
  +import java.util.Collections;
  +import java.util.HashMap;
   import java.util.List;
  -import javax.xml.parsers.DocumentBuilderFactory;
  -import javax.xml.parsers.ParserConfigurationException;
  +import java.util.Map;
  +import java.util.jar.JarEntry;
  +import java.util.jar.JarOutputStream;
  +import java.util.zip.ZipEntry;
   import javax.management.ObjectName;
   
  -import org.apache.geronimo.deployment.service.ServiceDeployer;
  -import org.apache.geronimo.deployment.util.FileUtil;
  +import org.apache.geronimo.deployment.service.ServiceConfigBuilder;
  +import org.apache.geronimo.gbean.jmx.GBeanMBean;
   import org.apache.geronimo.kernel.Kernel;
  +import org.apache.geronimo.kernel.config.Configuration;
   import org.apache.geronimo.kernel.config.LocalConfigStore;
  -import org.apache.geronimo.deployment.util.URLInfo;
  -import org.apache.geronimo.deployment.util.URLType;
  +import org.apache.geronimo.system.repository.ReadOnlyRepository;
  +import org.apache.geronimo.system.serverinfo.ServerInfo;
   
   /**
  - * Helper class to bootstrap a Geronimo instance from a service archive.
  - * This deploys the service definition to create a bootstrap Configuration,
  - * and then creates a Kernel to run that configuration. This allows someone
  - * to boot a Kernel without pre-deploying and installing the Configuration.
  + * Helper class to bootstrap the Geronimo deployer.
    *
    * @version $Revision$ $Date$
    */
   public class Bootstrap {
  -    public static final URI CONFIG_ID = URI.create("org/apache/geronimo/Bootstrap");
  +    public static final URI CONFIG_ID = URI.create("org/apache/geronimo/ServiceDeployer");
   
       /**
  -     * Entry point. Arguments are:
  -     * <li>directory to use for the ConfigurationStore</li>
  -     * <li>URL for initial service to deploy</li>
  -     *
  -     * @param args command line arguments
  +     * Invoked from maven.xml during the build to create the first Deployment Configuration
  +     * @param car the configuration file to generate
  +     * @param store the store to install the configuration into
        */
  -    public static void main(String[] args) {
  -        if (args.length < 1) {
  -            System.err.println("usage: " + Bootstrap.class.getName() + " <service-url>");
  -            System.exit(1);
  -        }
  -        URL serviceURL;
  -        try {
  -            serviceURL = new URL(args[0]);
  -        } catch (MalformedURLException e) {
  -            e.printStackTrace();
  -            System.exit(1);
  -            throw new AssertionError();
  -        }
  +    public static void bootstrap(String car, String store, String systemJar) {
  +        File carfile = new File(car);
  +        File storeDir = new File(store);
  +        File system = new File(systemJar);
   
  -        List deployers = new ArrayList();
  +        ClassLoader oldCL = Thread.currentThread().getContextClassLoader();
  +        Thread.currentThread().setContextClassLoader(Bootstrap.class.getClassLoader());
           try {
  -            deployers.add(new ServiceDeployer(DocumentBuilderFactory.newInstance().newDocumentBuilder()));
  -        } catch (ParserConfigurationException e) {
  -            e.printStackTrace();
  -            System.exit(2);
  -            throw new AssertionError();
  -        }
  +            Map gbeans = new HashMap();
   
  -        File tmpDir = new File(System.getProperty("java.io.tmpdir"), "geronimo");
  -        FileUtil.recursiveDelete(tmpDir);
  -        tmpDir.mkdir();
  -
  -        File storeDir = new File(tmpDir, "config");
  -        storeDir.mkdir();
  -        final Kernel kernel = new Kernel("geronimo.kernel", "geronimo", LocalConfigStore.GBEAN_INFO, storeDir);
  -        Runtime.getRuntime().addShutdownHook(new Thread("Shutdown Thread") {
  -            public void run() {
  -                kernel.shutdown();
  -            }
  -        });
  -        try {
  -            kernel.boot();
  -        } catch (Exception e) {
  -            e.printStackTrace();
  -            System.exit(2);
  -            throw new AssertionError();
  -        }
  +            // Install ServerInfo GBean
  +            ObjectName serverName = new ObjectName("geronimo.deployer:role=ServerInfo");
  +            GBeanMBean server = new GBeanMBean(ServerInfo.getGBeanInfo());
  +            gbeans.put(serverName, server);
  +
  +            // Install default local Repository
  +            ObjectName repoName = new ObjectName("geronimo.deployer:role=Repository,root=repository");
  +            GBeanMBean localRepo = new GBeanMBean(ReadOnlyRepository.GBEAN_INFO);
  +            localRepo.setAttribute("Root", URI.create("repository/"));
  +            localRepo.setReferencePatterns("ServerInfo", Collections.singleton(serverName));
  +            gbeans.put(repoName, localRepo);
  +
  +            // Install ServiceConfigBuilder
  +            ObjectName builderName = new ObjectName("geronimo.deployer:role=Builder,type=Service,id=" + CONFIG_ID.toString());
  +            GBeanMBean serviceBuilder = new GBeanMBean(ServiceConfigBuilder.GBEAN_INFO);
  +            serviceBuilder.setReferencePatterns("Repository", Collections.singleton(repoName));
  +            serviceBuilder.setReferencePatterns("Kernel", Collections.singleton(Kernel.KERNEL));
  +            gbeans.put(builderName, serviceBuilder);
  +
  +            // Install Deployer
  +            ObjectName deployerName = Deployer.getDeployerName(CONFIG_ID);
  +            GBeanMBean deployer = new GBeanMBean(Deployer.GBEAN_INFO);
  +            deployer.setReferencePatterns("Builders", Collections.singleton(new ObjectName("geronimo.deployer:role=Builder,id=" + CONFIG_ID.toString() + ",*")));
  +            gbeans.put(deployerName, deployer);
   
  -        try {
  -            File workDir = new File(tmpDir, "deployment");
  -            workDir.mkdir();
  -            URLDeployer deployer = new URLDeployer(null, CONFIG_ID, deployers, workDir);
  -            deployer.addSource(new URLInfo(serviceURL, URLType.getType(serviceURL)));
  -            deployer.deploy();
  +            List classPath = new ArrayList();
   
  -            ObjectName configName = kernel.load(deployer.getConfiguration(), workDir.toURL());
  -            kernel.getMBeanServer().invoke(configName, "startRecursive", null, null);
  +            JarOutputStream jos = new JarOutputStream(new BufferedOutputStream(new FileOutputStream(carfile)));
  +            try {
  +                URI path = URI.create(system.getName());
  +                addFile(jos, path, system);
  +                classPath.add(path);
  +
  +                GBeanMBean config = new GBeanMBean(Configuration.GBEAN_INFO);
  +                config.setAttribute("ID", CONFIG_ID);
  +                config.setReferencePatterns("Parent", null);
  +                config.setAttribute("ClassPath", classPath);
  +                config.setAttribute("GBeanState", Configuration.storeGBeans(gbeans));
  +                config.setAttribute("Dependencies", Collections.EMPTY_LIST);
  +
  +                jos.putNextEntry(new ZipEntry("META-INF/config.ser"));
  +                ObjectOutputStream ois = new ObjectOutputStream(jos);
  +                Configuration.storeGMBeanState(config, ois);
  +                ois.flush();
  +                jos.closeEntry();
  +            } finally {
  +                jos.close();
  +            }
  +
  +            LocalConfigStore configStore = new LocalConfigStore(storeDir);
  +            configStore.install(carfile.toURL());
           } catch (Exception e) {
               e.printStackTrace();
               System.exit(2);
               throw new AssertionError();
  +        } finally {
  +            Thread.currentThread().setContextClassLoader(oldCL);
           }
  +    }
   
  -        // loop to keep the kernel alive
  -        while (kernel.isRunning()) {
  -            try {
  -                synchronized (kernel) {
  -                    kernel.wait();
  -                }
  -            } catch (InterruptedException e) {
  -                // continue
  +    private static void addFile(JarOutputStream jos, URI path, File file) throws IOException {
  +        FileInputStream fis = new FileInputStream(file);
  +        try {
  +            jos.putNextEntry(new JarEntry(path.toString()));
  +            byte[] buffer = new byte[4096];
  +            int count;
  +            while ((count = fis.read(buffer)) > 0) {
  +                jos.write(buffer, 0, count);
               }
  +            jos.closeEntry();
  +        } finally {
  +            fis.close();
           }
       }
   }
  
  
  
  1.1                  incubator-geronimo/modules/deployment/src/java/org/apache/geronimo/deployment/ConfigurationBuilder.java
  
  Index: ConfigurationBuilder.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.deployment;
  
  import java.io.IOException;
  import java.io.File;
  
  import org.apache.xmlbeans.XmlObject;
  
  /**
   * 
   * 
   * @version $Revision: 1.1 $ $Date: 2004/02/12 18:27:39 $
   */
  public interface ConfigurationBuilder {
      boolean canConfigure(XmlObject plan);
      void buildConfiguration(File outfile, XmlObject plan, boolean install) throws IOException, DeploymentException;
  }
  
  
  
  1.1                  incubator-geronimo/modules/deployment/src/java/org/apache/geronimo/deployment/Deployer.java
  
  Index: Deployer.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.deployment;
  
  import java.io.File;
  import java.net.MalformedURLException;
  import java.net.URI;
  import java.net.URL;
  import java.util.Collection;
  import java.util.Iterator;
  import java.util.Properties;
  import javax.management.MalformedObjectNameException;
  import javax.management.ObjectName;
  
  import org.apache.commons.cli.CommandLine;
  import org.apache.commons.cli.HelpFormatter;
  import org.apache.commons.cli.Options;
  import org.apache.commons.cli.ParseException;
  import org.apache.commons.cli.PosixParser;
  import org.apache.commons.logging.LogFactory;
  import org.apache.geronimo.gbean.GBeanInfo;
  import org.apache.geronimo.gbean.GBeanInfoFactory;
  import org.apache.geronimo.gbean.GConstructorInfo;
  import org.apache.geronimo.gbean.GOperationInfo;
  import org.apache.geronimo.gbean.GReferenceInfo;
  import org.apache.geronimo.kernel.Kernel;
  import org.apache.geronimo.kernel.log.GeronimoLogFactory;
  import org.apache.geronimo.kernel.config.LocalConfigStore;
  import org.apache.xmlbeans.SchemaTypeLoader;
  import org.apache.xmlbeans.XmlBeans;
  import org.apache.xmlbeans.XmlObject;
  
  /**
   * Command line based deployment utility which combines multiple deployable modules
   * into a single configuration.
   *
   * @version $Revision: 1.1 $ $Date: 2004/02/12 18:27:39 $
   */
  public class Deployer {
      static {
          // This MUST be done before the first log is acquired
          System.setProperty(LogFactory.FACTORY_PROPERTY, GeronimoLogFactory.class.getName());
      }
  
      public static final URI DEFAULT_CONFIG = URI.create("org/apache/geronimo/J2EEDeployer");
  
      private final Collection builders;
  
      public Deployer(Collection builders) {
          this.builders = builders;
      }
  
      public void deploy(CommandLine cmd) throws Exception {
          URL planURL = getURL(cmd.getOptionValue('p'));
          XmlObject plan = getLoader().parse(planURL, null, null);
          ConfigurationBuilder builder = null;
          for (Iterator i = builders.iterator(); i.hasNext();) {
              builder = (ConfigurationBuilder) i.next();
              if (builder.canConfigure(plan)) {
                  break;
              }
              builder = null;
          }
          if (builder == null) {
              throw new DeploymentException("No deployer found for this plan type");
          }
  
          boolean saveOutput;
          File outfile;
          if (cmd.hasOption('o')) {
              saveOutput = true;
              outfile = new File(cmd.getOptionValue('o'));
          } else {
              saveOutput = false;
              outfile = File.createTempFile("deployer", ".car");
          }
  
          builder.buildConfiguration(outfile, plan, cmd.hasOption('I'));
  
          if (!saveOutput) {
              outfile.delete();
          }
      }
  
      private URL getURL(String location) throws MalformedURLException {
          File f = new File(location);
          if (f.exists() && f.canRead()) {
              return f.toURL();
          }
          return new URL(new File(".").toURL(), location);
      }
  
      private SchemaTypeLoader getLoader() {
          // @todo this should also set up the entity resolver and error handlers
          return XmlBeans.getContextTypeLoader();
      }
  
      /**
       * Command line entry point called by executable jar
       * @param args command line args
       */
      public static void main(String[] args) {
          try {
              CommandLine cmd = parseArgs(args);
              if (cmd == null) {
                  return;
              }
              URI deployerID = cmd.hasOption('d') ? new URI(cmd.getOptionValue('d')) : DEFAULT_CONFIG;
              File configStore = cmd.hasOption('s') ? new File(cmd.getOptionValue('s')) : new File("../config-store");
              if (!configStore.isDirectory()) {
                  System.err.println("Store does not exist or is not a directory: " + configStore);
                  System.exit(2);
                  throw new AssertionError();
              }
  
              Kernel kernel = new Kernel("geronimo.deployment", LocalConfigStore.GBEAN_INFO, configStore);
              kernel.boot();
  
              ObjectName configName = kernel.load(deployerID);
              kernel.startRecursiveGBean(configName);
  
              ObjectName deployerName = getDeployerName(deployerID);
              kernel.getMBeanServer().invoke(deployerName, "deploy", new Object[]{cmd}, new String[]{CommandLine.class.getName()});
  
              kernel.stopGBean(configName);
              kernel.shutdown();
          } catch (ParseException e) {
              e.printStackTrace();
              System.exit(2);
              throw new AssertionError();
          } catch (Exception e) {
              e.printStackTrace();
              System.exit(3);
              throw new AssertionError();
          }
      }
  
      public static ObjectName getDeployerName(URI configID) throws MalformedObjectNameException {
          Properties props = new Properties();
          props.put("role", "Deployer");
          props.put("config", configID.toString());
          return new ObjectName("geronimo.deployment", props);
      }
  
      public static CommandLine parseArgs(String[] args) throws ParseException {
          Options options = new Options();
          options.addOption("h", "help", false, "print this message");
          options.addOption("I", "install", false, "install configuration in store");
          options.addOption("o", "outfile", true, "output file to generate");
          options.addOption("m", "module", true, "module to deploy");
          options.addOption("p", "plan", true, "deployment plan");
          options.addOption("d", "deployer", true, "id of the config used to perform deployment");
          options.addOption("s", "store", true, "location of the config store");
          CommandLine cmd = new PosixParser().parse(options, args);
          if (cmd.hasOption("h")) {
              new HelpFormatter().printHelp("deploy.jar [OPTIONS] <module>...", options);
              return null;
          }
          return cmd;
      }
  
      /**
       * GBean entry point invoked from an executable CAR.
       * @param args command line args
       */
      public void deploy(String[] args) throws Exception {
          CommandLine cmd = parseArgs(args);
          if (cmd == null) {
              return;
          }
          deploy(cmd);
      }
  
      public static final GBeanInfo GBEAN_INFO;
  
      static {
          GBeanInfoFactory infoFactory = new GBeanInfoFactory(Deployer.class);
          infoFactory.addOperation(new GOperationInfo("deploy", new Class[]{String[].class}));
          infoFactory.addOperation(new GOperationInfo("deploy", new Class[]{CommandLine.class}));
          infoFactory.addReference(new GReferenceInfo("Builders", ConfigurationBuilder.class));
          infoFactory.setConstructor(new GConstructorInfo(
                  new String[]{"Builders"},
                  new Class[]{Collection.class}
          ));
          GBEAN_INFO = infoFactory.getBeanInfo();
      }
  }
  
  
  
  1.1                  incubator-geronimo/modules/deployment/src/java/org/apache/geronimo/deployment/DeploymentContext.java
  
  Index: DeploymentContext.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.deployment;
  
  import java.io.IOException;
  import java.io.InputStream;
  import java.io.ObjectOutputStream;
  import java.net.MalformedURLException;
  import java.net.URI;
  import java.net.URL;
  import java.net.URLClassLoader;
  import java.util.ArrayList;
  import java.util.Collections;
  import java.util.HashMap;
  import java.util.Iterator;
  import java.util.LinkedHashSet;
  import java.util.List;
  import java.util.Map;
  import java.util.Set;
  import java.util.jar.JarOutputStream;
  import java.util.zip.ZipEntry;
  import javax.management.MalformedObjectNameException;
  import javax.management.ObjectName;
  
  import org.apache.geronimo.gbean.jmx.GBeanMBean;
  import org.apache.geronimo.kernel.Kernel;
  import org.apache.geronimo.kernel.config.Configuration;
  import org.apache.geronimo.kernel.repository.Repository;
  
  /**
   *
   *
   * @version $Revision: 1.1 $ $Date: 2004/02/12 18:27:39 $
   */
  public class DeploymentContext {
      private final ObjectName configName;
      private final Kernel kernel;
      private final GBeanMBean config;
      private final Map gbeans = new HashMap();
      private final Set dependencies = new LinkedHashSet();
      private final Set classPath = new LinkedHashSet();
      private final Map includes = new HashMap();
      private final JarOutputStream jos;
      private final byte[] buffer = new byte[4096];
      private final List ancestors;
      private final ClassLoader parentCL;
  
      public DeploymentContext(JarOutputStream jos, URI id, URI parentID, Kernel kernel) throws IOException,MalformedObjectNameException, DeploymentException {
          this.jos = jos;
          this.kernel = kernel;
  
          configName = Kernel.getConfigObjectName(id);
          config = new GBeanMBean(Configuration.GBEAN_INFO);
  
          try {
              config.setAttribute("ID", id);
              config.setAttribute("ParentID", parentID);
          } catch (Exception e) {
              // we created this GBean ...
              throw new AssertionError();
          }
  
          if (parentID != null) {
              ObjectName parentName = Kernel.getConfigObjectName(parentID);
              config.setReferencePatterns("Parent", Collections.singleton(parentName));
              try {
                  ancestors = kernel.loadRecursive(parentID);
              } catch (Exception e) {
                  throw new DeploymentException("Unable to load parents", e);
              }
  
              for (Iterator i = ancestors.iterator(); i.hasNext();) {
                  ObjectName name = (ObjectName) i.next();
                  try {
                      // start the config to get the classloaders going,
                      // by not specfying startRecursive none of the GBeans should start
                      kernel.startGBean(name);
                  } catch (Exception e) {
                      throw new DeploymentException(e);
                  }
              }
              try {
                  parentCL = (ClassLoader) kernel.getMBeanServer().getAttribute(parentName, "ClassLoader");
              } catch (Exception e) {
                  throw new DeploymentException(e);
              }
          } else {
              ancestors = null;
              parentCL = ClassLoader.getSystemClassLoader();
          }
  
      }
  
      public void addGBean(ObjectName name, GBeanMBean gbean) {
          gbeans.put(name, gbean);
      }
  
      public void addDependency(URI uri) {
          dependencies.add(uri);
      }
  
      public void addInclude(URI path, URL url) throws IOException {
          InputStream is = url.openStream();
          try {
              addFile(path, is);
          } finally {
              is.close();
          }
          classPath.add(path);
          includes.put(path, url);
      }
  
      public ClassLoader getClassLoader(Repository repository) throws DeploymentException {
          // save the dependencies and classpath
          try {
              config.setReferencePatterns("Repositories", Collections.singleton(new ObjectName("*:role=Repository,*")));
              config.setAttribute("Dependencies", new ArrayList(dependencies));
              config.setAttribute("ClassPath", new ArrayList(classPath));
          } catch (Exception e) {
              throw new DeploymentException("Unable to initialize Configuration", e);
          }
  
          URL[] urls = new URL[dependencies.size() + classPath.size()];
          int j=0;
          for (Iterator i = dependencies.iterator(); i.hasNext();) {
              URI uri = (URI) i.next();
              try {
                  urls[j++] = repository.getURL(uri);
              } catch (MalformedURLException e) {
                  throw new DeploymentException(e);
              }
          }
          for (Iterator i = classPath.iterator(); i.hasNext();) {
              URI uri = (URI) i.next();
              urls[j++] = (URL) includes.get(uri);
          }
          return new URLClassLoader(urls, parentCL);
      }
  
      public void addFile(URI path, InputStream source) throws IOException {
          if (jos == null) {
              throw new IllegalStateException();
          }
          jos.putNextEntry(new ZipEntry(path.getPath()));
          int count;
          while ((count = source.read(buffer)) > 0) {
              jos.write(buffer, 0, count);
          }
          jos.closeEntry();
      }
  
      public void close() throws IOException, DeploymentException {
          if (jos == null) {
              throw new IllegalStateException();
          }
          saveConfiguration();
          jos.flush();
          jos.close();
  
          try {
              if (ancestors != null) {
                  kernel.stopGBean((ObjectName) ancestors.get(0));
              }
          } catch (Exception e) {
              throw new DeploymentException(e);
          }
      }
  
      private void saveConfiguration() throws IOException, DeploymentException {
          if (jos == null) {
              throw new IllegalStateException();
          }
  
          // persist all the GBeans in this Configuration
          try {
              config.setAttribute("GBeanState", Configuration.storeGBeans(gbeans));
          } catch (Exception e) {
              throw new DeploymentException("Unable to persist GBeans", e);
          }
  
          // save the persisted form in the archive
          jos.putNextEntry(new ZipEntry("META-INF/config.ser"));
          ObjectOutputStream oos = new ObjectOutputStream(jos);
          try {
              Configuration.storeGMBeanState(config, oos);
          } catch (IOException e) {
              throw e;
          } catch (Exception e) {
              throw new DeploymentException("Unable to save Configuration state", e);
          }
          oos.flush();
          jos.closeEntry();
      }
  }
  
  
  
  1.1                  incubator-geronimo/modules/deployment/src/java/org/apache/geronimo/deployment/service/ServiceConfigBuilder.java
  
  Index: ServiceConfigBuilder.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.deployment.service;
  
  import java.beans.PropertyEditor;
  import java.io.BufferedOutputStream;
  import java.io.File;
  import java.io.FileOutputStream;
  import java.io.IOException;
  import java.net.URI;
  import java.net.URISyntaxException;
  import java.net.URL;
  import java.util.Collections;
  import java.util.HashSet;
  import java.util.Set;
  import java.util.jar.Attributes;
  import java.util.jar.JarOutputStream;
  import java.util.jar.Manifest;
  import javax.management.MalformedObjectNameException;
  import javax.management.ObjectName;
  
  import org.apache.geronimo.common.propertyeditor.PropertyEditors;
  import org.apache.geronimo.deployment.ConfigurationBuilder;
  import org.apache.geronimo.deployment.DeploymentContext;
  import org.apache.geronimo.deployment.DeploymentException;
  import org.apache.geronimo.deployment.xbeans.AttributeType;
  import org.apache.geronimo.deployment.xbeans.ConfigurationDocument;
  import org.apache.geronimo.deployment.xbeans.ConfigurationType;
  import org.apache.geronimo.deployment.xbeans.DependencyType;
  import org.apache.geronimo.deployment.xbeans.GbeanType;
  import org.apache.geronimo.deployment.xbeans.ReferenceType;
  import org.apache.geronimo.deployment.xbeans.ReferencesType;
  import org.apache.geronimo.gbean.GBeanInfo;
  import org.apache.geronimo.gbean.GBeanInfoFactory;
  import org.apache.geronimo.gbean.GConstructorInfo;
  import org.apache.geronimo.gbean.GReferenceInfo;
  import org.apache.geronimo.gbean.jmx.GBeanMBean;
  import org.apache.geronimo.kernel.Kernel;
  import org.apache.geronimo.kernel.config.InvalidConfigException;
  import org.apache.geronimo.kernel.repository.Repository;
  import org.apache.xmlbeans.XmlObject;
  
  /**
   *
   *
   * @version $Revision: 1.1 $ $Date: 2004/02/12 18:27:39 $
   */
  public class ServiceConfigBuilder implements ConfigurationBuilder {
      private final Repository repository;
      private final Kernel kernel;
  
      public ServiceConfigBuilder(Repository repository, Kernel kernel) {
          this.repository = repository;
          this.kernel = kernel;
      }
  
      public boolean canConfigure(XmlObject plan) {
          return plan instanceof ConfigurationDocument;
      }
  
      public void buildConfiguration(File outfile, XmlObject plan, boolean install) throws IOException, DeploymentException {
          ConfigurationType configType = ((ConfigurationDocument) plan).getConfiguration();
          URI configID;
          try {
              configID = new URI(configType.getConfigId());
          } catch (URISyntaxException e) {
              throw new DeploymentException("Invalid configId " + configType.getConfigId(), e);
          }
          URI parentID;
          if (configType.isSetParentId()) {
              try {
                  parentID = new URI(configType.getParentId());
              } catch (URISyntaxException e) {
                  throw new DeploymentException("Invalid parentId " + configType.getParentId(), e);
              }
          } else {
              parentID = null;
          }
  
          Manifest manifest = new Manifest();
          manifest.getMainAttributes().putValue(Attributes.Name.MANIFEST_VERSION.toString(), "1.0");
          // @todo support attributes in plan to make CARfile executable
  
          FileOutputStream fos = new FileOutputStream(outfile);
          try {
              JarOutputStream os = new JarOutputStream(new BufferedOutputStream(fos), manifest);
              DeploymentContext context = null;
              try {
                  context = new DeploymentContext(os, configID, parentID, kernel);
              } catch (MalformedObjectNameException e) {
                  throw new DeploymentException(e);
              }
              addIncludes(context, configType);
              addDependencies(context, configType);
              ClassLoader cl = context.getClassLoader(repository);
              addGBeans(context, configType, cl);
              context.close();
              os.flush();
          } finally {
              fos.close();
          }
  
          try {
              if (install) {
                  kernel.install(outfile.toURL());
              }
          } catch (InvalidConfigException e) {
              // unlikely as we just built this
              throw new DeploymentException(e);
          }
      }
  
      private void addIncludes(DeploymentContext context, ConfigurationType configType) throws DeploymentException {
          DependencyType[] includes = configType.getIncludeArray();
          for (int i = 0; i < includes.length; i++) {
              DependencyType include = includes[i];
              URI uri = getDependencyURI(include);
              String name = uri.toString();
              int idx = name.lastIndexOf('/');
              if (idx != -1) {
                  name = name.substring(idx+1);
              }
              URI path;
              try {
                  path = new URI(name);
              } catch (URISyntaxException e) {
                  throw new DeploymentException("Unable to generate path for include: "+uri, e);
              }
              try {
                  URL url = repository.getURL(uri);
                  context.addInclude(path, url);
              } catch (IOException e) {
                  throw new DeploymentException("Unable to add include: "+uri, e);
              }
          }
      }
  
      private void addDependencies(DeploymentContext context, ConfigurationType configType) throws DeploymentException {
          DependencyType[] deps = configType.getDependencyArray();
          for (int i = 0; i < deps.length; i++) {
              context.addDependency(getDependencyURI(deps[i]));
          }
      }
  
      private URI getDependencyURI(DependencyType dep) throws DeploymentException {
          URI uri;
          if (dep.isSetUri()) {
              try {
                  uri = new URI(dep.getUri());
              } catch (URISyntaxException e) {
                  throw new DeploymentException("Invalid dependency URI " + dep.getUri(), e);
              }
          } else {
              // @todo support more than just jars
              String id = dep.getGroupId() + "/jars/" + dep.getArtifactId() + '-' + dep.getVersion() + ".jar";
              try {
                  uri = new URI(id);
              } catch (URISyntaxException e) {
                  throw new DeploymentException("Unable to construct URI for groupId=" + dep.getGroupId() + ", artifactId=" + dep.getArtifactId() + ", version=" + dep.getVersion(), e);
              }
          }
          return uri;
      }
  
      private void addGBeans(DeploymentContext context, ConfigurationType configType, ClassLoader cl) throws DeploymentException {
          GbeanType[] gbeans = configType.getGbeanArray();
          for (int i = 0; i < gbeans.length; i++) {
              GbeanType gbean = gbeans[i];
              ObjectName name;
              try {
                  name = new ObjectName(gbean.getName());
              } catch (MalformedObjectNameException e) {
                  throw new DeploymentException("Invalid ObjectName " + gbean.getName(), e);
              }
              String className = gbean.getClass1();
              GBeanMBean mbean;
              try {
                  mbean = new GBeanMBean(className, cl);
              } catch (Exception e) {
                  throw new DeploymentException("Unable to create GBean from class " + className, e);
              }
  
              // set up attributes
              AttributeType[] attrs = gbean.getAttributeArray();
              for (int j = 0; j < attrs.length; j++) {
                  AttributeType attr = attrs[j];
                  String attrName = attr.getName();
                  String type = attr.getType();
                  Object value = attr.getStringValue();
                  try {
                      // @todo we should not need all of common just for this
                      PropertyEditor editor = PropertyEditors.findEditor(type);
                      if (editor != null) {
                          editor.setAsText((String) value);
                          value = editor.getValue();
                      }
                  } catch (ClassNotFoundException e) {
                      throw new DeploymentException("Unable to find PropertyEditor for " + type, e);
                  }
                  try {
                      mbean.setAttribute(attrName, value);
                  } catch (Exception e) {
                      throw new DeploymentException("Unable to set attribute " + attrName, e);
                  }
              }
  
              // set up all single pattern references
              ReferenceType[] refs = gbean.getReferenceArray();
              for (int j = 0; j < refs.length; j++) {
                  ReferenceType ref = refs[j];
                  String refName = ref.getName();
                  String pattern = ref.getStringValue();
                  try {
                      mbean.setReferencePatterns(refName, Collections.singleton(new ObjectName(pattern)));
                  } catch (MalformedObjectNameException e) {
                      throw new DeploymentException("Invalid pattern for reference " + refName, e);
                  }
              }
  
              // set up app multi-patterned references
              ReferencesType[] refs2 = gbean.getReferencesArray();
              for (int j = 0; j < refs2.length; j++) {
                  ReferencesType type = refs2[j];
                  String refName = type.getName();
                  String[] patterns = type.getPatternArray();
                  Set patternNames = new HashSet(patterns.length);
                  for (int k = 0; k < patterns.length; k++) {
                      try {
                          patternNames.add(new ObjectName(patterns[k]));
                      } catch (MalformedObjectNameException e) {
                          throw new DeploymentException("Invalid pattern for reference " + refName + " : " + patterns[k], e);
                      }
                  }
                  mbean.setReferencePatterns(refName, patternNames);
              }
  
              context.addGBean(name, mbean);
          }
      }
  
      public static final GBeanInfo GBEAN_INFO;
  
      static {
          GBeanInfoFactory infoFactory = new GBeanInfoFactory(ServiceConfigBuilder.class);
          infoFactory.addInterface(ConfigurationBuilder.class);
          infoFactory.addReference(new GReferenceInfo("Repository", Repository.class));
          infoFactory.addReference(new GReferenceInfo("Kernel", Kernel.class));
          infoFactory.setConstructor(new GConstructorInfo(
                  new String[]{"Repository", "Kernel"},
                  new Class[]{Repository.class, Kernel.class}
          ));
          GBEAN_INFO = infoFactory.getBeanInfo();
      }
  
      public static GBeanInfo getGBeanInfo() {
          return GBEAN_INFO;
      }
  }
  
  
  
  1.2       +10 -5     incubator-geronimo/modules/deployment/src/schema/geronimo-common.xsd
  
  Index: geronimo-common.xsd
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/deployment/src/schema/geronimo-common.xsd,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- geronimo-common.xsd	9 Feb 2004 07:17:31 -0000	1.1
  +++ geronimo-common.xsd	12 Feb 2004 18:27:39 -0000	1.2
  @@ -16,11 +16,14 @@
       </xs:annotation>
   
       <xs:complexType name="dependencyType">
  -        <xs:sequence>
  -            <xs:element name="groupId" type="xs:string"/>
  -            <xs:element name="artifactId" type="xs:string"/>
  -            <xs:element name="version" type="xs:string"/>
  -        </xs:sequence>
  +        <xs:choice>
  +            <xs:sequence>
  +                <xs:element name="groupId" type="xs:string"/>
  +                <xs:element name="artifactId" type="xs:string"/>
  +                <xs:element name="version" type="xs:string"/>
  +            </xs:sequence>
  +            <xs:element name="uri" type="xs:string"/> 
  +        </xs:choice>
       </xs:complexType>
   
       <xs:complexType name="gbeanType">
  @@ -37,6 +40,7 @@
           <xs:simpleContent>
               <xs:extension base="xs:string">
                   <xs:attribute name="name" type="xs:string"/>
  +                <xs:attribute name="type" type="xs:string"/>
               </xs:extension>
           </xs:simpleContent>
       </xs:complexType>
  @@ -53,5 +57,6 @@
           <xs:sequence>
               <xs:element name="pattern" type="xs:string" minOccurs="1" maxOccurs="unbounded"/>
           </xs:sequence>
  +        <xs:attribute name="name" type="xs:string"/>
       </xs:complexType>
   </xs:schema>
  
  
  
  1.2       +1 -0      incubator-geronimo/modules/deployment/src/schema/geronimo-config.xsd
  
  Index: geronimo-config.xsd
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/deployment/src/schema/geronimo-config.xsd,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- geronimo-config.xsd	9 Feb 2004 07:17:31 -0000	1.1
  +++ geronimo-config.xsd	12 Feb 2004 18:27:39 -0000	1.2
  @@ -21,6 +21,7 @@
   
       <xs:complexType name="configurationType">
           <xs:sequence>
  +            <xs:element name="include" type="dependencyType" minOccurs="0" maxOccurs="unbounded"/>
               <xs:element name="dependency" type="dependencyType" minOccurs="0" maxOccurs="unbounded"/>
               <xs:element name="gbean" type="gbeanType" minOccurs="1" maxOccurs="unbounded"/>
           </xs:sequence>
  
  
  
  1.2       +2 -2      incubator-geronimo/modules/deployment/src/test-resources/services/plan1.xml
  
  Index: plan1.xml
  ===================================================================
  RCS file: /home/cvs/incubator-geronimo/modules/deployment/src/test-resources/services/plan1.xml,v
  retrieving revision 1.1
  retrieving revision 1.2
  diff -u -r1.1 -r1.2
  --- plan1.xml	9 Feb 2004 07:17:31 -0000	1.1
  +++ plan1.xml	12 Feb 2004 18:27:39 -0000	1.2
  @@ -11,8 +11,8 @@
       </dependency>
       
       <gbean name="geronimo.test:name=MyMockGMBean" class="org.apache.geronimo.deployment.MockGBean">
  -        <attribute name="Value">1234</attribute>
  -        <attribute name="IntValue">1234</attribute>
  +        <attribute name="Value" type="java.lang.String">1234</attribute>
  +        <attribute name="IntValue" type="int">1234</attribute>
           <reference name="MockEndpoint">geronimo.test:name=MyMockGMBean</reference>
       </gbean>
   </configuration>