You are viewing a plain text version of this content. The canonical link for it is here.
Posted to scm@geronimo.apache.org by dj...@apache.org on 2005/08/21 09:32:02 UTC

svn commit: r234156 - in /geronimo/trunk/modules: jetty/src/java/org/apache/geronimo/jetty/ jetty/src/test/org/apache/geronimo/jetty/ tomcat/src/java/org/apache/geronimo/tomcat/ tomcat/src/test/org/apache/geronimo/tomcat/ webservices/src/java/org/apach...

Author: djencks
Date: Sun Aug 21 00:31:50 2005
New Revision: 234156

URL: http://svn.apache.org/viewcvs?rev=234156&view=rev
Log:
GERONIMO-666 and part of GERONIMO-845.  Enable virtual host[] support for ejb web services

Modified:
    geronimo/trunk/modules/jetty/src/java/org/apache/geronimo/jetty/JettyContainerImpl.java
    geronimo/trunk/modules/jetty/src/java/org/apache/geronimo/jetty/JettyWebAppContext.java
    geronimo/trunk/modules/jetty/src/test/org/apache/geronimo/jetty/ContainerTest.java
    geronimo/trunk/modules/tomcat/src/java/org/apache/geronimo/tomcat/TomcatContainer.java
    geronimo/trunk/modules/tomcat/src/test/org/apache/geronimo/tomcat/ContainerTest.java
    geronimo/trunk/modules/webservices/src/java/org/apache/geronimo/webservices/SoapHandler.java

Modified: geronimo/trunk/modules/jetty/src/java/org/apache/geronimo/jetty/JettyContainerImpl.java
URL: http://svn.apache.org/viewcvs/geronimo/trunk/modules/jetty/src/java/org/apache/geronimo/jetty/JettyContainerImpl.java?rev=234156&r1=234155&r2=234156&view=diff
==============================================================================
--- geronimo/trunk/modules/jetty/src/java/org/apache/geronimo/jetty/JettyContainerImpl.java (original)
+++ geronimo/trunk/modules/jetty/src/java/org/apache/geronimo/jetty/JettyContainerImpl.java Sun Aug 21 00:31:50 2005
@@ -17,37 +17,37 @@
 
 package org.apache.geronimo.jetty;
 
+import java.util.ArrayList;
 import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.Iterator;
 import java.util.Hashtable;
+import java.util.Iterator;
 import java.util.List;
-import java.util.ArrayList;
-
-import javax.management.ObjectName;
+import java.util.Map;
+import java.util.Set;
 import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.geronimo.gbean.GBeanData;
 import org.apache.geronimo.gbean.GBeanInfo;
 import org.apache.geronimo.gbean.GBeanInfoBuilder;
 import org.apache.geronimo.gbean.GBeanLifecycle;
 import org.apache.geronimo.gbean.GBeanQuery;
-import org.apache.geronimo.gbean.GBeanData;
-import org.apache.geronimo.webservices.SoapHandler;
-import org.apache.geronimo.webservices.WebServiceContainer;
-import org.apache.geronimo.kernel.Kernel;
-import org.apache.geronimo.kernel.GBeanNotFoundException;
-import org.apache.geronimo.kernel.NoSuchOperationException;
-import org.apache.geronimo.kernel.management.StateManageable;
-import org.apache.geronimo.kernel.management.State;
 import org.apache.geronimo.j2ee.j2eeobjectnames.NameFactory;
 import org.apache.geronimo.j2ee.management.impl.Util;
+import org.apache.geronimo.jetty.connector.AJP13Connector;
 import org.apache.geronimo.jetty.connector.HTTPConnector;
 import org.apache.geronimo.jetty.connector.HTTPSConnector;
-import org.apache.geronimo.jetty.connector.AJP13Connector;
+import org.apache.geronimo.kernel.GBeanNotFoundException;
+import org.apache.geronimo.kernel.Kernel;
 import org.apache.geronimo.system.serverinfo.ServerInfo;
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.mortbay.http.*;
+import org.apache.geronimo.webservices.SoapHandler;
+import org.apache.geronimo.webservices.WebServiceContainer;
+import org.mortbay.http.HttpContext;
+import org.mortbay.http.HttpListener;
+import org.mortbay.http.RequestLog;
+import org.mortbay.http.UserRealm;
 import org.mortbay.jetty.Server;
 
 /**
@@ -162,8 +162,9 @@
         server.removeRealm(realm.getName());
     }
 
-    public void addWebService(String contextPath, WebServiceContainer webServiceContainer, String securityRealmName, String realmName, String transportGuarantee, String authMethod, ClassLoader classLoader) throws Exception {
+    public void addWebService(String contextPath, String[] virtualHosts, WebServiceContainer webServiceContainer, String securityRealmName, String realmName, String transportGuarantee, String authMethod, ClassLoader classLoader) throws Exception {
         JettyEJBWebServiceContext webServiceContext = new JettyEJBWebServiceContext(contextPath, webServiceContainer, securityRealmName, realmName, transportGuarantee, authMethod, classLoader);
+        webServiceContext.setHosts(virtualHosts);
         addContext(webServiceContext);
         webServiceContext.start();
         webServices.put(contextPath, webServiceContext);
@@ -198,7 +199,7 @@
         for (Iterator it = names.iterator(); it.hasNext();) {
             ObjectName name = (ObjectName) it.next();
             try {
-                if(kernel.getAttribute(name, "protocol").equals(protocol)) {
+                if (kernel.getAttribute(name, "protocol").equals(protocol)) {
                     result.add(name.getCanonicalName());
                 }
             } catch (Exception e) {
@@ -215,7 +216,7 @@
         GBeanQuery query = new GBeanQuery(null, JettyWebConnector.class.getName());
         Set names = kernel.listGBeans(query);
         String[] result = new String[names.size()];
-        int i=0;
+        int i = 0;
         for (Iterator it = names.iterator(); it.hasNext();) {
             ObjectName name = (ObjectName) it.next();
             result[i++] = name.getCanonicalName();
@@ -231,18 +232,18 @@
     public String addConnector(String uniqueName, String protocol, String host, int port) {
         ObjectName name = getConnectorName(protocol, uniqueName);
         GBeanData connector;
-        if(protocol.equals(PROTOCOL_HTTP)) {
+        if (protocol.equals(PROTOCOL_HTTP)) {
             connector = new GBeanData(name, HTTPConnector.GBEAN_INFO);
-        } else if(protocol.equals(PROTOCOL_HTTPS)) {
+        } else if (protocol.equals(PROTOCOL_HTTPS)) {
             connector = new GBeanData(name, HTTPSConnector.GBEAN_INFO);
             GBeanQuery query = new GBeanQuery(null, ServerInfo.class.getName());
             Set set = kernel.listGBeans(query);
-            connector.setReferencePattern("ServerInfo", (ObjectName)set.iterator().next());
+            connector.setReferencePattern("ServerInfo", (ObjectName) set.iterator().next());
             //todo: default HTTPS settings
-        } else if(protocol.equals(PROTOCOL_AJP)) {
+        } else if (protocol.equals(PROTOCOL_AJP)) {
             connector = new GBeanData(name, AJP13Connector.GBEAN_INFO);
         } else {
-            throw new IllegalArgumentException("Invalid protocol '"+protocol+"'");
+            throw new IllegalArgumentException("Invalid protocol '" + protocol + "'");
         }
         connector.setAttribute("host", host);
         connector.setAttribute("port", new Integer(port));
@@ -265,11 +266,11 @@
         table.put(NameFactory.J2EE_SERVER, myName.getKeyProperty(NameFactory.J2EE_SERVER));
         table.put(NameFactory.J2EE_MODULE, myName.getKeyProperty(NameFactory.J2EE_MODULE));
         table.put(NameFactory.J2EE_TYPE, myName.getKeyProperty(NameFactory.J2EE_TYPE));
-        table.put(NameFactory.J2EE_NAME, "JettyWebConnector-"+protocol+"-"+uniqueName);
+        table.put(NameFactory.J2EE_NAME, "JettyWebConnector-" + protocol + "-" + uniqueName);
         try {
             return ObjectName.getInstance(myName.getDomain(), table);
         } catch (MalformedObjectNameException e) {
-            throw new IllegalStateException("Never should have failed: "+e.getMessage());
+            throw new IllegalStateException("Never should have failed: " + e.getMessage());
         }
     }
 
@@ -283,7 +284,7 @@
         try {
             name = ObjectName.getInstance(objectName);
         } catch (MalformedObjectNameException e) {
-            throw new IllegalArgumentException("Invalid object name '"+objectName+"': "+e.getMessage());
+            throw new IllegalArgumentException("Invalid object name '" + objectName + "': " + e.getMessage());
         }
         try {
             GBeanInfo info = kernel.getGBeanInfo(name);
@@ -291,17 +292,17 @@
             Set intfs = info.getInterfaces();
             for (Iterator it = intfs.iterator(); it.hasNext();) {
                 String intf = (String) it.next();
-                if(intf.equals(JettyWebConnector.class.getName())) {
+                if (intf.equals(JettyWebConnector.class.getName())) {
                     found = true;
                 }
             }
-            if(!found) {
+            if (!found) {
                 throw new GBeanNotFoundException(name);
             }
             ObjectName config = Util.getConfiguration(kernel, name);
             kernel.invoke(config, "removeGBean", new Object[]{name}, new String[]{ObjectName.class.getName()});
         } catch (GBeanNotFoundException e) {
-            log.warn("No such GBean '"+objectName+"'"); //todo: what if we want to remove a failed GBean?
+            log.warn("No such GBean '" + objectName + "'"); //todo: what if we want to remove a failed GBean?
         } catch (Exception e) {
             log.error(e);
         }
@@ -365,7 +366,7 @@
 
         infoBuilder.addInterface(SoapHandler.class);
         infoBuilder.addInterface(JettyContainer.class);
-        infoBuilder.setConstructor(new String[]{"kernel","objectName"});
+        infoBuilder.setConstructor(new String[]{"kernel", "objectName"});
 
         GBEAN_INFO = infoBuilder.getBeanInfo();
     }

Modified: geronimo/trunk/modules/jetty/src/java/org/apache/geronimo/jetty/JettyWebAppContext.java
URL: http://svn.apache.org/viewcvs/geronimo/trunk/modules/jetty/src/java/org/apache/geronimo/jetty/JettyWebAppContext.java?rev=234156&r1=234155&r2=234156&view=diff
==============================================================================
--- geronimo/trunk/modules/jetty/src/java/org/apache/geronimo/jetty/JettyWebAppContext.java (original)
+++ geronimo/trunk/modules/jetty/src/java/org/apache/geronimo/jetty/JettyWebAppContext.java Sun Aug 21 00:31:50 2005
@@ -128,6 +128,7 @@
     public JettyWebAppContext(String objectName,
                               String originalSpecDD,
                               URI uri,
+                              String[] virtualHosts,
                               Map componentContext,
                               OnlineUserTransaction userTransaction,
                               ClassLoader classLoader,
@@ -207,6 +208,7 @@
         this.webClassLoader = new JettyClassLoader(urls, webAppRootURL, classLoader, contextPriorityClassLoader);
         setClassLoader(this.webClassLoader);
 
+        setHosts(virtualHosts);
         handler = new WebApplicationHandler();
         addHandler(handler);
 
@@ -536,6 +538,7 @@
 
 
         infoBuilder.addAttribute("uri", URI.class, true);
+        infoBuilder.addAttribute("virtualHosts", String[].class, true);
         infoBuilder.addAttribute("componentContext", Map.class, true);
         infoBuilder.addAttribute("userTransaction", OnlineUserTransaction.class, true);
         infoBuilder.addAttribute("classLoader", ClassLoader.class, false);
@@ -577,6 +580,7 @@
             "objectName",
             "deploymentDescriptor",
             "uri",
+            "virtualHosts",
             "componentContext",
             "userTransaction",
             "classLoader",

Modified: geronimo/trunk/modules/jetty/src/test/org/apache/geronimo/jetty/ContainerTest.java
URL: http://svn.apache.org/viewcvs/geronimo/trunk/modules/jetty/src/test/org/apache/geronimo/jetty/ContainerTest.java?rev=234156&r1=234155&r2=234156&view=diff
==============================================================================
--- geronimo/trunk/modules/jetty/src/test/org/apache/geronimo/jetty/ContainerTest.java (original)
+++ geronimo/trunk/modules/jetty/src/test/org/apache/geronimo/jetty/ContainerTest.java Sun Aug 21 00:31:50 2005
@@ -83,7 +83,7 @@
 
         String contextPath = "/foo/webservice.ws";
         MockWebServiceContainer webServiceInvoker = new MockWebServiceContainer();
-        kernel.invoke(containerName, "addWebService", new Object[] {contextPath, webServiceInvoker, null, null, null, null,cl}, new String[] {String.class.getName(), WebServiceContainer.class.getName(), String.class.getName(), String.class.getName(), String.class.getName(), String.class.getName(), ClassLoader.class.getName()});
+        kernel.invoke(containerName, "addWebService", new Object[] {contextPath, null, webServiceInvoker, null, null, null, null,cl}, new String[] {String.class.getName(), String[].class.getName(), WebServiceContainer.class.getName(), String.class.getName(), String.class.getName(), String.class.getName(), String.class.getName(), ClassLoader.class.getName()});
 
         HttpURLConnection connection = (HttpURLConnection) new URL("http://localhost:5678" + contextPath).openConnection();
         try {

Modified: geronimo/trunk/modules/tomcat/src/java/org/apache/geronimo/tomcat/TomcatContainer.java
URL: http://svn.apache.org/viewcvs/geronimo/trunk/modules/tomcat/src/java/org/apache/geronimo/tomcat/TomcatContainer.java?rev=234156&r1=234155&r2=234156&view=diff
==============================================================================
--- geronimo/trunk/modules/tomcat/src/java/org/apache/geronimo/tomcat/TomcatContainer.java (original)
+++ geronimo/trunk/modules/tomcat/src/java/org/apache/geronimo/tomcat/TomcatContainer.java Sun Aug 21 00:31:50 2005
@@ -54,10 +54,10 @@
 
 /**
  * Apache Tomcat GBean
- * 
+ *
  * @see http://wiki.apache.org/geronimo/Tomcat
  * @see http://nagoya.apache.org/jira/browse/GERONIMO-215
- * 
+ *
  * @version $Rev$ $Date$
  */
 public class TomcatContainer implements SoapHandler, GBeanLifecycle, TomcatWebContainer {
@@ -114,7 +114,7 @@
 
         if (catalinaHome == null)
             catalinaHome = DEFAULT_CATALINA_HOME;
-        
+
         setCatalinaHome(serverInfo.resolvePath(catalinaHome));
 
         if (classLoader == null){
@@ -126,7 +126,7 @@
         }
 
         this.classLoader = classLoader;
-        
+
         this.engine = (Engine)engineGBean.getInternalObject();
     }
 
@@ -139,14 +139,14 @@
 
     /**
      * Instantiate and start up Tomcat's Embedded class
-     * 
+     *
      * See org.apache.catalina.startup.Embedded for details (TODO: provide the link to the javadoc)
      */
     public void doStart() throws Exception {
         log.debug("doStart()");
 
         log.info("Endorsed Dirs set to:" + System.getProperty("java.endorsed.dirs"));
- 
+
         // The comments are from the javadoc of the Embedded class
 
         // 1. Instantiate a new instance of this class.
@@ -182,7 +182,7 @@
             }
             hosts[i].addChild(defaultContext);
         }
-        
+
         // 6. Call addEngine() to attach this Engine to the set of defined
         // Engines for this object.
         embedded.addEngine(engine);
@@ -201,11 +201,11 @@
 
     /**
      * Creates and adds the context to the running host
-     * 
+     *
      * It simply delegates the call to Tomcat's Embedded and Host classes
-     * 
+     *
      * @param ctx the context to be added
-     * 
+     *
      * @see org.apache.catalina.startup.Embedded
      * @see org.apache.catalina.Host
      */
@@ -214,30 +214,30 @@
 
         // Set the context for the Tomcat implementation
         ctx.setContext(anotherCtxObj);
-        
+
         // Have the context to set its properties if its a GeronimoStandardContext
-        if (anotherCtxObj instanceof GeronimoStandardContext) 
+        if (anotherCtxObj instanceof GeronimoStandardContext)
             ((GeronimoStandardContext)anotherCtxObj).setContextProperties(ctx);
 
         //Was a virtual server defined?
         String virtualServer = ctx.getVirtualServer();
         if (virtualServer == null)
             virtualServer = engine.getDefaultHost();
-        
+
         Container host = engine.findChild(virtualServer);
         if (host == null){
             throw new IllegalArgumentException("Invalid virtual host '" + virtualServer +"'.  Do you have a matchiing Host entry in the plan?");
         }
-        
+
         //Get the security-realm-name if the is one
         String securityRealmName = null;
         SecurityHolder secHolder = ctx.getSecurityHolder();
         if (secHolder != null)
             securityRealmName = secHolder.getSecurityRealm();
-        
+
         if (ctx.getRealm() != null){
             Realm realm = ctx.getRealm();
-                       
+
             //Allow for the <security-realm-name> override from the
             //geronimo-web.xml file to be used if our Realm is a JAAS type
             if (securityRealmName != null){
@@ -267,7 +267,7 @@
             } else {
                 anotherCtxObj.setRealm(realm);
             }
-        }            
+        }
 
         host.addChild(anotherCtxObj);
     }
@@ -279,7 +279,7 @@
             embedded.removeContext(context);
 
     }
-    
+
     public void setCatalinaHome(String catalinaHome) {
         System.setProperty("catalina.home", catalinaHome);
     }
@@ -291,23 +291,22 @@
     public void removeConnector(Connector connector) {
         embedded.removeConnector(connector);
     }
-    
-    public void addWebService(String contextPath, WebServiceContainer webServiceContainer, String securityRealmName, String realmName, String transportGuarantee, String authMethod, ClassLoader classLoader) throws Exception {
+
+    public void addWebService(String contextPath, String[] virtualHosts, WebServiceContainer webServiceContainer, String securityRealmName, String realmName, String transportGuarantee, String authMethod, ClassLoader classLoader) throws Exception {
         Context webServiceContext = embedded.createEJBWebServiceContext(contextPath, webServiceContainer, securityRealmName, realmName, transportGuarantee, authMethod, classLoader);
 
-        //TODO When OpenEJB supports virtual hosts, remove the next line
-        String virtualServer = engine.getDefaultHost();
+        String virtualServer;
+        if (virtualHosts != null && virtualHosts.length > 0) {
+            virtualServer = virtualHosts[0];
+        } else {
+            virtualServer = engine.getDefaultHost();
+        }
 
-        //TODO When OpenEJB supports virtual hosts, uncomment the code below.  The 
-        //virtualServer variable should be a String parameter from this function call
-        //if (virtualServer == null)
-        //    virtualServer = engine.getDefaultHost();
-        
         Container host = engine.findChild(virtualServer);
         if (host == null){
             throw new IllegalArgumentException("Invalid virtual host '" + virtualServer +"'.  Do you have a matchiing Host entry in the plan?");
         }
-        
+
         host.addChild(webServiceContext);
         webServices.put(contextPath, webServiceContext);
     }
@@ -325,7 +324,7 @@
 
     private TomcatClassLoader createRootClassLoader(File baseDir, ClassLoader cl) throws Exception{
         ArrayList urls = new ArrayList();
-        
+
         File webInfDir = new File(baseDir, "WEB-INF");
 
         // check for a classes dir
@@ -349,8 +348,8 @@
                     urls.add(lib.toURL());
                 }
             }
-        } 
-        
+        }
+
         return new TomcatClassLoader((URL[])urls.toArray(new URL[0]), null, cl, false);
     }
 

Modified: geronimo/trunk/modules/tomcat/src/test/org/apache/geronimo/tomcat/ContainerTest.java
URL: http://svn.apache.org/viewcvs/geronimo/trunk/modules/tomcat/src/test/org/apache/geronimo/tomcat/ContainerTest.java?rev=234156&r1=234155&r2=234156&view=diff
==============================================================================
--- geronimo/trunk/modules/tomcat/src/test/org/apache/geronimo/tomcat/ContainerTest.java (original)
+++ geronimo/trunk/modules/tomcat/src/test/org/apache/geronimo/tomcat/ContainerTest.java Sun Aug 21 00:31:50 2005
@@ -92,7 +92,7 @@
 
        String contextPath = "/foo/webservice.ws";
        MockWebServiceContainer webServiceInvoker = new MockWebServiceContainer();
-       kernel.invoke(containerName, "addWebService", new Object[] {contextPath, webServiceInvoker, null, null, null, null, cl}, new String[] {String.class.getName(), WebServiceContainer.class.getName(), String.class.getName(), String.class.getName(), String.class.getName(), String.class.getName(), ClassLoader.class.getName()});
+       kernel.invoke(containerName, "addWebService", new Object[] {contextPath, null, webServiceInvoker, null, null, null, null, cl}, new String[] {String.class.getName(), String[].class.getName(), WebServiceContainer.class.getName(), String.class.getName(), String.class.getName(), String.class.getName(), String.class.getName(), ClassLoader.class.getName()});
 
        HttpURLConnection connection = (HttpURLConnection) new URL("http://localhost:8080" + contextPath).openConnection();
        try {
@@ -112,10 +112,10 @@
            assertEquals(HttpURLConnection.HTTP_NOT_FOUND, connection.getResponseCode());
            connection.disconnect();
        }
-    
+
        tearDownWeb();
    }
-   
+
    public void testSecureWebServiceHandler() throws Exception {
 
        setUpWeb();
@@ -124,10 +124,10 @@
        assertEquals(State.RUNNING_INDEX, kernel.getGBeanState(containerName));
 
        setUpSecurity();
-       
+
        String contextPath = "/foo/webservice.ws";
        MockWebServiceContainer webServiceInvoker = new MockWebServiceContainer();
-       kernel.invoke(containerName, "addWebService", new Object[] {contextPath, webServiceInvoker, "Geronimo", "Geronimo", "NONE", "BASIC",cl}, new String[] {String.class.getName(), WebServiceContainer.class.getName(), String.class.getName(), String.class.getName(), String.class.getName(), String.class.getName(), ClassLoader.class.getName()});
+       kernel.invoke(containerName, "addWebService", new Object[] {contextPath, null, webServiceInvoker, "Geronimo", "Geronimo", "NONE", "BASIC",cl}, new String[] {String.class.getName(), String[].class.getName(), WebServiceContainer.class.getName(), String.class.getName(), String.class.getName(), String.class.getName(), String.class.getName(), ClassLoader.class.getName()});
 
        //Veryify its secured
        HttpURLConnection connection = (HttpURLConnection) new URL("http://localhost:8080" + contextPath).openConnection();
@@ -139,7 +139,7 @@
        } finally {
            connection.disconnect();
        }
-       
+
        //Authenticate
        connection = (HttpURLConnection) new URL("http://localhost:8080" + contextPath).openConnection();
        String authentication = (new BASE64Encoder()).encode(("alan:starcraft").getBytes());
@@ -161,7 +161,7 @@
            assertEquals(HttpURLConnection.HTTP_NOT_FOUND, connection.getResponseCode());
            connection.disconnect();
        }
-       
+
        tearDownSecurity();
        tearDownWeb();
    }
@@ -237,7 +237,7 @@
        stop(securityServiceName);
        stop(loginConfigurationName);
    }
-   
+
    private void setUpWeb() throws Exception{
        containerName = NameFactory.getWebComponentName(null, null, null, null, "tomcatContainer", "WebResource", moduleContext);
        connectorName = NameFactory.getWebComponentName(null, null, null, null, "tomcatConnector", "WebResource", moduleContext);
@@ -249,13 +249,13 @@
        hostName = NameFactory.getWebComponentName(null, null, null, null, "tomcatHost", "WebResource", moduleContext);
        kernel = KernelFactory.newInstance().createKernel("test.kernel");
        kernel.boot();
- 
+
        //ServerInfo
        serverInfoName = new ObjectName("geronimo.system:role=ServerInfo");
        serverInfoGBean = new GBeanData(serverInfoName, BasicServerInfo.GBEAN_INFO);
        serverInfoGBean.setAttribute("baseDirectory", ".");
        start(serverInfoGBean);
-       
+
        Map initParams = new HashMap();
 
        //Default Host
@@ -266,12 +266,12 @@
        host = new GBeanData(hostName, HostGBean.GBEAN_INFO);
        host.setAttribute("className", "org.apache.catalina.core.StandardHost");
        host.setAttribute("initParams", initParams);
-       start(host);       
+       start(host);
 
        //Default Engine
 //       ReferenceCollection hosts = new TestReferenceCollection();
 //       hosts.add(host);
-       
+
        initParams.clear();
        initParams.put("name","Geronimo");
        initParams.put("defaultHost","localhost");
@@ -294,7 +294,7 @@
        connector.setReferencePattern("TomcatContainer", containerName);
        start(connector);
    }
-   
+
    private void tearDownWeb() throws Exception {
        stop(connectorName);
        stop(containerName);
@@ -332,5 +332,5 @@
            return result;
        }
 
-   }   
+   }
 }

Modified: geronimo/trunk/modules/webservices/src/java/org/apache/geronimo/webservices/SoapHandler.java
URL: http://svn.apache.org/viewcvs/geronimo/trunk/modules/webservices/src/java/org/apache/geronimo/webservices/SoapHandler.java?rev=234156&r1=234155&r2=234156&view=diff
==============================================================================
--- geronimo/trunk/modules/webservices/src/java/org/apache/geronimo/webservices/SoapHandler.java (original)
+++ geronimo/trunk/modules/webservices/src/java/org/apache/geronimo/webservices/SoapHandler.java Sun Aug 21 00:31:50 2005
@@ -21,7 +21,7 @@
  */
 public interface SoapHandler {
 
-    void addWebService(String contextPath, WebServiceContainer webServiceContainer, String securityRealmName, String realmName, String transportGuarantee, String authMethod, ClassLoader classLoader) throws Exception;
+    void addWebService(String contextPath, String[] virtualHosts, WebServiceContainer webServiceContainer, String securityRealmName, String realmName, String transportGuarantee, String authMethod, ClassLoader classLoader) throws Exception;
 
     void removeWebService(String contextPath);