You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@etch.apache.org by sc...@apache.org on 2009/02/04 16:27:53 UTC

svn commit: r740767 - in /incubator/etch/trunk: binding-csharp/compiler/src/main/resources/org/apache/etch/bindings/csharp/compiler/ binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Support/ binding-csharp/runtime/src/main/csharp/...

Author: sccomer
Date: Wed Feb  4 15:27:52 2009
New Revision: 740767

URL: http://svn.apache.org/viewvc?rev=740767&view=rev
Log:
fix for ETCH-54: TransportFactory.newTransport returns stack topped by DeliveryService which makes it hard to implement meta transports

Modified:
    incubator/etch/trunk/binding-csharp/compiler/src/main/resources/org/apache/etch/bindings/csharp/compiler/helper.vm
    incubator/etch/trunk/binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Support/DefaultServerFactory.cs
    incubator/etch/trunk/binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Support/ServerFactory.cs
    incubator/etch/trunk/binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Support/TransportFactory.cs
    incubator/etch/trunk/binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Transport/TcpTransportFactory.cs
    incubator/etch/trunk/binding-java/compiler/src/main/resources/org/apache/etch/bindings/java/compiler/helper.vm
    incubator/etch/trunk/binding-java/runtime/src/main/java/org/apache/etch/bindings/java/support/ServerFactory.java
    incubator/etch/trunk/binding-java/runtime/src/main/java/org/apache/etch/bindings/java/support/TransportFactory.java
    incubator/etch/trunk/binding-java/runtime/src/main/java/org/apache/etch/bindings/java/transport/Tcp2TransportFactory.java
    incubator/etch/trunk/binding-java/runtime/src/main/java/org/apache/etch/bindings/java/transport/TcpTransportFactory.java
    incubator/etch/trunk/tests/src/main/csharp/etch.tests/MyCuaeHelper.cs
    incubator/etch/trunk/tests/src/main/java/org/apache/etch/tests/MyCuaeHelper.java

Modified: incubator/etch/trunk/binding-csharp/compiler/src/main/resources/org/apache/etch/bindings/csharp/compiler/helper.vm
URL: http://svn.apache.org/viewvc/incubator/etch/trunk/binding-csharp/compiler/src/main/resources/org/apache/etch/bindings/csharp/compiler/helper.vm?rev=740767&r1=740766&r2=740767&view=diff
==============================================================================
--- incubator/etch/trunk/binding-csharp/compiler/src/main/resources/org/apache/etch/bindings/csharp/compiler/helper.vm (original)
+++ incubator/etch/trunk/binding-csharp/compiler/src/main/resources/org/apache/etch/bindings/csharp/compiler/helper.vm Wed Feb  4 15:27:52 2009
@@ -19,6 +19,7 @@
 
 using Org.Apache.Etch.Bindings.Csharp.Msg;
 using Org.Apache.Etch.Bindings.Csharp.Support;
+using Org.Apache.Etch.Bindings.Csharp.Transport;
 using Org.Apache.Etch.Bindings.Csharp.Util;
 
 /// <summary>
@@ -64,13 +65,17 @@
 				_implFactory = implFactory;
 			}
 		
-			public override void NewServer( DeliveryService d, ValueFactory vf )
+			public override DeliveryService NewServer(TransportMessage m, ValueFactory vf)
 			{
-				Remote${i}Client client = new Remote${i}Client( d, vf );
-				${i}Server server = _implFactory.New${i}Server( client );
-				Pool qp = (Pool) _resources[ QUEUED_POOL ];
-				Pool fp = (Pool) _resources[ FREE_POOL ];
-				new Stub${i}Server( d, server, qp, fp );
+                URL u = new URL(_uri);
+                MailboxManager r = new PlainMailboxManager(m, u, _resources);
+                DeliveryService d = new DefaultDeliveryService(r, u, _resources);
+				Remote${i}Client client = new Remote${i}Client(d, vf);
+				${i}Server server = _implFactory.New${i}Server(client);
+				Pool qp = (Pool)_resources[QUEUED_POOL];
+				Pool fp = (Pool)_resources[FREE_POOL];
+				new Stub${i}Server(d, server, qp, fp);
+				return d;
 			}
 
             public override ValueFactory NewValueFactory()
@@ -111,16 +116,22 @@
 		public static Remote${i}Server NewServer( string uri,
 			Resources resources, ${i}ClientFactory implFactory )
 		{
-			resources = InitResources( resources );
+			resources = InitResources(resources);
+			
 			ValueFactory${i} vf= new ValueFactory${i}(uri);
 			resources.Add( TransportConsts.VALUE_FACTORY, vf );
-			DeliveryService d = TransportFactory.GetTransport( uri, resources );
-			Remote${i}Server server = new Remote${i}Server( d, vf );
-			${i}Client client = implFactory.New${i}Client( server );			
-			 new Stub${i}Client( d,
+			
+			URL u = new URL(uri);
+
+            TransportMessage m = TransportFactory.GetTransport(uri, resources);
+            MailboxManager r = new PlainMailboxManager(m, u, resources);
+            DeliveryService d = new DefaultDeliveryService(r, u, resources);
+			Remote${i}Server server = new Remote${i}Server(d, vf);
+			${i}Client client = implFactory.New${i}Client(server);			
+			new Stub${i}Client(d,
 					client,
-					(Pool) resources[ QUEUED_POOL ],
-					(Pool) resources[ FREE_POOL ] );
+					(Pool) resources[QUEUED_POOL],
+					(Pool) resources[FREE_POOL]);
 	
 			return server;
 		}

Modified: incubator/etch/trunk/binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Support/DefaultServerFactory.cs
URL: http://svn.apache.org/viewvc/incubator/etch/trunk/binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Support/DefaultServerFactory.cs?rev=740767&r1=740766&r2=740767&view=diff
==============================================================================
--- incubator/etch/trunk/binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Support/DefaultServerFactory.cs (original)
+++ incubator/etch/trunk/binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Support/DefaultServerFactory.cs Wed Feb  4 15:27:52 2009
@@ -15,13 +15,15 @@
 // under the License.
 
 using System;
+using Org.Apache.Etch.Bindings.Csharp.Msg;
+using Org.Apache.Etch.Bindings.Csharp.Transport;
 using Org.Apache.Etch.Bindings.Csharp.Util;
 
 namespace Org.Apache.Etch.Bindings.Csharp.Support
 {
     abstract public class DefaultServerFactory : ServerFactory
     {
-        public DefaultServerFactory(object implFactory)
+        protected DefaultServerFactory(object implFactory)
         {
             this.implFactory = implFactory;
         }
@@ -57,9 +59,9 @@
 
         #region ServerFactory Members
 
-        abstract public void NewServer(DeliveryService d, Org.Apache.Etch.Bindings.Csharp.Msg.ValueFactory vf);
+        abstract public DeliveryService NewServer( TransportMessage m, ValueFactory vf);
 
-        abstract public Org.Apache.Etch.Bindings.Csharp.Msg.ValueFactory NewValueFactory();
+        abstract public ValueFactory NewValueFactory();
 
         #endregion
     }

Modified: incubator/etch/trunk/binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Support/ServerFactory.cs
URL: http://svn.apache.org/viewvc/incubator/etch/trunk/binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Support/ServerFactory.cs?rev=740767&r1=740766&r2=740767&view=diff
==============================================================================
--- incubator/etch/trunk/binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Support/ServerFactory.cs (original)
+++ incubator/etch/trunk/binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Support/ServerFactory.cs Wed Feb  4 15:27:52 2009
@@ -15,6 +15,7 @@
 // under the License.
 
 using Org.Apache.Etch.Bindings.Csharp.Msg;
+using Org.Apache.Etch.Bindings.Csharp.Transport;
 using Org.Apache.Etch.Bindings.Csharp.Util;
 
 namespace Org.Apache.Etch.Bindings.Csharp.Support
@@ -28,9 +29,10 @@
         /// <summary>
         /// Constructs a new server session.
         /// </summary>
-        /// <param name="d"> the delivery service to be used with the new session</param>
+        /// <param name="m"> the TransportMessage to be used with the new session</param>
         /// <param name="vf">the value factory to be used with the new session</param>
-        void NewServer( DeliveryService d, ValueFactory vf );
+        /// <returns>the constructed DeliveryService</returns>
+        DeliveryService NewServer(TransportMessage m, ValueFactory vf);
 
         /// <summary>
         /// Constructs a new instance of value factory for this session.

Modified: incubator/etch/trunk/binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Support/TransportFactory.cs
URL: http://svn.apache.org/viewvc/incubator/etch/trunk/binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Support/TransportFactory.cs?rev=740767&r1=740766&r2=740767&view=diff
==============================================================================
--- incubator/etch/trunk/binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Support/TransportFactory.cs (original)
+++ incubator/etch/trunk/binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Support/TransportFactory.cs Wed Feb  4 15:27:52 2009
@@ -29,24 +29,24 @@
 	public abstract class TransportFactory
 	{
         /// <summary>
-        /// Constructs a new Transport stack topped by a DeliveryService
+        /// Constructs a new Transport stack topped by a TransportMessage
         /// which is used by the remote service implementations to send
-        /// messages and make calls.
+        /// messages.
         /// </summary>
         /// <param name="uri">transport configuration parameters.</param>
         /// <param name="resources">additional resources needed by the stack.</param>
-        /// <returns>the DeliveryService topping the transport stack.</returns>
-        protected abstract DeliveryService NewTransport( string uri, Resources resources );
+        /// <returns>the TransportMessage topping the transport stack.</returns>
+        protected abstract TransportMessage NewTransport(string uri, Resources resources);
 
         /// <summary>
-        /// Constructs a new Transport stack topped by a DeliveryService
+        /// Constructs a new Transport stack topped by a TransportMessage
         /// which is used by the remote service implementations to send
-        /// messages and make calls.
+        /// messages.
         /// </summary>
         /// <param name="uri">transport configuration parameters.</param>
         /// <param name="resources">additional resources needed by the stack.</param>
-        /// <returns>the DeliveryService topping the transport stack.</returns>
-        public static DeliveryService GetTransport( string uri,Resources resources ) 
+        /// <returns>the TransportMessage topping the transport stack.</returns>
+        public static TransportMessage GetTransport(string uri, Resources resources) 
 	    {
 		    URL u = new URL( uri );
 		    TransportFactory f = GetTransportFactory( u.Scheme );

Modified: incubator/etch/trunk/binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Transport/TcpTransportFactory.cs
URL: http://svn.apache.org/viewvc/incubator/etch/trunk/binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Transport/TcpTransportFactory.cs?rev=740767&r1=740766&r2=740767&view=diff
==============================================================================
--- incubator/etch/trunk/binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Transport/TcpTransportFactory.cs (original)
+++ incubator/etch/trunk/binding-csharp/runtime/src/main/csharp/Org.Apache.Etch.Bindings.Csharp/Transport/TcpTransportFactory.cs Wed Feb  4 15:27:52 2009
@@ -42,7 +42,7 @@
         
         private const String SOCKET = "TcpTransportFactory.socket";
 
-        protected override DeliveryService NewTransport( string uri, Resources resources )
+        protected override TransportMessage NewTransport(string uri, Resources resources)
         {
             URL u = new URL(uri);
 
@@ -61,14 +61,14 @@
 
             m = AddFilters(m, u, resources);
 
-            MailboxManager r = new PlainMailboxManager(m, u, resources);
+            //MailboxManager r = new PlainMailboxManager(m, u, resources);
 
-            DeliveryService d = new DefaultDeliveryService(r, u, resources);
+            //DeliveryService d = new DefaultDeliveryService(r, u, resources);
 
             ValueFactory vf = (ValueFactory) resources.Get(TransportConsts.VALUE_FACTORY);
             vf.LockDynamicTypes();
 
-            return d;
+            return m;
         }
 
         protected override Transport<ServerFactory> NewListener( string uri, Resources resources,
@@ -100,7 +100,7 @@
 
             public override string ToString()
             {
-                return "TcpTransportFactory.MySessionListener" + transport;
+                return "TcpTransportFactory.MySessionListener/" + transport;
             }
 
             public void SessionAccepted(Socket socket)
@@ -111,9 +111,9 @@
                 ValueFactory vf = session.NewValueFactory();
                 r.Add(TransportConsts.VALUE_FACTORY, vf);
 
-                DeliveryService d = ttf.NewTransport(uri, r);
+                TransportMessage m = ttf.NewTransport(uri, r);
 
-                session.NewServer(d, vf);
+                DeliveryService d = session.NewServer(m, vf);
 
                 d.TransportControl(TransportConsts.START, null);
             }

Modified: incubator/etch/trunk/binding-java/compiler/src/main/resources/org/apache/etch/bindings/java/compiler/helper.vm
URL: http://svn.apache.org/viewvc/incubator/etch/trunk/binding-java/compiler/src/main/resources/org/apache/etch/bindings/java/compiler/helper.vm?rev=740767&r1=740766&r2=740767&view=diff
==============================================================================
--- incubator/etch/trunk/binding-java/compiler/src/main/resources/org/apache/etch/bindings/java/compiler/helper.vm (original)
+++ incubator/etch/trunk/binding-java/compiler/src/main/resources/org/apache/etch/bindings/java/compiler/helper.vm Wed Feb  4 15:27:52 2009
@@ -29,7 +29,12 @@
 import org.apache.etch.util.core.io.Transport;
 import org.apache.etch.bindings.java.support.TransportFactory;
 import org.apache.etch.bindings.java.support.TransportHelper;
+import org.apache.etch.bindings.java.transport.DefaultDeliveryService;
+import org.apache.etch.bindings.java.transport.MailboxManager;
+import org.apache.etch.bindings.java.transport.PlainMailboxManager;
+import org.apache.etch.bindings.java.transport.TransportMessage;
 import org.apache.etch.util.Resources;
+import org.apache.etch.util.URL;
 
 /**
  * Transport helper for ${i}. All methods are static.
@@ -62,17 +67,21 @@
 		throws Exception
 	{
 		final Resources res = initResources( resources );
-
+		final URL u = new URL( uri );
+		
 		return TransportFactory.getListener( uri, res,
 			new DefaultServerFactory( implFactory )
 			{
-				public void newServer( DeliveryService d, ValueFactory vf ) throws Exception
+				public DeliveryService newServer( TransportMessage m, ValueFactory vf ) throws Exception
 				{
+					MailboxManager x = new PlainMailboxManager( m, u, res );
+					DeliveryService d = new DefaultDeliveryService( x, u, res );
 					Remote${i}Client client = new Remote${i}Client( d, vf );
 					${i}Server server = implFactory.new${i}Server( client );
 					Pool qp = (Pool) res.get( QUEUED_POOL );
 					Pool fp = (Pool) res.get( FREE_POOL );
 					new Stub${i}Server( d, server, qp, fp );
+					return d;
 				}
 	
 				public ValueFactory newValueFactory()
@@ -129,10 +138,15 @@
 		throws Exception
 	{
 		final Resources res = initResources( resources );
+		
 		final ValueFactory${i} vf = new ValueFactory${i}( uri );
 		res.put( Transport.VALUE_FACTORY, vf );
-
-		DeliveryService d = TransportFactory.getTransport( uri, res );
+		
+		URL u = new URL( uri );
+		
+		TransportMessage m = TransportFactory.getTransport( uri, res );
+		MailboxManager r = new PlainMailboxManager( m, u, resources );
+		DeliveryService d = new DefaultDeliveryService( r, u, resources );
 		Remote${i}Server server = new Remote${i}Server( d, vf );
 		${i}Client client = implFactory.new${i}Client( server );
 		Pool qp = (Pool) res.get( QUEUED_POOL );

Modified: incubator/etch/trunk/binding-java/runtime/src/main/java/org/apache/etch/bindings/java/support/ServerFactory.java
URL: http://svn.apache.org/viewvc/incubator/etch/trunk/binding-java/runtime/src/main/java/org/apache/etch/bindings/java/support/ServerFactory.java?rev=740767&r1=740766&r2=740767&view=diff
==============================================================================
--- incubator/etch/trunk/binding-java/runtime/src/main/java/org/apache/etch/bindings/java/support/ServerFactory.java (original)
+++ incubator/etch/trunk/binding-java/runtime/src/main/java/org/apache/etch/bindings/java/support/ServerFactory.java Wed Feb  4 15:27:52 2009
@@ -18,6 +18,7 @@
 package org.apache.etch.bindings.java.support;
 
 import org.apache.etch.bindings.java.msg.ValueFactory;
+import org.apache.etch.bindings.java.transport.TransportMessage;
 import org.apache.etch.util.core.io.Session;
 
 
@@ -28,11 +29,12 @@
 public interface ServerFactory extends Session
 {
 	/**
-	 * @param d the delivery service to use with the new server.
+	 * @param m the TransportMessage to use with the new server.
 	 * @param vf the value factory to use with the new server.
+	 * @return the constructed DeliveryService
 	 * @throws Exception
 	 */
-	public void newServer( DeliveryService d, ValueFactory vf ) throws Exception;
+	public DeliveryService newServer( TransportMessage m, ValueFactory vf ) throws Exception;
 	
 	/**
 	 * @return a new instance of value factory for this connection.

Modified: incubator/etch/trunk/binding-java/runtime/src/main/java/org/apache/etch/bindings/java/support/TransportFactory.java
URL: http://svn.apache.org/viewvc/incubator/etch/trunk/binding-java/runtime/src/main/java/org/apache/etch/bindings/java/support/TransportFactory.java?rev=740767&r1=740766&r2=740767&view=diff
==============================================================================
--- incubator/etch/trunk/binding-java/runtime/src/main/java/org/apache/etch/bindings/java/support/TransportFactory.java (original)
+++ incubator/etch/trunk/binding-java/runtime/src/main/java/org/apache/etch/bindings/java/support/TransportFactory.java Wed Feb  4 15:27:52 2009
@@ -39,27 +39,27 @@
 abstract public class TransportFactory
 {
 	/**
-	 * Constructs a new Transport stack topped by a DeliveryService
+	 * Constructs a new Transport stack topped by a TransportMessage
 	 * which is used by the remote service implementations to send
-	 * messages and make calls.
+	 * messages.
 	 * @param uri transport configuration parameters.
 	 * @param resources additional resources needed by the stack.
-	 * @return the DeliveryService topping the transport stack.
+	 * @return the TransportMessage topping the transport stack.
 	 * @throws Exception
 	 */
-	abstract protected DeliveryService newTransport( String uri,
+	abstract protected TransportMessage newTransport( String uri,
 		Resources resources ) throws Exception;
 
 	/**
-	 * Constructs a new Transport stack topped by a DeliveryService
+	 * Constructs a new Transport stack topped by a TransportMessage
 	 * which is used by the remote service implementations to send
-	 * messages and make calls.
+	 * messages.
 	 * @param uri transport configuration parameters.
 	 * @param resources additional resources needed by the stack.
-	 * @return the DeliveryService topping the transport stack.
+	 * @return the TransportMessage topping the transport stack.
 	 * @throws Exception
 	 */
-	static public DeliveryService getTransport( String uri,
+	static public TransportMessage getTransport( String uri,
 		Resources resources ) throws Exception
 	{
 		URL u = new URL( uri );

Modified: incubator/etch/trunk/binding-java/runtime/src/main/java/org/apache/etch/bindings/java/transport/Tcp2TransportFactory.java
URL: http://svn.apache.org/viewvc/incubator/etch/trunk/binding-java/runtime/src/main/java/org/apache/etch/bindings/java/transport/Tcp2TransportFactory.java?rev=740767&r1=740766&r2=740767&view=diff
==============================================================================
--- incubator/etch/trunk/binding-java/runtime/src/main/java/org/apache/etch/bindings/java/transport/Tcp2TransportFactory.java (original)
+++ incubator/etch/trunk/binding-java/runtime/src/main/java/org/apache/etch/bindings/java/transport/Tcp2TransportFactory.java Wed Feb  4 15:27:52 2009
@@ -1,19 +1,18 @@
-/*
- * $Id$
- * 
+/* $Id$
+ *
  * Copyright 2007-2008 Cisco Systems Inc.
- * 
+ *
  * Licensed under the Apache License, Version 2.0 (the "License"); you may not
- * use this file except in compliance with the License. You may obtain a copy of
- * the License at
- * 
- * http://www.apache.org/licenses/LICENSE-2.0
- * 
+ * use this file except in compliance with the License. You may obtain a copy
+ * of the License at
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
  * Unless required by applicable law or agreed to in writing, software
  * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
  * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
- * License for the specific language governing permissions and limitations under
- * the License.
+ * License for the specific language governing permissions and limitations
+ * under the License.
  */
 
 package org.apache.etch.bindings.java.transport;
@@ -38,7 +37,7 @@
 /**
  * Selector based transport factory for tcp or tls connections.
  * 
- * TransportFactory.define( "tcp2", "nio.Tcp2TransportFactory" );
+ * TransportFactory.define( "tcp2", "org.apache.etch.bindings.java.transport.Tcp2TransportFactory" );
  */
 public class Tcp2TransportFactory extends TransportFactory
 {
@@ -49,11 +48,10 @@
 	{
 		this( false );
 	}
-
+	
 	/**
 	 * Constructs a TcpTransportFactory which delivers TcpConnection or a
 	 * TlsConnection depending upon the isSecure parameter.
-	 * 
 	 * @param isSecure true if TlsConnection is desired, false otherwise.
 	 */
 	public Tcp2TransportFactory( boolean isSecure )
@@ -62,34 +60,29 @@
 			throw new UnsupportedOperationException( "isSecure" );
 		// this.isSecure = isSecure;
 	}
-
+	
 	// private final boolean isSecure;
-
-//	private final static String CONNECTION = "Tcp2TransportFactory.connection";
-
+	
+	// private final static String CONNECTION = "Tcp2TransportFactory.connection";
+	
 	@Override
-	public DeliveryService newTransport( String uri, Resources resources )
+	public TransportMessage newTransport( String uri, Resources resources )
 		throws Exception
 	{
 		URL u = new URL( uri );
-
+		
 		TransportData c = new Tcp2Connection( u, resources );
-
+		
 		TransportPacket p = new Packetizer( c, u, resources );
-
+		
 		TransportMessage m = new Messagizer( p, u, resources );
-
+		
 		m = addFilters( m, u, resources );
-
-		MailboxManager r = new PlainMailboxManager( m, u, resources );
-
-		DeliveryService d = new DefaultDeliveryService( r, u, resources );
-
-		ValueFactory vf = (ValueFactory) resources
-			.get( Transport.VALUE_FACTORY );
+		
+		ValueFactory vf = (ValueFactory) resources.get( Transport.VALUE_FACTORY );
 		vf.lockDynamicTypes();
-
-		return d;
+		
+		return m;
 	}
 
 	@Override
@@ -98,15 +91,15 @@
 		throws Exception
 	{
 		URL u = new URL( uri );
-
+		
 		Transport<SessionListener<SocketChannel>> l = new Tcp2Listener( u,
 			resources );
-
+		
 		MySessionListener b = new MySessionListener( l, uri, resources );
 		b.setSession( factory );
 		return b;
 	}
-
+	
 	private class MySessionListener implements Transport<ServerFactory>,
 		SessionListener<SocketChannel>
 	{
@@ -122,21 +115,15 @@
 			this.transport = transport;
 			this.uri = uri;
 			this.resources = resources;
-
+			
 			transport.setSession( this );
 		}
-
+		
 		private final Transport<SessionListener<SocketChannel>> transport;
-
+		
 		private final String uri;
-
-		private final Resources resources;
 		
-		@Override
-		public String toString()
-		{
-			return transport.toString();
-		}
+		private final Resources resources;
 
 		public ServerFactory getSession()
 		{
@@ -147,8 +134,14 @@
 		{
 			this.session = session;
 		}
-
+		
 		private ServerFactory session;
+		
+		@Override
+		public String toString()
+		{
+			return "Tcp2TransportFactory.MySessionListener/"+transport;
+		}
 
 		public Object transportQuery( Object query ) throws Exception
 		{
@@ -171,14 +164,14 @@
 		{
 			Resources r = new Resources( resources );
 			r.put( "connection", connection );
-
+			
 			ValueFactory vf = session.newValueFactory();
 			r.put( Transport.VALUE_FACTORY, vf );
-
-			DeliveryService d = newTransport( uri, r );
-
-			session.newServer( d, vf );
-
+			
+			TransportMessage m = newTransport( uri, r );
+			
+			DeliveryService d = session.newServer( m, vf );
+			
 			d.transportControl( Transport.START, null );
 		}
 

Modified: incubator/etch/trunk/binding-java/runtime/src/main/java/org/apache/etch/bindings/java/transport/TcpTransportFactory.java
URL: http://svn.apache.org/viewvc/incubator/etch/trunk/binding-java/runtime/src/main/java/org/apache/etch/bindings/java/transport/TcpTransportFactory.java?rev=740767&r1=740766&r2=740767&view=diff
==============================================================================
--- incubator/etch/trunk/binding-java/runtime/src/main/java/org/apache/etch/bindings/java/transport/TcpTransportFactory.java (original)
+++ incubator/etch/trunk/binding-java/runtime/src/main/java/org/apache/etch/bindings/java/transport/TcpTransportFactory.java Wed Feb  4 15:27:52 2009
@@ -66,7 +66,7 @@
 	private final static String SOCKET = "TcpTransportFactory.socket";
 	
 	@Override
-	public DeliveryService newTransport( String uri,
+	public TransportMessage newTransport( String uri,
 		Resources resources ) throws Exception
 	{
 		URL u = new URL( uri );
@@ -86,14 +86,10 @@
 		
 		m = addFilters( m, u, resources );
 		
-		MailboxManager r = new PlainMailboxManager( m, u, resources );
-		
-		DeliveryService d = new DefaultDeliveryService( r, u, resources );
-		
 		ValueFactory vf = (ValueFactory) resources.get( Transport.VALUE_FACTORY );
 		vf.lockDynamicTypes();
 		
-		return d;
+		return m;
 	}
 
 	@Override
@@ -179,9 +175,9 @@
 			ValueFactory vf = session.newValueFactory();
 			r.put( Transport.VALUE_FACTORY, vf );
 			
-			DeliveryService d = newTransport( uri, r );
+			TransportMessage m = newTransport( uri, r );
 			
-			session.newServer( d, vf );
+			DeliveryService d = session.newServer( m, vf );
 			
 			d.transportControl( Transport.START, null );
 		}

Modified: incubator/etch/trunk/tests/src/main/csharp/etch.tests/MyCuaeHelper.cs
URL: http://svn.apache.org/viewvc/incubator/etch/trunk/tests/src/main/csharp/etch.tests/MyCuaeHelper.cs?rev=740767&r1=740766&r2=740767&view=diff
==============================================================================
--- incubator/etch/trunk/tests/src/main/csharp/etch.tests/MyCuaeHelper.cs (original)
+++ incubator/etch/trunk/tests/src/main/csharp/etch.tests/MyCuaeHelper.cs Wed Feb  4 15:27:52 2009
@@ -45,11 +45,15 @@
                 _implFactory = implFactory;
             }
 
-            public override void NewServer(DeliveryService d, ValueFactory vf)
+            public override DeliveryService NewServer(TransportMessage m, ValueFactory vf)
             {
+                URL u = new URL(_uri);
+                MailboxManager x = new PlainMailboxManager(m, u, _resources);
+                DeliveryService d = new DefaultDeliveryService(x, u, _resources);
                 Pool qp = (Pool)_resources[QUEUED_POOL];
                 Pool fp = (Pool)_resources[FREE_POOL];
                 _implFactory.NewMyCuaeServer(d, qp, fp, (MyValueFactoryCuae)vf);
+                return d;
             }
 
             public override ValueFactory NewValueFactory()

Modified: incubator/etch/trunk/tests/src/main/java/org/apache/etch/tests/MyCuaeHelper.java
URL: http://svn.apache.org/viewvc/incubator/etch/trunk/tests/src/main/java/org/apache/etch/tests/MyCuaeHelper.java?rev=740767&r1=740766&r2=740767&view=diff
==============================================================================
--- incubator/etch/trunk/tests/src/main/java/org/apache/etch/tests/MyCuaeHelper.java (original)
+++ incubator/etch/trunk/tests/src/main/java/org/apache/etch/tests/MyCuaeHelper.java Wed Feb  4 15:27:52 2009
@@ -23,8 +23,13 @@
 import org.apache.etch.bindings.java.support.Pool;
 import org.apache.etch.bindings.java.support.ServerFactory;
 import org.apache.etch.bindings.java.support.TransportFactory;
+import org.apache.etch.bindings.java.transport.DefaultDeliveryService;
+import org.apache.etch.bindings.java.transport.MailboxManager;
+import org.apache.etch.bindings.java.transport.PlainMailboxManager;
 import org.apache.etch.bindings.java.transport.SessionMessage;
+import org.apache.etch.bindings.java.transport.TransportMessage;
 import org.apache.etch.util.Resources;
+import org.apache.etch.util.URL;
 import org.apache.etch.util.core.io.Transport;
 
 
@@ -59,14 +64,18 @@
 		final Resources res = initResources( resources );
 		if (!res.containsKey( Transport.VALUE_FACTORY ))
 			res.put( Transport.VALUE_FACTORY, new MyValueFactoryCuae( "tcp:" ) );
+		final URL u = new URL( uri );
 
 		return TransportFactory.getListener( uri, res, new DefaultServerFactory( implFactory )
 		{
-			public void newServer( DeliveryService d, ValueFactory vf ) throws Exception
+			public DeliveryService newServer( TransportMessage m, ValueFactory vf ) throws Exception
 			{
+				MailboxManager x = new PlainMailboxManager( m, u, res );
+				DeliveryService d = new DefaultDeliveryService( x, u, res );
 				Pool qp = (Pool) res.get( QUEUED_POOL );
 				Pool fp = (Pool) res.get( FREE_POOL );
 				implFactory.newMyCuaeServer( d, qp, fp, (MyValueFactoryCuae) vf );
+				return d;
 			}
 
 			public ValueFactory newValueFactory()