Return-Path: Delivered-To: apmail-geronimo-activemq-commits-archive@www.apache.org Received: (qmail 35335 invoked from network); 5 Mar 2006 12:06:23 -0000 Received: from hermes.apache.org (HELO mail.apache.org) (209.237.227.199) by minotaur.apache.org with SMTP; 5 Mar 2006 12:06:23 -0000 Received: (qmail 2654 invoked by uid 500); 5 Mar 2006 12:07:09 -0000 Delivered-To: apmail-geronimo-activemq-commits-archive@geronimo.apache.org Received: (qmail 2620 invoked by uid 500); 5 Mar 2006 12:07:09 -0000 Mailing-List: contact activemq-commits-help@geronimo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: activemq-dev@geronimo.apache.org Delivered-To: mailing list activemq-commits@geronimo.apache.org Received: (qmail 2611 invoked by uid 99); 5 Mar 2006 12:07:08 -0000 Received: from asf.osuosl.org (HELO asf.osuosl.org) (140.211.166.49) by apache.org (qpsmtpd/0.29) with ESMTP; Sun, 05 Mar 2006 04:07:08 -0800 X-ASF-Spam-Status: No, hits=-8.4 required=10.0 tests=ALL_TRUSTED,INFO_TLD,NO_REAL_NAME,PORN_URL_SEX X-Spam-Check-By: apache.org Received: from [209.237.227.194] (HELO minotaur.apache.org) (209.237.227.194) by apache.org (qpsmtpd/0.29) with SMTP; Sun, 05 Mar 2006 04:07:07 -0800 Received: (qmail 35262 invoked by uid 65534); 5 Mar 2006 12:05:59 -0000 Message-ID: <20060305120559.35261.qmail@minotaur.apache.org> Content-Type: text/plain; charset="utf-8" MIME-Version: 1.0 Content-Transfer-Encoding: 8bit Subject: svn commit: r383309 - in /incubator/activemq/trunk/openwire-dotnet: src/ src/ActiveMQ/ src/ActiveMQ/Transport/ src/ActiveMQ/Transport/Tcp/ src/JMS/ tests/ActiveMQ/ tests/JMS/ Date: Sun, 05 Mar 2006 12:05:57 -0000 To: activemq-commits@geronimo.apache.org From: chirino@apache.org X-Mailer: svnmailer-1.0.7 X-Virus-Checked: Checked by ClamAV on apache.org X-Spam-Rating: minotaur.apache.org 1.6.2 0/1000/N Author: chirino Date: Sun Mar 5 04:05:53 2006 New Revision: 383309 URL: http://svn.apache.org/viewcvs?rev=383309&view=rev Log: Implemented the filtered layer concept similar to what we have with the Java implemenation. Added: incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/ITransportFactory.cs incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/LoggingTransport.cs incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/MutexTransport.cs incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/ResponseCorrelator.cs incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/Tcp/ incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/Tcp/TcpTransport.cs incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/Tcp/TcpTransportFactory.cs incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/TransportFilter.cs incubator/activemq/trunk/openwire-dotnet/src/JMS/JMSConnectionException.cs incubator/activemq/trunk/openwire-dotnet/src/JMS/JMSException.cs Removed: incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/SocketTransport.cs Modified: incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Connection.cs incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/ConnectionFactory.cs incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/ITransport.cs incubator/activemq/trunk/openwire-dotnet/src/src.csproj incubator/activemq/trunk/openwire-dotnet/tests/ActiveMQ/TestMain.cs incubator/activemq/trunk/openwire-dotnet/tests/JMS/JMSTestSupport.cs Modified: incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Connection.cs URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Connection.cs?rev=383309&r1=383308&r2=383309&view=diff ============================================================================== --- incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Connection.cs (original) +++ incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Connection.cs Sun Mar 5 04:05:53 2006 @@ -29,7 +29,8 @@ { this.transport = transport; this.info = info; - this.transport.Command += new CommandHandler(OnCommand); + this.transport.Command = new CommandHandler(OnCommand); + this.transport.Exception = new ExceptionHandler(OnException); this.transport.Start(); } @@ -255,9 +256,14 @@ } else { - Console.WriteLine("ERROR:ĂŠUnknown command: " + command); + Console.WriteLine("ERROR: Unknown command: " + command); } } + + protected void OnException(ITransport sender, Exception exception) { + Console.WriteLine("ERROR: Transport Exception: " + exception); + } + protected SessionInfo CreateSessionInfo(AcknowledgementMode acknowledgementMode) { Modified: incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/ConnectionFactory.cs URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/ConnectionFactory.cs?rev=383309&r1=383308&r2=383309&view=diff ============================================================================== --- incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/ConnectionFactory.cs (original) +++ incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/ConnectionFactory.cs Sun Mar 5 04:05:53 2006 @@ -16,6 +16,7 @@ */ using ActiveMQ.Commands; using ActiveMQ.Transport; +using ActiveMQ.Transport.Tcp; using JMS; using System; @@ -26,8 +27,7 @@ /// public class ConnectionFactory : IConnectionFactory { - private string host = "localhost"; - private int port = 61616; + private Uri brokerUri = new Uri("tcp://localhost:61616"); private string userName; private string password; private string clientId; @@ -36,10 +36,9 @@ { } - public ConnectionFactory(string host, int port) + public ConnectionFactory(Uri brokerUri) { - this.host = host; - this.port = port; + this.brokerUri=brokerUri; } public IConnection CreateConnection() @@ -50,7 +49,7 @@ public IConnection CreateConnection(string userName, string password) { ConnectionInfo info = CreateConnectionInfo(userName, password); - ITransport transport = CreateTransport(); + ITransport transport = new TcpTransportFactory().CreateTransport(brokerUri); Connection connection = new Connection(transport, info); connection.ClientId = info.ClientId; return connection; @@ -58,18 +57,12 @@ // Properties - public string Host + public Uri BrokerUri { - get { return host; } - set { host = value; } + get { return brokerUri; } + set { brokerUri = value; } } - - public int Port - { - get { return port; } - set { port = value; } - } - + public string UserName { get { return userName; } @@ -112,9 +105,5 @@ return Guid.NewGuid().ToString(); } - protected ITransport CreateTransport() - { - return new SocketTransport(host, port); - } } } Modified: incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/ITransport.cs URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/ITransport.cs?rev=383309&r1=383308&r2=383309&view=diff ============================================================================== --- incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/ITransport.cs (original) +++ incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/ITransport.cs Sun Mar 5 04:05:53 2006 @@ -19,22 +19,14 @@ using JMS; using System; - - -namespace ActiveMQ.Transport -{ - public delegate void CommandHandler(ITransport sender, Command command); -} -namespace ActiveMQ.Transport -{ - public delegate void ExceptionHandler(ITransport sender, Exception command); -} - /// /// Represents the logical networking transport layer. /// namespace ActiveMQ.Transport { + public delegate void CommandHandler(ITransport sender, Command command); + public delegate void ExceptionHandler(ITransport sender, Exception command); + public interface ITransport : IStartable, IDisposable { void Oneway(Command command); @@ -43,8 +35,15 @@ Response Request(Command command); - event CommandHandler Command; - event ExceptionHandler Exception; + CommandHandler Command{ + get; + set; + } + + ExceptionHandler Exception{ + get; + set; + } } } Added: incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/ITransportFactory.cs URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/ITransportFactory.cs?rev=383309&view=auto ============================================================================== --- incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/ITransportFactory.cs (added) +++ incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/ITransportFactory.cs Sun Mar 5 04:05:53 2006 @@ -0,0 +1,26 @@ +/* + * Copyright 2006 The Apache Software Foundation or its licensors, as + * applicable. + * + * 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 + * + * 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. + */ + +using System; + +namespace ActiveMQ.Transport +{ + public interface ITransportFactory + { + ITransport CreateTransport(Uri location); + } +} Added: incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/LoggingTransport.cs URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/LoggingTransport.cs?rev=383309&view=auto ============================================================================== --- incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/LoggingTransport.cs (added) +++ incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/LoggingTransport.cs Sun Mar 5 04:05:53 2006 @@ -0,0 +1,50 @@ +/* + * Copyright 2006 The Apache Software Foundation or its licensors, as + * applicable. + * + * 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 + * + * 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. + */ +using ActiveMQ.Commands; +using ActiveMQ.Transport; +using JMS; +using System; + +/// +/// A Transport filter that is used to log the commands sent and received. +/// +namespace ActiveMQ.Transport +{ + public class LoggingTransport : TransportFilter + { + public LoggingTransport(ITransport next) : base(next) { + } + + protected override void OnCommand(ITransport sender, Command command) { + Console.WriteLine("RECEIVED: " + command); + this.command(sender, command); + } + + protected override void OnException(ITransport sender, Exception error) { + Console.WriteLine("RECEIVED Exception: " + error); + this.exception(sender, error); + } + + public override void Oneway(Command command) + { + Console.WriteLine("SENDING: " + command); + this.next.Oneway(command); + } + + } +} + Added: incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/MutexTransport.cs URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/MutexTransport.cs?rev=383309&view=auto ============================================================================== --- incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/MutexTransport.cs (added) +++ incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/MutexTransport.cs Sun Mar 5 04:05:53 2006 @@ -0,0 +1,70 @@ +/* + * Copyright 2006 The Apache Software Foundation or its licensors, as + * applicable. + * + * 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 + * + * 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. + */ +using ActiveMQ.Commands; +using ActiveMQ.Transport; +using JMS; +using System; + +/// +/// A Transport which gaurds access to the next transport using a mutex. +/// +namespace ActiveMQ.Transport +{ + public class MutexTransport : TransportFilter + { + + private readonly object transmissionLock = new object(); + + public MutexTransport(ITransport next) : base(next) { + } + + + public override void Oneway(Command command) + { + lock (transmissionLock) + { + this.next.Oneway(command); + } + } + + public override FutureResponse AsyncRequest(Command command) + { + lock (transmissionLock) + { + return base.AsyncRequest(command); + } + } + + public override Response Request(Command command) + { + lock (transmissionLock) + { + return base.Request(command); + } + } + + public override void Dispose() + { + lock (transmissionLock) + { + base.Dispose(); + } + } + + } +} + Added: incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/ResponseCorrelator.cs URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/ResponseCorrelator.cs?rev=383309&view=auto ============================================================================== --- incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/ResponseCorrelator.cs (added) +++ incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/ResponseCorrelator.cs Sun Mar 5 04:05:53 2006 @@ -0,0 +1,104 @@ +/* + * Copyright 2006 The Apache Software Foundation or its licensors, as + * applicable. + * + * 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 + * + * 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. + */ + +using System; +using System.Collections; + +using ActiveMQ.Commands; +using ActiveMQ.Transport; +using JMS; + +/// +/// A Transport which gaurds access to the next transport using a mutex. +/// +namespace ActiveMQ.Transport +{ + public class ResponseCorrelator : TransportFilter + { + + private readonly IDictionary requestMap = Hashtable.Synchronized(new Hashtable()); + private readonly Object mutex = new Object(); + private short nextCommandId; + + public ResponseCorrelator(ITransport next) : base(next) { + } + + short GetNextCommandId() { + lock(mutex) { + return ++nextCommandId; + } + } + + public override void Oneway(Command command) + { + command.CommandId = GetNextCommandId(); + command.ResponseRequired = false; + next.Oneway(command); + } + + public override FutureResponse AsyncRequest(Command command) + { + command.CommandId = GetNextCommandId(); + command.ResponseRequired = true; + FutureResponse future = new FutureResponse(); + requestMap[command.CommandId] = future; + next.Oneway(command); + return future; + + } + + public override Response Request(Command command) + { + FutureResponse future = AsyncRequest(command); + Response response = future.Response; + if (response is ExceptionResponse) + { + ExceptionResponse er = (ExceptionResponse) response; + BrokerError brokerError = er.Exception; + throw new BrokerException(brokerError); + } + return response; + } + + protected override void OnCommand(ITransport sender, Command command) + { + if( command is Response ) { + + Response response = (Response) command; + FutureResponse future = (FutureResponse) requestMap[response.CorrelationId]; + if (future != null) + { + if (response is ExceptionResponse) + { + ExceptionResponse er = (ExceptionResponse) response; + BrokerError brokerError = er.Exception; + this.exception(this, new BrokerException(brokerError)); + } + future.Response = response; + } + else + { + Console.WriteLine("ERROR: Unknown response ID: " + response.CommandId + " for response: " + response); + } + } else { + this.command(sender, command); + } + } + + } +} + Added: incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/Tcp/TcpTransport.cs URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/Tcp/TcpTransport.cs?rev=383309&view=auto ============================================================================== --- incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/Tcp/TcpTransport.cs (added) +++ incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/Tcp/TcpTransport.cs Sun Mar 5 04:05:53 2006 @@ -0,0 +1,145 @@ +/* + * Copyright 2006 The Apache Software Foundation or its licensors, as + * applicable. + * + * 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 + * + * 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. + */ +using ActiveMQ; +using ActiveMQ.Commands; +using ActiveMQ.OpenWire; +using ActiveMQ.Transport; +using System; +using System.Collections; +using System.IO; +using System.Net; +using System.Net.Sockets; +using System.Threading; + + + +/// +/// An implementation of ITransport that uses sockets to communicate with the broker +/// + +namespace ActiveMQ.Transport.Tcp +{ + public class TcpTransport : ITransport + { + private Socket socket; + private OpenWireFormat wireformat = new OpenWireFormat(); + private BinaryReader socketReader; + private BinaryWriter socketWriter; + private Thread readThread; + private bool started; + volatile private bool closed; + + public CommandHandler command; + public ExceptionHandler exception; + + public TcpTransport(Socket socket) + { + this.socket = socket; + } + + /// + /// Method Start + /// + public void Start() + { + if (!started) + { + if( command == null ) + throw new InvalidOperationException ("command cannot be null when Start is called."); + if( exception == null ) + throw new InvalidOperationException ("exception cannot be null when Start is called."); + + started = true; + + NetworkStream networkStream = new NetworkStream(socket); + socketWriter = new BinaryWriter(networkStream); + socketReader = new BinaryReader(networkStream); + + // now lets create the background read thread + readThread = new Thread(new ThreadStart(ReadLoop)); + readThread.Start(); + + // lets send the wireformat we're using + Oneway(wireformat.WireFormatInfo); + } + } + + public void Oneway(Command command) + { + wireformat.Marshal(command, socketWriter); + socketWriter.Flush(); + } + + public FutureResponse AsyncRequest(Command command) + { + throw new NotImplementedException("Use a ResponseCorrelator if you want to issue AsyncRequest calls"); + } + + public Response Request(Command command) + { + throw new NotImplementedException("Use a ResponseCorrelator if you want to issue Request calls"); + } + + public void Dispose() + { + closed = true; + socket.Close(); + readThread.Join(); + socketWriter.Close(); + socketReader.Close(); + } + + public void ReadLoop() + { + while (!closed) + { + try + { + Command command = (Command) wireformat.Unmarshal(socketReader); + this.command(this, command); + } + catch (ObjectDisposedException) + { + break; + } + catch (Exception e) + { + this.exception(this,e); + } + } + } + + + + + // Implementation methods + + public CommandHandler Command { + get { return command; } + set { this.command = value; } + } + + public ExceptionHandler Exception { + get { return exception; } + set { this.exception = value; } + } + + } +} + + + Added: incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/Tcp/TcpTransportFactory.cs URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/Tcp/TcpTransportFactory.cs?rev=383309&view=auto ============================================================================== --- incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/Tcp/TcpTransportFactory.cs (added) +++ incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/Tcp/TcpTransportFactory.cs Sun Mar 5 04:05:53 2006 @@ -0,0 +1,62 @@ +/* +* Copyright 2006 The Apache Software Foundation or its licensors, as +* applicable. +* +* 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 +* +* 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. +*/ + +using System; +using System.Net; +using System.Net.Sockets; +using ActiveMQ.Transport; + +namespace ActiveMQ.Transport.Tcp +{ + public class TcpTransportFactory : ITransportFactory + { + public ITransport CreateTransport(Uri location) { + + // Console.WriteLine("Opening socket to: " + host + " on port: " + port); + Socket socket = Connect(location.Host, location.Port); + ITransport rc = new TcpTransport(socket); + // TODO: use URI query string to enable the LoggingTransport + // rc = new LoggingTransport(rc); + rc = new ResponseCorrelator(rc); + rc = new MutexTransport(rc); + return rc; + + } + + protected Socket Connect(string host, int port) + { + // Looping through the AddressList allows different type of connections to be tried + // (IPv4, IPv6 and whatever else may be available). + IPHostEntry hostEntry = Dns.Resolve(host); + foreach (IPAddress address in hostEntry.AddressList) + { + Socket socket = new Socket( + address.AddressFamily, + SocketType.Stream, + ProtocolType.Tcp); + socket.Connect(new IPEndPoint(address, port)); + if (socket.Connected) + { + return socket; + } + } + throw new SocketException(); + } + + } + +} Added: incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/TransportFilter.cs URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/TransportFilter.cs?rev=383309&view=auto ============================================================================== --- incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/TransportFilter.cs (added) +++ incubator/activemq/trunk/openwire-dotnet/src/ActiveMQ/Transport/TransportFilter.cs Sun Mar 5 04:05:53 2006 @@ -0,0 +1,109 @@ +/* + * Copyright 2006 The Apache Software Foundation or its licensors, as + * applicable. + * + * 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 + * + * 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. + */ +using ActiveMQ.Commands; +using ActiveMQ.Transport; +using JMS; +using System; + +/// +/// Used to implement a filter on the transport layer. +/// +namespace ActiveMQ.Transport +{ + public class TransportFilter : ITransport + { + protected readonly ITransport next; + protected CommandHandler command; + protected ExceptionHandler exception; + + public TransportFilter(ITransport next) { + this.next = next; + this.next.Command = new CommandHandler(OnCommand); + this.next.Exception = new ExceptionHandler(OnException); + } + + protected virtual void OnCommand(ITransport sender, Command command) { + this.command(sender, command); + } + + protected virtual void OnException(ITransport sender, Exception command) { + this.exception(sender, command); + } + + + /// + /// Method Oneway + /// + /// A Command + public virtual void Oneway(Command command) + { + this.next.Oneway(command); + } + + /// + /// Method AsyncRequest + /// + /// A FutureResponse + /// A Command + public virtual FutureResponse AsyncRequest(Command command) + { + return this.next.AsyncRequest(command); + } + + /// + /// Method Request + /// + /// A Response + /// A Command + public virtual Response Request(Command command) + { + return this.next.Request(command); + } + + /// + /// Method Start + /// + public virtual void Start() + { + if( command == null ) + throw new InvalidOperationException ("command cannot be null when Start is called."); + if( exception == null ) + throw new InvalidOperationException ("exception cannot be null when Start is called."); + this.next.Start(); + } + + /// + /// Method Dispose + /// + public virtual void Dispose() + { + this.next.Dispose(); + } + + public CommandHandler Command { + get { return command; } + set { this.command = value; } + } + + public ExceptionHandler Exception { + get { return exception; } + set { this.exception = value; } + } + + } +} + Added: incubator/activemq/trunk/openwire-dotnet/src/JMS/JMSConnectionException.cs URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/openwire-dotnet/src/JMS/JMSConnectionException.cs?rev=383309&view=auto ============================================================================== --- incubator/activemq/trunk/openwire-dotnet/src/JMS/JMSConnectionException.cs (added) +++ incubator/activemq/trunk/openwire-dotnet/src/JMS/JMSConnectionException.cs Sun Mar 5 04:05:53 2006 @@ -0,0 +1,32 @@ +/* + * Copyright 2006 The Apache Software Foundation or its licensors, as + * applicable. + * + * 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 + * + * 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. + */ +using System; + + +/// +/// Represents a connection failure. +/// +namespace JMS +{ + public class ConnectionException : JMSException + { + public JMSException(string message) : base(message) + { + } + } +} + Added: incubator/activemq/trunk/openwire-dotnet/src/JMS/JMSException.cs URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/openwire-dotnet/src/JMS/JMSException.cs?rev=383309&view=auto ============================================================================== --- incubator/activemq/trunk/openwire-dotnet/src/JMS/JMSException.cs (added) +++ incubator/activemq/trunk/openwire-dotnet/src/JMS/JMSException.cs Sun Mar 5 04:05:53 2006 @@ -0,0 +1,32 @@ +/* + * Copyright 2006 The Apache Software Foundation or its licensors, as + * applicable. + * + * 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 + * + * 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. + */ +using System; + + +/// +/// Represents a JMS exception +/// +namespace JMS +{ + public class JMSException : Exception + { + public JMSException(string message) : base(message) + { + } + } +} + Modified: incubator/activemq/trunk/openwire-dotnet/src/src.csproj URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/openwire-dotnet/src/src.csproj?rev=383309&r1=383308&r2=383309&view=diff ============================================================================== --- incubator/activemq/trunk/openwire-dotnet/src/src.csproj (original) +++ incubator/activemq/trunk/openwire-dotnet/src/src.csproj Sun Mar 5 04:05:53 2006 @@ -164,7 +164,13 @@ - + + + + + + + Modified: incubator/activemq/trunk/openwire-dotnet/tests/ActiveMQ/TestMain.cs URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/openwire-dotnet/tests/ActiveMQ/TestMain.cs?rev=383309&r1=383308&r2=383309&view=diff ============================================================================== --- incubator/activemq/trunk/openwire-dotnet/tests/ActiveMQ/TestMain.cs (original) +++ incubator/activemq/trunk/openwire-dotnet/tests/ActiveMQ/TestMain.cs Sun Mar 5 04:05:53 2006 @@ -29,7 +29,7 @@ Console.WriteLine("About to connect to ActiveMQ"); // START SNIPPET: demo - IConnectionFactory factory = new ConnectionFactory("localhost", 61616); + IConnectionFactory factory = new ConnectionFactory(new Uri("tcp://localhost:61616")); using (IConnection connection = factory.CreateConnection()) { Console.WriteLine("Created a connection!"); Modified: incubator/activemq/trunk/openwire-dotnet/tests/JMS/JMSTestSupport.cs URL: http://svn.apache.org/viewcvs/incubator/activemq/trunk/openwire-dotnet/tests/JMS/JMSTestSupport.cs?rev=383309&r1=383308&r2=383309&view=diff ============================================================================== --- incubator/activemq/trunk/openwire-dotnet/tests/JMS/JMSTestSupport.cs (original) +++ incubator/activemq/trunk/openwire-dotnet/tests/JMS/JMSTestSupport.cs Sun Mar 5 04:05:53 2006 @@ -113,7 +113,7 @@ } protected virtual IConnectionFactory CreateConnectionFactory() { - return new ActiveMQ.ConnectionFactory("localhost", 61616); + return new ActiveMQ.ConnectionFactory(new Uri("tcp://localhost:61616")); } protected virtual IConnection CreateConnection()