AMQNET-637 NMS 2.0

Add deliverydelay
Add async send methods
Add shared and shared durable consumers
add missing ack method on session thats in JMS
Create INMSConsumer
Create INMSProducer
Create INMSContext
Add Create Context Methods to ConnectionFactory
Add CreateContext wrapper methods in factory
fix missing file type
remove duplicate method signature
Remove duplicated method
remove unspec'd methods
changes to address review comments from Krystof.
diff --git a/src/nms-api/IConnectionFactory.cs b/src/nms-api/IConnectionFactory.cs
index 8094185..dc7fd28 100644
--- a/src/nms-api/IConnectionFactory.cs
+++ b/src/nms-api/IConnectionFactory.cs
@@ -33,6 +33,27 @@
 		/// Creates a new connection with the given user name and password
 		/// </summary>
 		IConnection CreateConnection(string userName, string password);
+		
+		/// <summary>
+		/// Creates a new context
+		/// </summary>
+		INMSContext CreateContext();
+		
+		/// <summary>
+		/// Creates a new context with the given acknowledgement mode.
+		/// </summary>
+		INMSContext CreateContext(AcknowledgementMode acknowledgementMode);
+
+		/// <summary>
+		/// Creates a new context with the given user name and password
+		/// </summary>
+		INMSContext CreateContext(string userName, string password);
+
+		/// <summary>
+		/// Creates a new context with the given user name, password and acknowledgement mode
+		/// </summary>
+		INMSContext CreateContext(string userName, string password, AcknowledgementMode acknowledgementMode);
+		
 
 		/// <summary>
 		/// Get/or set the broker Uri.
diff --git a/src/nms-api/IMessage.cs b/src/nms-api/IMessage.cs
index 2255178..fbb1207 100644
--- a/src/nms-api/IMessage.cs
+++ b/src/nms-api/IMessage.cs
@@ -100,5 +100,18 @@
 		/// The type name of this message.
 		/// </summary>
 		string NMSType { get; set; }
+		
+		/// <summary>
+		/// When a message is sent, the NMSDeliveryTime header field is
+        /// left unassigned. After completion of the send or
+        /// publish method, it holds the delivery time of the message.
+        /// This is the the difference, measured in milliseconds, 
+        /// between the delivery time and midnight, January 1, 1970 UTC.
+        ///
+		/// A message's delivery time is the earliest time when a JMS provider may
+	    /// deliver the message to a consumer. The provider must not deliver messages
+	    /// before the delivery time has been reached.
+		/// <summary>
+		DateTime NMSDeliveryTime { get; set; }
 	}
 }
diff --git a/src/nms-api/IMessageProducer.cs b/src/nms-api/IMessageProducer.cs
index c72d488..d3b6e74 100644
--- a/src/nms-api/IMessageProducer.cs
+++ b/src/nms-api/IMessageProducer.cs
@@ -15,9 +15,13 @@
  * limitations under the License.
  */
 using System;
+using System.Threading.Tasks;
 
 namespace Apache.NMS
 {
+        public delegate void CompletionListener(IMessage message, Exception e);
+
+
 	/// <summary>
 	/// A delegate that a client can register that will be called each time a Producer's send method is
 	/// called to allow the client to Transform a sent message from one type to another, StreamMessage to
@@ -54,6 +58,47 @@
 		/// </summary>
 		void Send(IDestination destination, IMessage message, MsgDeliveryMode deliveryMode, MsgPriority priority, TimeSpan timeToLive);
 
+        /// <summary>
+        /// Sends the message to the default destination for this producer
+        /// </summary>
+        void Send(IMessage message, CompletionListener completionListener);
+
+        /// <summary>
+        /// Sends the message to the default destination with the explicit QoS configuration
+        /// </summary>
+        void Send(IMessage message, MsgDeliveryMode deliveryMode, MsgPriority priority, TimeSpan timeToLive, CompletionListener completionListener);
+
+        /// <summary>
+        /// Sends the message to the given destination
+        /// </summary>
+        void Send(IDestination destination, IMessage message, CompletionListener completionListener);
+
+        /// <summary>
+        /// Sends the message to the given destination with the explicit QoS configuration
+        /// </summary>
+        void Send(IDestination destination, IMessage message, MsgDeliveryMode deliveryMode, MsgPriority priority, TimeSpan timeToLive, CompletionListener completionListener);
+
+        /// <summary>
+        /// Sends the message to the default destination for this producer
+        /// </summary>
+        Task SendAsync(IMessage message);
+
+        /// <summary>
+        /// Sends the message to the default destination with the explicit QoS configuration
+        /// </summary>
+        Task SendAsync(IMessage message, MsgDeliveryMode deliveryMode, MsgPriority priority, TimeSpan timeToLive);
+
+        /// <summary>
+        /// Sends the message to the given destination
+        /// </summary>
+        Task SendAsync(IDestination destination, IMessage message);
+
+        /// <summary>
+        /// Sends the message to the given destination with the explicit QoS configuration
+        /// </summary>
+        Task SendAsync(IDestination destination, IMessage message, MsgDeliveryMode deliveryMode, MsgPriority priority, TimeSpan timeToLive);
+
+
 		/// <summary>
 		/// Close the producer.
 		/// </summary>
@@ -76,6 +121,8 @@
 		bool DisableMessageID { get; set; }
 
 		bool DisableMessageTimestamp { get; set; }
+		
+		TimeSpan DeliveryDelay { get; set; }
 
 		#region Factory methods to create messages
 
diff --git a/src/nms-api/INMSConsumer.cs b/src/nms-api/INMSConsumer.cs
new file mode 100644
index 0000000..f084db9
--- /dev/null
+++ b/src/nms-api/INMSConsumer.cs
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 Apache.NMS
+{
+
+	/// <summary>
+	/// An object capable of sending messages to some destination
+	/// </summary>
+	public interface INMSConsumer : System.IDisposable
+	{
+
+		string MessageSelector { get; }
+
+		/// <summary>
+		/// Waits until a message is available and returns it
+		/// </summary>
+		IMessage Receive();
+
+		/// <summary>
+		/// If a message is available within the timeout duration it is returned otherwise this method returns null
+		/// </summary>
+		IMessage Receive(TimeSpan timeout);
+
+		/// <summary>
+		/// Receives the next message if one is immediately available for delivery on the client side
+		/// otherwise this method returns null. It is never an error for this method to return null, the
+		/// time of Message availability varies so your client cannot rely on this method to receive a
+		/// message immediately after one has been sent.
+		/// </summary>
+		IMessage ReceiveNoWait();
+
+
+		T ReceiveBody<T>();
+		
+		T ReceiveBody<T>(TimeSpan timeout);
+
+		T ReceiveBodyNoWait<T>();
+		
+		
+		/// <summary>
+		/// An asynchronous listener which can be used to consume messages asynchronously
+		/// </summary>
+		event MessageListener Listener;
+
+		/// <summary>
+		/// Closes the message consumer.
+		/// </summary>
+		/// <remarks>
+		/// Clients should close message consumers when they are not needed.
+		/// This call blocks until a receive or message listener in progress has completed.
+		/// A blocked message consumer receive call returns null when this message consumer is closed.
+		/// </remarks>
+		void Close();
+
+		/// <summary>
+		/// A Delegate that is called each time a Message is dispatched to allow the client to do
+		/// any necessary transformations on the received message before it is delivered.
+		/// </summary>
+		ConsumerTransformerDelegate ConsumerTransformer { get; set; }
+	}
+}
diff --git a/src/nms-api/INMSContext.cs b/src/nms-api/INMSContext.cs
new file mode 100644
index 0000000..4aada4f
--- /dev/null
+++ b/src/nms-api/INMSContext.cs
@@ -0,0 +1,269 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 Apache.NMS
+{
+
+	/// <summary>
+	/// A INMSContext is the main interface in the simplified NMS API
+	/// introduced for NMS 2.0. This combines in a single object the functionality of
+	/// two separate objects from the NMS 1.x API: an IConnection and an ISession.
+	///
+	/// This is comparable to JMS 2.0 API that extended JMS 1.x API
+	/// </summary>
+	public interface INMSContext : IDisposable, IStartable, IStoppable
+	{
+
+		INMSContext CreateContext(AcknowledgementMode acknowledgementMode);
+		
+		/// <summary>
+		/// Creates a producer of messages
+		/// </summary>
+		INMSProducer CreateProducer();
+		
+		
+		/// <summary>
+		/// Creates a consumer of messages on a given destination
+		/// </summary>
+		INMSConsumer CreateConsumer(IDestination destination);
+
+		/// <summary>
+		/// Creates a consumer of messages on a given destination with a selector
+		/// </summary>
+		INMSConsumer CreateConsumer(IDestination destination, string selector);
+
+		/// <summary>
+		/// Creates a consumer of messages on a given destination with a selector
+		/// </summary>
+		INMSConsumer CreateConsumer(IDestination destination, string selector, bool noLocal);
+
+		INMSConsumer CreateDurableConsumer(ITopic destination, string subscriptionName);
+
+		INMSConsumer CreateDurableConsumer(ITopic destination, string subscriptionName, string selector);
+
+        /// <summary>
+        /// Creates a named durable consumer of messages on a given destination with a selector
+        /// </summary>
+        INMSConsumer CreateDurableConsumer(ITopic destination, string subscriptionName, string selector, bool noLocal);
+
+        INMSConsumer CreateSharedConsumer(ITopic destination, string subscriptionName);
+
+        INMSConsumer CreateSharedConsumer(ITopic destination, string subscriptionName, string selector);
+
+        INMSConsumer CreateSharedDurableConsumer(ITopic destination, string subscriptionName);
+
+        INMSConsumer CreateSharedDurableConsumer(ITopic destination, string subscriptionName, string selector);
+
+
+
+        void Unsubscribe(string name);
+
+		/// <summary>
+		/// Creates a QueueBrowser object to peek at the messages on the specified queue.
+		/// </summary>
+		/// <param name="queue">
+		/// A <see cref="IQueue"/>
+		/// </param>
+		/// <returns>
+		/// A <see cref="IQueueBrowser"/>
+		/// </returns>
+		/// <exception cref="System.NotSupportedException">
+		/// If the Prodiver does not support creation of Queue Browsers.
+		/// </exception>
+		IQueueBrowser CreateBrowser(IQueue queue);
+
+		/// <summary>
+		/// Creates a QueueBrowser object to peek at the messages on the specified queue
+		/// using a message selector.
+		/// </summary>
+		/// <param name="queue">
+		/// A <see cref="IQueue"/>
+		/// </param>
+		/// <param name="selector">
+		/// A <see cref="System.String"/>
+		/// </param>
+		/// <returns>
+		/// A <see cref="IQueueBrowser"/>
+		/// </returns>
+		/// <exception cref="System.NotSupportedException">
+		/// If the Prodiver does not support creation of Queue Browsers.
+		/// </exception>
+		IQueueBrowser CreateBrowser(IQueue queue, string selector);
+
+		/// <summary>
+		/// Returns the queue for the given name
+		/// </summary>
+		IQueue GetQueue(string name);
+
+		/// <summary>
+		/// Returns the topic for the given name
+		/// </summary>
+		ITopic GetTopic(string name);
+
+		/// <summary>
+		/// Creates a temporary queue
+		/// </summary>
+		ITemporaryQueue CreateTemporaryQueue();
+
+		/// <summary>
+		/// Creates a temporary topic
+		/// </summary>
+		ITemporaryTopic CreateTemporaryTopic();
+
+		// Factory methods to create messages
+
+		/// <summary>
+		/// Creates a new message with an empty body
+		/// </summary>
+		IMessage CreateMessage();
+
+		/// <summary>
+		/// Creates a new text message with an empty body
+		/// </summary>
+		ITextMessage CreateTextMessage();
+
+		/// <summary>
+		/// Creates a new text message with the given body
+		/// </summary>
+		ITextMessage CreateTextMessage(string text);
+
+		/// <summary>
+		/// Creates a new Map message which contains primitive key and value pairs
+		/// </summary>
+		IMapMessage CreateMapMessage();
+
+		/// <summary>
+		/// Creates a new Object message containing the given .NET object as the body
+		/// </summary>
+		IObjectMessage CreateObjectMessage(object body);
+
+		/// <summary>
+		/// Creates a new binary message
+		/// </summary>
+		IBytesMessage CreateBytesMessage();
+
+		/// <summary>
+		/// Creates a new binary message with the given body
+		/// </summary>
+		IBytesMessage CreateBytesMessage(byte[] body);
+
+		/// <summary>
+		/// Creates a new stream message
+		/// </summary>
+		IStreamMessage CreateStreamMessage();
+
+		/// <summary>
+		/// Closes the session.  There is no need to close the producers and consumers
+		/// of a closed session.
+		/// </summary>
+		void Close();
+
+		/// <summary>
+		/// A Delegate that is called each time a Message is dispatched to allow the client to do
+		/// any necessary transformations on the received message before it is delivered.
+		/// The Session instance sets the delegate on each Consumer it creates.
+		/// </summary>
+		ConsumerTransformerDelegate ConsumerTransformer { get; set; }
+
+		/// <summary>
+		/// A delegate that is called each time a Message is sent from this Producer which allows
+		/// the application to perform any needed transformations on the Message before it is sent.
+		/// The Session instance sets the delegate on each Producer it creates.
+		/// </summary>
+		ProducerTransformerDelegate ProducerTransformer { get; set; }
+
+        /// <summary>
+        /// Stops all Message delivery in this session and restarts it again
+        /// with the oldest unabcknowledged message.  Messages that were delivered
+        /// but not acknowledge should have their redelivered property set.
+        /// This is an optional method that may not by implemented by all NMS
+        /// providers, if not implemented an Exception will be thrown.
+        /// Message redelivery is not requried to be performed in the original
+        /// order.  It is not valid to call this method on a Transacted Session.
+        /// </summary>
+        void Recover();
+        
+        void Acknowledge();
+
+		#region Transaction methods
+
+		/// <summary>
+		/// If this is a transactional session then commit all message
+		/// send and acknowledgements for producers and consumers in this session
+		/// </summary>
+		void Commit();
+
+		/// <summary>
+		/// If this is a transactional session then rollback all message
+		/// send and acknowledgements for producers and consumers in this session
+		/// </summary>
+		void Rollback();
+
+		#endregion
+
+        #region Session Events
+
+        event SessionTxEventDelegate TransactionStartedListener;
+        event SessionTxEventDelegate TransactionCommittedListener;
+        event SessionTxEventDelegate TransactionRolledBackListener;
+
+        /// <summary>
+        /// An asynchronous listener which can be notified if an error occurs
+        /// </summary>
+        event ExceptionListener ExceptionListener;
+
+        /// <summary>
+        /// An asynchronous listener that is notified when a Fault tolerant connection
+        /// has been interrupted.
+        /// </summary>
+        event ConnectionInterruptedListener ConnectionInterruptedListener;
+
+        /// <summary>
+        /// An asynchronous listener that is notified when a Fault tolerant connection
+        /// has been resumed.
+        /// </summary>
+        event ConnectionResumedListener ConnectionResumedListener;
+        
+        #endregion
+
+		#region Attributes
+
+		TimeSpan RequestTimeout { get; set; }
+
+		bool Transacted { get; }
+
+		AcknowledgementMode AcknowledgementMode { get; }
+		
+		String ClientID { get; set; }
+
+		#endregion
+		
+		/// <summary>
+		/// For a long running Connection that creates many temp destinations
+		/// this method will close and destroy all previously created temp
+		/// destinations to reduce resource consumption.  This can be useful
+		/// when the Connection is pooled or otherwise used for long periods
+		/// of time.  Only locally created temp destinations should be removed
+		/// by this call.
+		/// NOTE: This is an optional operation and for NMS providers that
+		/// do not support this functionality the method should just return
+		/// without throwing any exceptions.
+		/// </summary>
+		void PurgeTempDestinations();
+	}
+}
diff --git a/src/nms-api/INMSProducer.cs b/src/nms-api/INMSProducer.cs
new file mode 100644
index 0000000..961ef48
--- /dev/null
+++ b/src/nms-api/INMSProducer.cs
@@ -0,0 +1,153 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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.Threading.Tasks;
+
+namespace Apache.NMS
+{
+	/// <summary>
+	/// An object capable of sending messages to some destination
+	/// </summary>
+	public interface INMSProducer : System.IDisposable
+	{
+
+		INMSProducer Send(IDestination destination, IMessage message);
+		
+		INMSProducer Send(IDestination destination, String body);
+		
+		INMSProducer Send(IDestination destination, IPrimitiveMap body);
+
+		INMSProducer Send(IDestination destination, byte[] body);
+
+		INMSProducer Send(IDestination destination, object body);
+
+		Task<INMSProducer> SendAsync(IDestination destination, IMessage message);
+		
+		Task<INMSProducer> SendAsync(IDestination destination, String body);
+		
+		Task<INMSProducer> SendAsync(IDestination destination, IPrimitiveMap body);
+
+		Task<INMSProducer> SendAsync(IDestination destination, byte[] body);
+
+		Task<INMSProducer> SendAsync(IDestination destination, object body);
+
+		INMSProducer SetAsync(CompletionListener completionListener);
+
+		CompletionListener GetAsync();
+
+		/// <summary>
+		/// Provides access to the message properties (headers).
+		/// </summary>
+		IPrimitiveMap Properties { get; }
+		
+		/// <summary>
+		/// Clears a message's properties.
+		///
+		/// The message's header fields and body are not cleared.
+		/// </summary>
+		void ClearProperties();
+		
+		/// <summary>
+		/// The correlation ID used to correlate messages from conversations or long running business processes.
+		/// </summary>
+		string NMSCorrelationID { get; set; }
+
+
+		/// <summary>
+		/// The destination that the consumer of this message should send replies to
+		/// </summary>
+		IDestination NMSReplyTo { get; set; }
+
+		/// <summary>
+		/// Specifies that messages sent using this NMSProducer will
+		/// have their NMSType header value set to the specified message type.
+		/// </summary>
+		string NMSType { get; set; }
+		
+		
+		
+
+		/// <summary>
+		/// A delegate that is called each time a Message is sent from this Producer which allows
+		/// the application to perform any needed transformations on the Message before it is sent.
+		/// </summary>
+		ProducerTransformerDelegate ProducerTransformer { get; set; }
+
+		MsgDeliveryMode DeliveryMode { get; set; }
+
+                TimeSpan DeliveryDelay { get; set; }
+
+                TimeSpan TimeToLive { get; set; }
+
+		TimeSpan RequestTimeout { get; set; }
+
+		MsgPriority Priority { get; set; }
+
+		bool DisableMessageID { get; set; }
+
+		bool DisableMessageTimestamp { get; set; }
+
+		#region Factory methods to create messages
+
+		/// <summary>
+		/// Creates a new message with an empty body
+		/// </summary>
+		IMessage CreateMessage();
+
+		/// <summary>
+		/// Creates a new text message with an empty body
+		/// </summary>
+		ITextMessage CreateTextMessage();
+
+		/// <summary>
+		/// Creates a new text message with the given body
+		/// </summary>
+		ITextMessage CreateTextMessage(string text);
+
+		/// <summary>
+		/// Creates a new Map message which contains primitive key and value pairs
+		/// </summary>
+		IMapMessage CreateMapMessage();
+
+		/// <summary>
+		/// Creates a new Object message containing the given .NET object as the body
+		/// </summary>
+		IObjectMessage CreateObjectMessage(object body);
+
+		/// <summary>
+		/// Creates a new binary message
+		/// </summary>
+		IBytesMessage CreateBytesMessage();
+
+		/// <summary>
+		/// Creates a new binary message with the given body
+		/// </summary>
+		IBytesMessage CreateBytesMessage(byte[] body);
+
+		/// <summary>
+		/// Creates a new stream message
+		/// </summary>
+		IStreamMessage CreateStreamMessage();
+
+		#endregion
+		/// <summary>
+		/// Close the producer.
+		/// </summary>
+		void Close();
+
+	}
+}
diff --git a/src/nms-api/ISession.cs b/src/nms-api/ISession.cs
index a54435b..0468a44 100644
--- a/src/nms-api/ISession.cs
+++ b/src/nms-api/ISession.cs
@@ -54,17 +54,32 @@
 		/// Creates a consumer of messages on a given destination with a selector
 		/// </summary>
 		IMessageConsumer CreateConsumer(IDestination destination, string selector, bool noLocal);
+		
+		IMessageConsumer CreateDurableConsumer(ITopic destination, string name);
 
-		/// <summary>
-		/// Creates a named durable consumer of messages on a given destination with a selector
-		/// </summary>
-		IMessageConsumer CreateDurableConsumer(ITopic destination, string name, string selector, bool noLocal);
+       		IMessageConsumer CreateDurableConsumer(ITopic destination, string name, string selector);
 
-		/// <summary>
-		/// Deletes a durable consumer created with CreateDurableConsumer().
-		/// </summary>
-		/// <param name="name">Name of the durable consumer</param>
-		void DeleteDurableConsumer(string name);
+        	/// <summary>
+    	    	/// Creates a named durable consumer of messages on a given destination with a selector
+        	/// </summary>
+        	IMessageConsumer CreateDurableConsumer(ITopic destination, string name, string selector, bool noLocal);
+
+        	IMessageConsumer CreateSharedConsumer(ITopic destination, string name);
+
+      		IMessageConsumer CreateSharedConsumer(ITopic destination, string name, string selector);
+
+       		IMessageConsumer CreateSharedDurableConsumer(ITopic destination, string name);
+
+       		IMessageConsumer CreateSharedDurableConsumer(ITopic destination, string name, string selector);
+
+     	   	/// <summary>
+     	   	/// Deletes a durable consumer created with CreateDurableConsumer().
+    	    	/// </summary>
+    		/// <param name="name">Name of the durable consumer</param>
+      		[Obsolete("should use unsubscribe instead")]
+      		void DeleteDurableConsumer(string name);
+
+      		void Unsubscribe(string name);
 
 		/// <summary>
 		/// Creates a QueueBrowser object to peek at the messages on the specified queue.
@@ -195,6 +210,8 @@
         /// order.  It is not valid to call this method on a Transacted Session.
         /// </summary>
         void Recover();
+	
+	void Acknowledge();
 
 		#region Transaction methods
 
diff --git a/src/nms-api/NMSConnectionFactory.cs b/src/nms-api/NMSConnectionFactory.cs
index b09ee4f..8f26581 100644
--- a/src/nms-api/NMSConnectionFactory.cs
+++ b/src/nms-api/NMSConnectionFactory.cs
@@ -1,440 +1,460 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You 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 System.Collections.Generic;
-using System.IO;
-using System.Reflection;
-using System.Xml;
-
-using Apache.NMS.Util;
-
-namespace Apache.NMS
-{
-	/// <summary>
-	/// Provider implementation mapping class.
-	/// </summary>
-	public class ProviderFactoryInfo
-	{
-		public string assemblyFileName;
-		public string factoryClassName;
-
-		public ProviderFactoryInfo(string aFileName, string fClassName)
-		{
-			assemblyFileName = aFileName;
-			factoryClassName = fClassName;
-		}
-	}
-
-	/// <summary>
-	/// Implementation of a factory for <see cref="IConnection" /> instances.
-	/// </summary>
-	public class NMSConnectionFactory : IConnectionFactory
-	{
-		protected readonly IConnectionFactory factory;
-		protected static readonly Dictionary<string, ProviderFactoryInfo> schemaProviderFactoryMap;
-
-		/// <summary>
-		/// Static class constructor
-		/// </summary>
-		static NMSConnectionFactory()
-		{
-			schemaProviderFactoryMap = new Dictionary<string, ProviderFactoryInfo>();
-            schemaProviderFactoryMap["activemq"] = new ProviderFactoryInfo("Apache.NMS.ActiveMQ", "Apache.NMS.ActiveMQ.ConnectionFactory");
-            schemaProviderFactoryMap["activemqnettx"] = new ProviderFactoryInfo("Apache.NMS.ActiveMQ", "Apache.NMS.ActiveMQ.NetTxConnectionFactory");
-			schemaProviderFactoryMap["tcp"] = new ProviderFactoryInfo("Apache.NMS.ActiveMQ", "Apache.NMS.ActiveMQ.ConnectionFactory");
-			schemaProviderFactoryMap["ems"] = new ProviderFactoryInfo("Apache.NMS.EMS", "Apache.NMS.EMS.ConnectionFactory");
-            schemaProviderFactoryMap["mqtt"] = new ProviderFactoryInfo("Apache.NMS.MQTT", "Apache.NMS.MQTT.ConnectionFactory");
-            schemaProviderFactoryMap["msmq"] = new ProviderFactoryInfo("Apache.NMS.MSMQ", "Apache.NMS.MSMQ.ConnectionFactory");
-			schemaProviderFactoryMap["stomp"] = new ProviderFactoryInfo("Apache.NMS.Stomp", "Apache.NMS.Stomp.ConnectionFactory");
+/*

+ * Licensed to the Apache Software Foundation (ASF) under one or more

+ * contributor license agreements.  See the NOTICE file distributed with

+ * this work for additional information regarding copyright ownership.

+ * The ASF licenses this file to You 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 System.Collections.Generic;

+using System.IO;

+using System.Reflection;

+using System.Xml;

+

+using Apache.NMS.Util;

+

+namespace Apache.NMS

+{

+	/// <summary>

+	/// Provider implementation mapping class.

+	/// </summary>

+	public class ProviderFactoryInfo

+	{

+		public string assemblyFileName;

+		public string factoryClassName;

+

+		public ProviderFactoryInfo(string aFileName, string fClassName)

+		{

+			assemblyFileName = aFileName;

+			factoryClassName = fClassName;

+		}

+	}

+

+	/// <summary>

+	/// Implementation of a factory for <see cref="IConnection" /> instances.

+	/// </summary>

+	public class NMSConnectionFactory : IConnectionFactory

+	{

+		protected readonly IConnectionFactory factory;

+		protected static readonly Dictionary<string, ProviderFactoryInfo> schemaProviderFactoryMap;

+

+		/// <summary>

+		/// Static class constructor

+		/// </summary>

+		static NMSConnectionFactory()

+		{

+			schemaProviderFactoryMap = new Dictionary<string, ProviderFactoryInfo>();

+            schemaProviderFactoryMap["activemq"] = new ProviderFactoryInfo("Apache.NMS.ActiveMQ", "Apache.NMS.ActiveMQ.ConnectionFactory");

+            schemaProviderFactoryMap["activemqnettx"] = new ProviderFactoryInfo("Apache.NMS.ActiveMQ", "Apache.NMS.ActiveMQ.NetTxConnectionFactory");

+			schemaProviderFactoryMap["tcp"] = new ProviderFactoryInfo("Apache.NMS.ActiveMQ", "Apache.NMS.ActiveMQ.ConnectionFactory");

+			schemaProviderFactoryMap["ems"] = new ProviderFactoryInfo("Apache.NMS.EMS", "Apache.NMS.EMS.ConnectionFactory");

+            schemaProviderFactoryMap["mqtt"] = new ProviderFactoryInfo("Apache.NMS.MQTT", "Apache.NMS.MQTT.ConnectionFactory");

+            schemaProviderFactoryMap["msmq"] = new ProviderFactoryInfo("Apache.NMS.MSMQ", "Apache.NMS.MSMQ.ConnectionFactory");

+			schemaProviderFactoryMap["stomp"] = new ProviderFactoryInfo("Apache.NMS.Stomp", "Apache.NMS.Stomp.ConnectionFactory");

 			schemaProviderFactoryMap["xms"] = new ProviderFactoryInfo("Apache.NMS.XMS", "Apache.NMS.XMS.ConnectionFactory");

 			schemaProviderFactoryMap["zmq"] = new ProviderFactoryInfo("Apache.NMS.ZMQ", "Apache.NMS.ZMQ.ConnectionFactory");

 			schemaProviderFactoryMap["amqp"] = new ProviderFactoryInfo("Apache.NMS.AMQP", "Apache.NMS.AMQP.ConnectionFactory");

-		}
-
-		/// <summary>
-		/// The ConnectionFactory object must define a constructor that takes as a minimum a Uri object.
-		/// Any additional parameters are optional, but will typically include a Client ID string.
-		/// </summary>
-		/// <param name="providerURI">The URI for the ActiveMQ provider.</param>
-		/// <param name="constructorParams">Optional parameters to use when creating the ConnectionFactory.</param>
-		public NMSConnectionFactory(string providerURI, params object[] constructorParams)
-			: this(URISupport.CreateCompatibleUri(providerURI), constructorParams)
-		{
-		}
-
-		/// <summary>
-		/// The ConnectionFactory object must define a constructor that takes as a minimum a Uri object.
-		/// Any additional parameters are optional, but will typically include a Client ID string.
-		/// </summary>
-		/// <param name="uriProvider">The URI for the ActiveMQ provider.</param>
-		/// <param name="constructorParams">Optional parameters to use when creating the ConnectionFactory.</param>
-		public NMSConnectionFactory(Uri uriProvider, params object[] constructorParams)
-		{
-			this.factory = CreateConnectionFactory(uriProvider, constructorParams);
-		}
-
-		/// <summary>
-		/// Create a connection factory that can create connections for the given scheme in the URI.
-		/// </summary>
-		/// <param name="uriProvider">The URI for the ActiveMQ provider.</param>
-		/// <param name="constructorParams">Optional parameters to use when creating the ConnectionFactory.</param>
-		/// <returns>A <see cref="IConnectionFactory" /> implementation that will be used.</returns>
-		public static IConnectionFactory CreateConnectionFactory(Uri uriProvider, params object[] constructorParams)
-		{
-			IConnectionFactory connectionFactory = null;
-
-			try
-			{
-				Type factoryType = GetTypeForScheme(uriProvider.Scheme);
-
-				// If an implementation was found, try to instantiate it.
-				if(factoryType != null)
-				{
-#if NETCF
-					// Compact framework does not allow the activator ta pass parameters to a constructor.
-					connectionFactory = (IConnectionFactory) Activator.CreateInstance(factoryType);
-					connectionFactory.BrokerUri = uriProvider;
-#else
-					object[] parameters = MakeParameterArray(uriProvider, constructorParams);
-					connectionFactory = (IConnectionFactory) Activator.CreateInstance(factoryType, parameters);
-#endif
-				}
-
-				if(null == connectionFactory)
-				{
-					throw new NMSConnectionException("No IConnectionFactory implementation found for connection URI: " + uriProvider);
-				}
-			}
-			catch(NMSConnectionException)
-			{
-				throw;
-			}
-			catch(Exception ex)
-			{
-				throw new NMSConnectionException("Could not create the IConnectionFactory implementation: " + ex.Message, ex);
-			}
-
-			return connectionFactory;
-		}
-
-		/// <summary>
-		/// Finds the <see cref="System.Type" /> associated with the given scheme.
-		/// </summary>
-		/// <param name="scheme">The scheme (e.g. <c>tcp</c>, <c>activemq</c> or <c>stomp</c>).</param>
-		/// <returns>The <see cref="System.Type" /> of the ConnectionFactory that will be used
-		/// to create the connection for the specified <paramref name="scheme" />.</returns>
-		private static Type GetTypeForScheme(string scheme)
-		{
-			string[] paths = GetConfigSearchPaths();
-			string assemblyFileName;
-			string factoryClassName;
-			Type factoryType = null;
-
-			Tracer.DebugFormat("Locating provider for scheme: {0}", scheme);
-			if(LookupConnectionFactoryInfo(paths, scheme, out assemblyFileName, out factoryClassName))
-			{
-				Assembly assembly = null;
-
-				Tracer.DebugFormat("Attempting to load provider assembly: {0}", assemblyFileName);
-				try
-				{
-					assembly = Assembly.Load(assemblyFileName);
-					if(null != assembly)
-					{
-						Tracer.Debug("Succesfully loaded provider.");
-					}
-				}
-				catch(Exception ex)
-				{
-					Tracer.ErrorFormat("Exception loading assembly failed: {0}", ex.Message);
-					assembly = null;
-				}
-
-				if(null == assembly)
-				{
-					foreach(string path in paths)
-					{
-						string fullpath = Path.Combine(path, assemblyFileName) + ".dll";
-
-						Tracer.DebugFormat("Looking for: {0}", fullpath);
-						if(File.Exists(fullpath))
-						{
-							Tracer.Debug("\tAssembly found!  Attempting to load...");
-							try
-							{
-								assembly = Assembly.LoadFrom(fullpath);
-							}
-							catch(Exception ex)
-							{
-								Tracer.ErrorFormat("Exception loading assembly failed: {0}", ex.Message);
-								assembly = null;
-							}
-
-							if(null != assembly)
-							{
-								Tracer.Debug("Successfully loaded provider.");
-								break;
-							}
-
-							Tracer.Debug("Failed to load provider.  Continuing search...");
-						}
-					}
-				}
-
-				if(null != assembly)
-				{
-#if NETCF
-					factoryType = assembly.GetType(factoryClassName, true);
-#else
-					factoryType = assembly.GetType(factoryClassName, true, true);
-#endif
-					if(null == factoryType)
-					{
-						Tracer.Fatal("Failed to load class factory from provider.");
-					}
-				}
-				else
-				{
-					Tracer.Fatal("Failed to load provider assembly.");
-				}
-			}
-
-			return factoryType;
-		}
-
-		/// <summary>
-		/// Lookup the connection factory assembly filename and class name.
-		/// Read an external configuration file that maps scheme to provider implementation.
-		/// Load XML config files named: nmsprovider-{scheme}.config
-		/// Following is a sample configuration file named nmsprovider-jms.config.  Replace
-		/// the parenthesis with angle brackets for proper XML formatting.
-		///
-		///     (?xml version="1.0" encoding="utf-8" ?)
-		///     (configuration)
-		///         (provider assembly="MyCompany.NMS.JMSProvider.dll" classFactory="MyCompany.NMS.JMSProvider.ConnectionFactory"/)
-		///     (/configuration)
-		///
-		/// This configuration file would be loaded and parsed when a connection uri with a scheme of 'jms'
-		/// is used for the provider.  In this example the connection string might look like:
-		///     jms://localhost:7222
-		///
-		/// </summary>
-		/// <param name="paths">Folder paths to look in.</param>
-		/// <param name="scheme">The scheme.</param>
-		/// <param name="assemblyFileName">Name of the assembly file.</param>
-		/// <param name="factoryClassName">Name of the factory class.</param>
-		/// <returns><c>true</c> if the configuration file for the specified <paramref name="scheme" /> could
-		/// be found; otherwise, <c>false</c>.</returns>
-		private static bool LookupConnectionFactoryInfo(string[] paths, string scheme, out string assemblyFileName, out string factoryClassName)
-		{
-			bool foundFactory = false;
-			string schemeLower = scheme.ToLower();
-			ProviderFactoryInfo pfi;
-
-			// Look for a custom configuration to handle this scheme.
-			string configFileName = String.Format("nmsprovider-{0}.config", schemeLower);
-
-			assemblyFileName = String.Empty;
-			factoryClassName = String.Empty;
-
-			Tracer.DebugFormat("Attempting to locate provider configuration: {0}", configFileName);
-			foreach(string path in paths)
-			{
-				string fullpath = Path.Combine(path, configFileName);
-				Tracer.DebugFormat("Looking for: {0}", fullpath);
-
-				try
-				{
-					if(File.Exists(fullpath))
-					{
-						Tracer.DebugFormat("\tConfiguration file found in {0}", fullpath);
-						XmlDocument configDoc = new XmlDocument();
-
-						configDoc.Load(fullpath);
-						XmlElement providerNode = (XmlElement) configDoc.SelectSingleNode("/configuration/provider");
-
-						if(null != providerNode)
-						{
-							assemblyFileName = providerNode.GetAttribute("assembly");
-							factoryClassName = providerNode.GetAttribute("classFactory");
-							if(!String.IsNullOrEmpty(assemblyFileName) && !String.IsNullOrEmpty(factoryClassName))
-							{
-								foundFactory = true;
-								Tracer.DebugFormat("Selected custom provider for {0}: {1}, {2}", schemeLower, assemblyFileName, factoryClassName);
-								break;
-							}
-						}
-					}
-				}
-				catch(Exception ex)
-				{
-					Tracer.DebugFormat("Exception while scanning {0}: {1}", fullpath, ex.Message);
-				}
-			}
-
-			if(!foundFactory)
-			{
-				// Check for standard provider implementations.
-				if(schemaProviderFactoryMap.TryGetValue(schemeLower, out pfi))
-				{
-					assemblyFileName = pfi.assemblyFileName;
-					factoryClassName = pfi.factoryClassName;
-					foundFactory = true;
-					Tracer.DebugFormat("Selected standard provider for {0}: {1}, {2}", schemeLower, assemblyFileName, factoryClassName);
-				}
-			}
-
-			return foundFactory;
-		}
-
-		/// <summary>
-		/// Get an array of search paths to look for config files.
-		/// </summary>
-		/// <returns>
-		/// A collection of search paths, including the current directory, the current AppDomain's
-		/// BaseDirectory and the current AppDomain's RelativeSearchPath.
-		/// </returns>
-		private static string[] GetConfigSearchPaths()
-		{
-			ArrayList pathList = new ArrayList();
-
-			// Check the current folder first.
-			pathList.Add("");
-#if !NETCF
-			try
-			{
-				AppDomain currentDomain = AppDomain.CurrentDomain;
-
-				// Check the folder the assembly is located in.
-				Assembly executingAssembly = Assembly.GetExecutingAssembly();
-				try
-				{
-					pathList.Add(Path.GetDirectoryName(executingAssembly.Location));
-				}
-				catch(Exception ex)
-				{
-					Tracer.DebugFormat("Error parsing executing assembly location: {0} : {1}", executingAssembly.Location, ex.Message);
-				}
-
-				if(null != currentDomain.BaseDirectory)
-				{
-					pathList.Add(currentDomain.BaseDirectory);
-				}
-
-				if(null != currentDomain.RelativeSearchPath)
-				{
-					pathList.Add(currentDomain.RelativeSearchPath);
-				}
-			}
-			catch(Exception ex)
-			{
-				Tracer.DebugFormat("Error configuring search paths: {0}", ex.Message);
-			}
-#endif
-
-			return (string[]) pathList.ToArray(typeof(string));
-		}
-
-		/// <summary>
-		/// Converts a <c>params object[]</c> collection into a plain <c>object[]</c>s, to pass to the constructor.
-		/// </summary>
-		/// <param name="firstParam">The first parameter in the collection.</param>
-		/// <param name="varParams">The remaining parameters.</param>
-		/// <returns>An array of <see cref="Object" /> instances.</returns>
-		private static object[] MakeParameterArray(object firstParam, params object[] varParams)
-		{
-			ArrayList paramList = new ArrayList();
-			paramList.Add(firstParam);
-			foreach(object param in varParams)
-			{
-				paramList.Add(param);
-			}
-
-			return paramList.ToArray();
-		}
-
-		/// <summary>
-		/// Creates a new connection.
-		/// </summary>
-		/// <returns>An <see cref="IConnection" /> created by the requested ConnectionFactory.</returns>
-		public IConnection CreateConnection()
-		{
-			return this.factory.CreateConnection();
-		}
-
-		/// <summary>
-		/// Creates a new connection with the given <paramref name="userName" /> and <paramref name="password" /> credentials.
-		/// </summary>
-		/// <param name="userName">The username to use when establishing the connection.</param>
-		/// <param name="password">The password to use when establishing the connection.</param>
-		/// <returns>An <see cref="IConnection" /> created by the requested ConnectionFactory.</returns>
-		public IConnection CreateConnection(string userName, string password)
-		{
-			return this.factory.CreateConnection(userName, password);
-		}
-
-		/// <summary>
-		/// Get/or set the broker Uri.
-		/// </summary>
-		public Uri BrokerUri
-		{
-			get { return ConnectionFactory.BrokerUri; }
-			set { ConnectionFactory.BrokerUri = value; }
-		}
-
-		/// <summary>
-		/// The actual IConnectionFactory implementation that is being used.  This implementation
-		/// depends on the scheme of the URI used when constructed.
-		/// </summary>
-		public IConnectionFactory ConnectionFactory
-		{
-			get { return factory; }
-		}
-
-		/// <summary>
-		/// Get/or Set the IRedeliveryPolicy instance using the IConnectionFactory implementation
-		/// that is being used.
-		/// </summary>
-		public IRedeliveryPolicy RedeliveryPolicy
-		{
-			get { return this.factory.RedeliveryPolicy; }
-			set { this.factory.RedeliveryPolicy = value; }
-		}
-
-        /// <summary>
-        /// Get/or Set the ConsumerTransformerDelegate using the IConnectionFactory implementation
-        /// that is currently being used.
-        /// </summary>
-        public ConsumerTransformerDelegate ConsumerTransformer
-        {
-            get { return this.factory.ConsumerTransformer; }
-            set { this.factory.ConsumerTransformer = value; }
-        }
-
-        /// <summary>
-        /// Get/or Set the ProducerTransformerDelegate using the IConnectionFactory implementation
-        /// that is currently being used.
-        /// </summary>
-        public ProducerTransformerDelegate ProducerTransformer
-        {
-            get { return this.factory.ProducerTransformer; }
-            set { this.factory.ProducerTransformer = value; }
-        }
-	}
-}
+		}

+

+		/// <summary>

+		/// The ConnectionFactory object must define a constructor that takes as a minimum a Uri object.

+		/// Any additional parameters are optional, but will typically include a Client ID string.

+		/// </summary>

+		/// <param name="providerURI">The URI for the ActiveMQ provider.</param>

+		/// <param name="constructorParams">Optional parameters to use when creating the ConnectionFactory.</param>

+		public NMSConnectionFactory(string providerURI, params object[] constructorParams)

+			: this(URISupport.CreateCompatibleUri(providerURI), constructorParams)

+		{

+		}

+

+		/// <summary>

+		/// The ConnectionFactory object must define a constructor that takes as a minimum a Uri object.

+		/// Any additional parameters are optional, but will typically include a Client ID string.

+		/// </summary>

+		/// <param name="uriProvider">The URI for the ActiveMQ provider.</param>

+		/// <param name="constructorParams">Optional parameters to use when creating the ConnectionFactory.</param>

+		public NMSConnectionFactory(Uri uriProvider, params object[] constructorParams)

+		{

+			this.factory = CreateConnectionFactory(uriProvider, constructorParams);

+		}

+

+		/// <summary>

+		/// Create a connection factory that can create connections for the given scheme in the URI.

+		/// </summary>

+		/// <param name="uriProvider">The URI for the ActiveMQ provider.</param>

+		/// <param name="constructorParams">Optional parameters to use when creating the ConnectionFactory.</param>

+		/// <returns>A <see cref="IConnectionFactory" /> implementation that will be used.</returns>

+		public static IConnectionFactory CreateConnectionFactory(Uri uriProvider, params object[] constructorParams)

+		{

+			IConnectionFactory connectionFactory = null;

+

+			try

+			{

+				Type factoryType = GetTypeForScheme(uriProvider.Scheme);

+

+				// If an implementation was found, try to instantiate it.

+				if(factoryType != null)

+				{

+#if NETCF

+					// Compact framework does not allow the activator ta pass parameters to a constructor.

+					connectionFactory = (IConnectionFactory) Activator.CreateInstance(factoryType);

+					connectionFactory.BrokerUri = uriProvider;

+#else

+					object[] parameters = MakeParameterArray(uriProvider, constructorParams);

+					connectionFactory = (IConnectionFactory) Activator.CreateInstance(factoryType, parameters);

+#endif

+				}

+

+				if(null == connectionFactory)

+				{

+					throw new NMSConnectionException("No IConnectionFactory implementation found for connection URI: " + uriProvider);

+				}

+			}

+			catch(NMSConnectionException)

+			{

+				throw;

+			}

+			catch(Exception ex)

+			{

+				throw new NMSConnectionException("Could not create the IConnectionFactory implementation: " + ex.Message, ex);

+			}

+

+			return connectionFactory;

+		}

+

+		/// <summary>

+		/// Finds the <see cref="System.Type" /> associated with the given scheme.

+		/// </summary>

+		/// <param name="scheme">The scheme (e.g. <c>tcp</c>, <c>activemq</c> or <c>stomp</c>).</param>

+		/// <returns>The <see cref="System.Type" /> of the ConnectionFactory that will be used

+		/// to create the connection for the specified <paramref name="scheme" />.</returns>

+		private static Type GetTypeForScheme(string scheme)

+		{

+			string[] paths = GetConfigSearchPaths();

+			string assemblyFileName;

+			string factoryClassName;

+			Type factoryType = null;

+

+			Tracer.DebugFormat("Locating provider for scheme: {0}", scheme);

+			if(LookupConnectionFactoryInfo(paths, scheme, out assemblyFileName, out factoryClassName))

+			{

+				Assembly assembly = null;

+

+				Tracer.DebugFormat("Attempting to load provider assembly: {0}", assemblyFileName);

+				try

+				{

+					assembly = Assembly.Load(assemblyFileName);

+					if(null != assembly)

+					{

+						Tracer.Debug("Succesfully loaded provider.");

+					}

+				}

+				catch(Exception ex)

+				{

+					Tracer.ErrorFormat("Exception loading assembly failed: {0}", ex.Message);

+					assembly = null;

+				}

+

+				if(null == assembly)

+				{

+					foreach(string path in paths)

+					{

+						string fullpath = Path.Combine(path, assemblyFileName) + ".dll";

+

+						Tracer.DebugFormat("Looking for: {0}", fullpath);

+						if(File.Exists(fullpath))

+						{

+							Tracer.Debug("\tAssembly found!  Attempting to load...");

+							try

+							{

+								assembly = Assembly.LoadFrom(fullpath);

+							}

+							catch(Exception ex)

+							{

+								Tracer.ErrorFormat("Exception loading assembly failed: {0}", ex.Message);

+								assembly = null;

+							}

+

+							if(null != assembly)

+							{

+								Tracer.Debug("Successfully loaded provider.");

+								break;

+							}

+

+							Tracer.Debug("Failed to load provider.  Continuing search...");

+						}

+					}

+				}

+

+				if(null != assembly)

+				{

+#if NETCF

+					factoryType = assembly.GetType(factoryClassName, true);

+#else

+					factoryType = assembly.GetType(factoryClassName, true, true);

+#endif

+					if(null == factoryType)

+					{

+						Tracer.Fatal("Failed to load class factory from provider.");

+					}

+				}

+				else

+				{

+					Tracer.Fatal("Failed to load provider assembly.");

+				}

+			}

+

+			return factoryType;

+		}

+

+		/// <summary>

+		/// Lookup the connection factory assembly filename and class name.

+		/// Read an external configuration file that maps scheme to provider implementation.

+		/// Load XML config files named: nmsprovider-{scheme}.config

+		/// Following is a sample configuration file named nmsprovider-jms.config.  Replace

+		/// the parenthesis with angle brackets for proper XML formatting.

+		///

+		///     (?xml version="1.0" encoding="utf-8" ?)

+		///     (configuration)

+		///         (provider assembly="MyCompany.NMS.JMSProvider.dll" classFactory="MyCompany.NMS.JMSProvider.ConnectionFactory"/)

+		///     (/configuration)

+		///

+		/// This configuration file would be loaded and parsed when a connection uri with a scheme of 'jms'

+		/// is used for the provider.  In this example the connection string might look like:

+		///     jms://localhost:7222

+		///

+		/// </summary>

+		/// <param name="paths">Folder paths to look in.</param>

+		/// <param name="scheme">The scheme.</param>

+		/// <param name="assemblyFileName">Name of the assembly file.</param>

+		/// <param name="factoryClassName">Name of the factory class.</param>

+		/// <returns><c>true</c> if the configuration file for the specified <paramref name="scheme" /> could

+		/// be found; otherwise, <c>false</c>.</returns>

+		private static bool LookupConnectionFactoryInfo(string[] paths, string scheme, out string assemblyFileName, out string factoryClassName)

+		{

+			bool foundFactory = false;

+			string schemeLower = scheme.ToLower();

+			ProviderFactoryInfo pfi;

+

+			// Look for a custom configuration to handle this scheme.

+			string configFileName = String.Format("nmsprovider-{0}.config", schemeLower);

+

+			assemblyFileName = String.Empty;

+			factoryClassName = String.Empty;

+

+			Tracer.DebugFormat("Attempting to locate provider configuration: {0}", configFileName);

+			foreach(string path in paths)

+			{

+				string fullpath = Path.Combine(path, configFileName);

+				Tracer.DebugFormat("Looking for: {0}", fullpath);

+

+				try

+				{

+					if(File.Exists(fullpath))

+					{

+						Tracer.DebugFormat("\tConfiguration file found in {0}", fullpath);

+						XmlDocument configDoc = new XmlDocument();

+

+						configDoc.Load(fullpath);

+						XmlElement providerNode = (XmlElement) configDoc.SelectSingleNode("/configuration/provider");

+

+						if(null != providerNode)

+						{

+							assemblyFileName = providerNode.GetAttribute("assembly");

+							factoryClassName = providerNode.GetAttribute("classFactory");

+							if(!String.IsNullOrEmpty(assemblyFileName) && !String.IsNullOrEmpty(factoryClassName))

+							{

+								foundFactory = true;

+								Tracer.DebugFormat("Selected custom provider for {0}: {1}, {2}", schemeLower, assemblyFileName, factoryClassName);

+								break;

+							}

+						}

+					}

+				}

+				catch(Exception ex)

+				{

+					Tracer.DebugFormat("Exception while scanning {0}: {1}", fullpath, ex.Message);

+				}

+			}

+

+			if(!foundFactory)

+			{

+				// Check for standard provider implementations.

+				if(schemaProviderFactoryMap.TryGetValue(schemeLower, out pfi))

+				{

+					assemblyFileName = pfi.assemblyFileName;

+					factoryClassName = pfi.factoryClassName;

+					foundFactory = true;

+					Tracer.DebugFormat("Selected standard provider for {0}: {1}, {2}", schemeLower, assemblyFileName, factoryClassName);

+				}

+			}

+

+			return foundFactory;

+		}

+

+		/// <summary>

+		/// Get an array of search paths to look for config files.

+		/// </summary>

+		/// <returns>

+		/// A collection of search paths, including the current directory, the current AppDomain's

+		/// BaseDirectory and the current AppDomain's RelativeSearchPath.

+		/// </returns>

+		private static string[] GetConfigSearchPaths()

+		{

+			ArrayList pathList = new ArrayList();

+

+			// Check the current folder first.

+			pathList.Add("");

+#if !NETCF

+			try

+			{

+				AppDomain currentDomain = AppDomain.CurrentDomain;

+

+				// Check the folder the assembly is located in.

+				Assembly executingAssembly = Assembly.GetExecutingAssembly();

+				try

+				{

+					pathList.Add(Path.GetDirectoryName(executingAssembly.Location));

+				}

+				catch(Exception ex)

+				{

+					Tracer.DebugFormat("Error parsing executing assembly location: {0} : {1}", executingAssembly.Location, ex.Message);

+				}

+

+				if(null != currentDomain.BaseDirectory)

+				{

+					pathList.Add(currentDomain.BaseDirectory);

+				}

+

+				if(null != currentDomain.RelativeSearchPath)

+				{

+					pathList.Add(currentDomain.RelativeSearchPath);

+				}

+			}

+			catch(Exception ex)

+			{

+				Tracer.DebugFormat("Error configuring search paths: {0}", ex.Message);

+			}

+#endif

+

+			return (string[]) pathList.ToArray(typeof(string));

+		}

+

+		/// <summary>

+		/// Converts a <c>params object[]</c> collection into a plain <c>object[]</c>s, to pass to the constructor.

+		/// </summary>

+		/// <param name="firstParam">The first parameter in the collection.</param>

+		/// <param name="varParams">The remaining parameters.</param>

+		/// <returns>An array of <see cref="Object" /> instances.</returns>

+		private static object[] MakeParameterArray(object firstParam, params object[] varParams)

+		{

+			ArrayList paramList = new ArrayList();

+			paramList.Add(firstParam);

+			foreach(object param in varParams)

+			{

+				paramList.Add(param);

+			}

+

+			return paramList.ToArray();

+		}

+

+		/// <summary>

+		/// Creates a new connection.

+		/// </summary>

+		/// <returns>An <see cref="IConnection" /> created by the requested ConnectionFactory.</returns>

+		public IConnection CreateConnection()

+		{

+			return this.factory.CreateConnection();

+		}

+

+		/// <summary>

+		/// Creates a new connection with the given <paramref name="userName" /> and <paramref name="password" /> credentials.

+		/// </summary>

+		/// <param name="userName">The username to use when establishing the connection.</param>

+		/// <param name="password">The password to use when establishing the connection.</param>

+		/// <returns>An <see cref="IConnection" /> created by the requested ConnectionFactory.</returns>

+		public IConnection CreateConnection(string userName, string password)

+		{

+			return this.factory.CreateConnection(userName, password);

+		}

+		

+		public INMSContext CreateContext()

+		{

+			return this.factory.CreateContext();

+		}

+		

+		public INMSContext CreateContext(AcknowledgementMode acknowledgementMode)

+		{

+			return this.factory.CreateContext(acknowledgementMode);

+		}

+		

+		public INMSContext CreateContext(string userName, string password)

+		{

+			return this.factory.CreateContext(userName, password);

+		}

+		

+		public INMSContext CreateContext(string userName, string password, AcknowledgementMode acknowledgementMode)

+		{

+			return this.factory.CreateContext(userName, password, acknowledgementMode);

+		}

+

+		/// <summary>

+		/// Get/or set the broker Uri.

+		/// </summary>

+		public Uri BrokerUri

+		{

+			get { return ConnectionFactory.BrokerUri; }

+			set { ConnectionFactory.BrokerUri = value; }

+		}

+

+		/// <summary>

+		/// The actual IConnectionFactory implementation that is being used.  This implementation

+		/// depends on the scheme of the URI used when constructed.

+		/// </summary>

+		public IConnectionFactory ConnectionFactory

+		{

+			get { return factory; }

+		}

+

+		/// <summary>

+		/// Get/or Set the IRedeliveryPolicy instance using the IConnectionFactory implementation

+		/// that is being used.

+		/// </summary>

+		public IRedeliveryPolicy RedeliveryPolicy

+		{

+			get { return this.factory.RedeliveryPolicy; }

+			set { this.factory.RedeliveryPolicy = value; }

+		}

+

+        /// <summary>

+        /// Get/or Set the ConsumerTransformerDelegate using the IConnectionFactory implementation

+        /// that is currently being used.

+        /// </summary>

+        public ConsumerTransformerDelegate ConsumerTransformer

+        {

+            get { return this.factory.ConsumerTransformer; }

+            set { this.factory.ConsumerTransformer = value; }

+        }

+

+        /// <summary>

+        /// Get/or Set the ProducerTransformerDelegate using the IConnectionFactory implementation

+        /// that is currently being used.

+        /// </summary>

+        public ProducerTransformerDelegate ProducerTransformer

+        {

+            get { return this.factory.ProducerTransformer; }

+            set { this.factory.ProducerTransformer = value; }

+        }

+	}

+}

diff --git a/src/nms-api/nms-api.csproj b/src/nms-api/nms-api.csproj
index a5e210f..113e87d 100644
--- a/src/nms-api/nms-api.csproj
+++ b/src/nms-api/nms-api.csproj
@@ -19,7 +19,7 @@
   <PropertyGroup>

     <Platform Condition=" '$(Platform)' == '' ">AnyCPU</Platform>

     <OutputType>Library</OutputType>

-    <TargetFrameworks>net35;net40;netstandard2.0</TargetFrameworks>

+    <TargetFrameworks>net40;netstandard2.0</TargetFrameworks>

     <RootNamespace>Apache.NMS</RootNamespace>

     <AssemblyName>Apache.NMS</AssemblyName>

   </PropertyGroup>

@@ -52,11 +52,6 @@
     <None Include="..\..\NOTICE.txt" Pack="true" PackagePath="NOTICE.txt"/>

   </ItemGroup>

 

-  <ItemGroup Condition=" '$(TargetFramework)' == 'net35' ">

-    <Reference Include="System.Transactions" />

-    <Reference Include="System.Web" />

-  </ItemGroup>

-

   <ItemGroup Condition=" '$(TargetFramework)' == 'net40' ">

     <Reference Include="System.Transactions" />

     <Reference Include="System.Web" />