You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@activemq.apache.org by ch...@apache.org on 2011/06/08 18:46:36 UTC

svn commit: r1133464 - in /activemq/activemq-apollo/trunk: apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/ apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/ apollo-cli/src/main/resources/org/apache/activemq/apollo...

Author: chirino
Date: Wed Jun  8 16:46:36 2011
New Revision: 1133464

URL: http://svn.apache.org/viewvc?rev=1133464&view=rev
Log:
Fixes https://issues.apache.org/jira/browse/APLO-38 : Add a JAAS login module which can authorize/reject based on IP addresses

Added:
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SocketAddressCallback.scala
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SocketAddressLoginModule.scala
    activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/black-list.txt
Modified:
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Connection.scala
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/QueueBinding.scala
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/JaasAuthenticator.scala
    activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SecurityContext.scala
    activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/login.config
    activemq/activemq-apollo/trunk/apollo-cli/src/main/scala/org/apache/activemq/apollo/cli/commands/Create.scala
    activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ConnectionStatusDTO.java
    activemq/activemq-apollo/trunk/apollo-tcp/src/main/java/org/apache/activemq/apollo/transport/tcp/TcpTransport.java
    activemq/activemq-apollo/trunk/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/Transport.java
    activemq/activemq-apollo/trunk/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/TransportFilter.java
    activemq/activemq-apollo/trunk/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/pipe/PipeTransport.java
    activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/resources/BrokerResource.scala
    activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/resources/Support.scala

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Connection.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Connection.scala?rev=1133464&r1=1133463&r2=1133464&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Connection.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/Connection.scala Wed Jun  8 16:46:36 2011
@@ -141,7 +141,8 @@ class BrokerConnection(var connector: Co
     result.state_since = service_state.since
     result.protocol = protocol_handler.protocol
     result.transport = transport.getTypeId
-    result.remote_address = transport.getRemoteAddress
+    result.remote_address = transport.getRemoteAddress.toString
+    result.local_address = transport.getLocalAddress.toString
     val wf = transport.getProtocolCodec
     if( wf!=null ) {
       result.write_counter = wf.getWriteCounter

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/QueueBinding.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/QueueBinding.scala?rev=1133464&r1=1133463&r2=1133464&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/QueueBinding.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/QueueBinding.scala Wed Jun  8 16:46:36 2011
@@ -24,6 +24,7 @@ import org.apache.activemq.apollo.util.p
 import java.lang.String
 import org.fusesource.hawtbuf.{Buffer, AsciiBuffer}
 import Buffer._
+import javax.management.remote.rmi._RMIConnection_Stub
 
 /**
  * <p>
@@ -275,7 +276,7 @@ object TempQueueBinding extends QueueBin
 class TempQueueBinding(val key:AnyRef, val id:String) extends QueueBinding {
   import TempQueueBinding._
 
-  def this(c:DeliveryConsumer) = this(c, c.connection.map(_.transport.getRemoteAddress).getOrElse("known") )
+  def this(c:DeliveryConsumer) = this(c, c.connection.map(_.transport.getRemoteAddress.toString).getOrElse("known") )
 
   val destination = null
   def binding_kind = TEMP_KIND

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/JaasAuthenticator.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/JaasAuthenticator.scala?rev=1133464&r1=1133463&r2=1133464&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/JaasAuthenticator.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/JaasAuthenticator.scala Wed Jun  8 16:46:36 2011
@@ -91,6 +91,9 @@ class JaasAuthenticator(val config: Auth
                 case x: NameCallback => x.setName(security_ctx.user)
                 case x: PasswordCallback => x.setPassword(Option(security_ctx.password).getOrElse("").toCharArray)
                 case x: CertificateCallback => x.setCertificates(security_ctx.certificates)
+                case x: SocketAddressCallback =>
+                  x.local = security_ctx.local_address
+                  x.remote = security_ctx.remote_address
                 case _ => throw new UnsupportedCallbackException(callback)
               }
           }

Modified: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SecurityContext.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SecurityContext.scala?rev=1133464&r1=1133463&r2=1133464&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SecurityContext.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SecurityContext.scala Wed Jun  8 16:46:36 2011
@@ -24,6 +24,7 @@ import org.apache.activemq.apollo.util.O
 import org.apache.activemq.jaas.{GroupPrincipal, UserPrincipal}
 import org.apache.activemq.apollo.dto.PrincipalDTO
 import javax.security.auth.login.LoginContext
+import java.net.SocketAddress
 
 /**
  * <p>
@@ -36,8 +37,8 @@ class SecurityContext {
   var user:String = _
   var password:String = _
   var certificates:Array[X509Certificate] = _
-  var local_address:String = _
-  var remote_address:String = _
+  var local_address:SocketAddress = _
+  var remote_address:SocketAddress = _
 
   var login_context:LoginContext = _
 

Added: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SocketAddressCallback.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SocketAddressCallback.scala?rev=1133464&view=auto
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SocketAddressCallback.scala (added)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SocketAddressCallback.scala Wed Jun  8 16:46:36 2011
@@ -0,0 +1,39 @@
+/**
+  * 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.
+  */
+package org.apache.activemq.apollo.broker.security
+
+import javax.security.auth.callback.Callback
+import reflect.BeanProperty
+import java.net.SocketAddress
+
+/**
+  * A Callback for Socket Addresses.
+  *
+  * Will return a local and remote address to its client.
+  *
+  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
+  *
+  */
+class SocketAddressCallback extends Callback {
+
+  @BeanProperty
+  var local: SocketAddress = _
+
+  @BeanProperty
+  var remote: SocketAddress = _
+}
+

Added: activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SocketAddressLoginModule.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SocketAddressLoginModule.scala?rev=1133464&view=auto
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SocketAddressLoginModule.scala (added)
+++ activemq/activemq-apollo/trunk/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/security/SocketAddressLoginModule.scala Wed Jun  8 16:46:36 2011
@@ -0,0 +1,138 @@
+package org.apache.activemq.apollo.broker.security
+
+/**
+ * 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.
+ */
+
+import javax.security.auth.Subject
+import javax.security.auth.callback.CallbackHandler
+import javax.security.auth.callback.UnsupportedCallbackException
+import javax.security.auth.login.LoginException
+import java.{util => ju}
+import java.io.{File, IOException}
+import org.apache.activemq.apollo.util.{FileSupport, Log}
+import java.lang.String
+import javax.security.auth.spi.LoginModule
+import java.net.{InetSocketAddress, SocketAddress, InetAddress}
+
+/**
+ * <p>
+ * </p>
+ *
+ * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
+ */
+object SocketAddressLoginModule {
+  val LOGIN_CONFIG = "java.security.auth.login.config"
+  val WHITE_LIST_OPTION = "white_list_file"
+  val BLACK_LIST_OPTION = "black_list_file"
+  val DEFAULT_LOG = Log(getClass)
+}
+
+/**
+ * <p>
+ * </p>
+ *
+ * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
+ */
+class SocketAddressLoginModule extends LoginModule {
+
+  import SocketAddressLoginModule._
+
+  val log = JaasAuthenticator.broker_log.getOrElse(DEFAULT_LOG)
+  import log._
+
+  var callback_handler: CallbackHandler = _
+
+  var white_list_file: Option[File] = None
+  var black_list_file: Option[File] = None
+
+  /**
+   * Overriding to allow for proper initialization. Standard JAAS.
+   */
+  def initialize(subject: Subject, callback_handler: CallbackHandler, shared_state: ju.Map[String, _], options: ju.Map[String, _]): Unit = {
+    this.callback_handler = callback_handler
+
+    val base_dir = if (System.getProperty(LOGIN_CONFIG) != null) {
+      new File(System.getProperty(LOGIN_CONFIG)).getParentFile()
+    } else {
+      new File(".")
+    }
+
+    white_list_file = Option(options.get(WHITE_LIST_OPTION)).map(x=> new File(base_dir,x.asInstanceOf[String]))
+    black_list_file = Option(options.get(BLACK_LIST_OPTION)).map(x=> new File(base_dir,x.asInstanceOf[String]))
+
+    debug("Initialized white_list_file=%s, black_list_file=%s", white_list_file, black_list_file)
+  }
+
+  def login: Boolean = {
+    val address_callback = new SocketAddressCallback()
+    try {
+      callback_handler.handle(Array(address_callback))
+    } catch {
+      case ioe: IOException =>
+        throw new LoginException(ioe.getMessage())
+      case uce: UnsupportedCallbackException =>
+        return false;
+    }
+
+    if( address_callback.remote==null ) {
+      return false;
+    }
+
+    white_list_file match {
+      case None =>
+      case Some(file)=>
+        if( !matches(file, address_callback.remote) ) {
+          throw new LoginException("Remote address is not whitelisted.")
+        }
+    }
+
+    black_list_file match {
+      case None =>
+      case Some(file)=>
+        if( matches(file, address_callback.remote) ) {
+          throw new LoginException("Remote address blacklisted.")
+        }
+    }
+
+    return true
+  }
+
+  def matches(file:File, address:SocketAddress):Boolean = {
+
+    val needle = address match {
+      case address:InetSocketAddress =>
+        address.getAddress.getHostAddress
+      case _ => return false
+    }
+
+    import FileSupport._
+    file.read_text().split("\n").find( _.trim() == needle ).isDefined
+  }
+
+  def commit: Boolean = {
+    return true
+  }
+
+  def abort: Boolean = {
+    return true
+  }
+
+  def logout: Boolean = {
+    return true
+  }
+
+}
\ No newline at end of file

Added: activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/black-list.txt
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/black-list.txt?rev=1133464&view=auto
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/black-list.txt (added)
+++ activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/black-list.txt Wed Jun  8 16:46:36 2011
@@ -0,0 +1,7 @@
+# Add to this file the IP address which you want to prohibit
+# from connecting to this server.  Sorry address ranges
+# by netmask are not supported yet.
+#
+
+4.4.4.4
+4.4.4.3
\ No newline at end of file

Modified: activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/login.config
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/login.config?rev=1133464&r1=1133463&r2=1133464&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/login.config (original)
+++ activemq/activemq-apollo/trunk/apollo-cli/src/main/resources/org/apache/activemq/apollo/cli/commands/etc/login.config Wed Jun  8 16:46:36 2011
@@ -17,6 +17,18 @@
 apollo {
 
   //
+  // Support accepting/rejecting connections based on the remote IP address.
+  //
+  // The black-list.txt is a text file that contains a new line separated IP address
+  // which are not allowed to connect to this server.
+  //
+  org.apache.activemq.apollo.broker.security.SocketAddressLoginModule required
+    // Uncomment to use a while list of allowed address that can connect to us
+    // white_list_file="white-list.txt"
+    black_list_file="black-list.txt"
+    ;
+
+  //
   // Allow ssl certificate based authentication.  All certificates trusted
   // trusted by the keystore pass authorization.
   //

Modified: activemq/activemq-apollo/trunk/apollo-cli/src/main/scala/org/apache/activemq/apollo/cli/commands/Create.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-cli/src/main/scala/org/apache/activemq/apollo/cli/commands/Create.scala?rev=1133464&r1=1133463&r2=1133464&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-cli/src/main/scala/org/apache/activemq/apollo/cli/commands/Create.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-cli/src/main/scala/org/apache/activemq/apollo/cli/commands/Create.scala Wed Jun  8 16:46:36 2011
@@ -77,6 +77,9 @@ class Create extends Action {
       target = etc / "login.config"
       write("etc/login.config", target)
 
+      target = etc / "black-list.txt"
+      write("etc/black-list.txt", target)
+
       // Generate a keystore with a new key
       println("Generating ssl keystore...")
       val ssl = system(etc, Array(

Modified: activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ConnectionStatusDTO.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ConnectionStatusDTO.java?rev=1133464&r1=1133463&r2=1133464&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ConnectionStatusDTO.java (original)
+++ activemq/activemq-apollo/trunk/apollo-dto/src/main/java/org/apache/activemq/apollo/dto/ConnectionStatusDTO.java Wed Jun  8 16:46:36 2011
@@ -75,4 +75,9 @@ public class ConnectionStatusDTO extends
 	@XmlAttribute(name="remote_address")
 	public String remote_address;
 
+    /**
+     * The local address of the connection
+     */
+	@XmlAttribute(name="local_address")
+	public String local_address;
 }

Modified: activemq/activemq-apollo/trunk/apollo-tcp/src/main/java/org/apache/activemq/apollo/transport/tcp/TcpTransport.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-tcp/src/main/java/org/apache/activemq/apollo/transport/tcp/TcpTransport.java?rev=1133464&r1=1133463&r2=1133464&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-tcp/src/main/java/org/apache/activemq/apollo/transport/tcp/TcpTransport.java (original)
+++ activemq/activemq-apollo/trunk/apollo-tcp/src/main/java/org/apache/activemq/apollo/transport/tcp/TcpTransport.java Wed Jun  8 16:46:36 2011
@@ -77,8 +77,8 @@ public class TcpTransport extends JavaBa
     class CONNECTED extends SocketState {
 
         public CONNECTED() {
-            localAddress = channel.socket().getLocalSocketAddress().toString();
-            remoteAddress = channel.socket().getRemoteSocketAddress().toString();
+            localAddress = channel.socket().getLocalSocketAddress();
+            remoteAddress = channel.socket().getRemoteSocketAddress();
         }
 
         void onStop(Runnable onCompleted) {
@@ -195,8 +195,8 @@ public class TcpTransport extends JavaBa
     int traffic_class = IPTOS_THROUGHPUT;
 
     protected RateLimitingChannel rateLimitingChannel;
-    String localAddress;
-    String remoteAddress;
+    SocketAddress localAddress;
+    SocketAddress remoteAddress;
 
     class RateLimitingChannel implements ReadableByteChannel, WritableByteChannel {
 
@@ -606,11 +606,11 @@ public class TcpTransport extends JavaBa
         }
     }
 
-    public String getLocalAddress() {
+    public SocketAddress getLocalAddress() {
         return localAddress;
     }
 
-    public String getRemoteAddress() {
+    public SocketAddress getRemoteAddress() {
         return remoteAddress;
     }
 

Modified: activemq/activemq-apollo/trunk/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/Transport.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/Transport.java?rev=1133464&r1=1133463&r2=1133464&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/Transport.java (original)
+++ activemq/activemq-apollo/trunk/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/Transport.java Wed Jun  8 16:46:36 2011
@@ -17,6 +17,7 @@
 package org.apache.activemq.apollo.transport;
 
 import java.io.IOException;
+import java.net.SocketAddress;
 import java.net.URI;
 
 import org.apache.activemq.apollo.util.Service;
@@ -87,12 +88,12 @@ public interface Transport extends Servi
     /**
      * @return the remote address for this connection
      */
-    String getRemoteAddress();
+    SocketAddress getRemoteAddress();
 
     /**
      * @return the remote address for this connection
      */
-    String getLocalAddress();
+    SocketAddress getLocalAddress();
 
     /**
      * Indicates if the transport can handle faults

Modified: activemq/activemq-apollo/trunk/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/TransportFilter.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/TransportFilter.java?rev=1133464&r1=1133463&r2=1133464&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/TransportFilter.java (original)
+++ activemq/activemq-apollo/trunk/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/TransportFilter.java Wed Jun  8 16:46:36 2011
@@ -19,6 +19,7 @@ package org.apache.activemq.apollo.trans
 import org.fusesource.hawtdispatch.DispatchQueue;
 
 import java.io.IOException;
+import java.net.SocketAddress;
 import java.net.URI;
 
 /**
@@ -143,11 +144,11 @@ public class TransportFilter implements 
         return next.narrow(target);
     }
 
-    public String getRemoteAddress() {
+    public SocketAddress getRemoteAddress() {
         return next.getRemoteAddress();
     }
 
-    public String getLocalAddress() {
+    public SocketAddress getLocalAddress() {
         return next.getLocalAddress();
     }
 

Modified: activemq/activemq-apollo/trunk/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/pipe/PipeTransport.java
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/pipe/PipeTransport.java?rev=1133464&r1=1133463&r2=1133464&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/pipe/PipeTransport.java (original)
+++ activemq/activemq-apollo/trunk/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/pipe/PipeTransport.java Wed Jun  8 16:46:36 2011
@@ -23,6 +23,7 @@ import org.fusesource.hawtdispatch.*;
 
 import java.io.EOFException;
 import java.io.IOException;
+import java.net.SocketAddress;
 import java.net.URI;
 import java.util.LinkedList;
 import java.util.concurrent.atomic.AtomicBoolean;
@@ -37,7 +38,7 @@ public class PipeTransport implements Tr
     final private PipeTransportServer server;
     PipeTransport peer;
     private TransportListener listener;
-    private String remoteAddress;
+    private SocketAddress remoteAddress;
     private AtomicBoolean stopping = new AtomicBoolean();
     private String name;
     private boolean marshal;
@@ -72,16 +73,16 @@ public class PipeTransport implements Tr
         server.dispatchQueue.execute(new Runnable(){
             public void run() {
                 dispatchSource = Dispatch.createSource(EventAggregators.linkedList(), dispatchQueue);
-                dispatchSource.setEventHandler(new Runnable(){
+                dispatchSource.setEventHandler(new Runnable() {
                     public void run() {
                         try {
                             final LinkedList<Object> commands = dispatchSource.getData();
                             for (Object o : commands) {
 
-                                if(o == EOF_TOKEN) {
+                                if (o == EOF_TOKEN) {
                                     throw new EOFException();
                                 }
-                                readCounter ++;
+                                readCounter++;
                                 listener.onTransportCommand(o);
                             }
 
@@ -190,11 +191,11 @@ public class PipeTransport implements Tr
         return readCounter;
     }
 
-    public String getLocalAddress() {
+    public SocketAddress getLocalAddress() {
         return remoteAddress;
     }
 
-    public String getRemoteAddress() {
+    public SocketAddress getRemoteAddress() {
         return remoteAddress;
     }
 
@@ -220,8 +221,13 @@ public class PipeTransport implements Tr
         return "pipe";
     }
 
-    public void setRemoteAddress(String remoteAddress) {
-        this.remoteAddress = remoteAddress;
+    public void setRemoteAddress(final String remoteAddress) {
+        this.remoteAddress = new SocketAddress() {
+            @Override
+            public String toString() {
+                return remoteAddress;
+            }
+        };
         if (name == null) {
             name = remoteAddress;
         }

Modified: activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/resources/BrokerResource.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/resources/BrokerResource.scala?rev=1133464&r1=1133463&r2=1133464&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/resources/BrokerResource.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/resources/BrokerResource.scala Wed Jun  8 16:46:36 2011
@@ -77,7 +77,7 @@ case class BrokerResource() extends Reso
 
         broker.connections.foreach { case (id,connection) =>
           // TODO: may need to sync /w connection's dispatch queue
-          result.connections.add( new LongIdLabeledDTO(id, connection.transport.getRemoteAddress ) )
+          result.connections.add( new LongIdLabeledDTO(id, connection.transport.getRemoteAddress.toString ) )
         }
         result
 
@@ -215,7 +215,7 @@ case class BrokerResource() extends Reso
     val link = new LinkDTO()
     link.kind = "connection"
     link.id = connection.id.toString
-    link.label = connection.transport.getRemoteAddress
+    link.label = connection.transport.getRemoteAddress.toString
     link
   }
 

Modified: activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/resources/Support.scala
URL: http://svn.apache.org/viewvc/activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/resources/Support.scala?rev=1133464&r1=1133463&r2=1133464&view=diff
==============================================================================
--- activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/resources/Support.scala (original)
+++ activemq/activemq-apollo/trunk/apollo-web/src/main/scala/org/apache/activemq/apollo/web/resources/Support.scala Wed Jun  8 16:46:36 2011
@@ -30,7 +30,6 @@ import java.util.concurrent.TimeUnit
 import org.apache.activemq.apollo.dto._
 import java.util.{Arrays, Collections}
 import org.fusesource.hawtdispatch._
-import java.net.URI
 import org.fusesource.scalate.{NoValueSetException, RenderContext}
 import com.sun.jersey.core.util.Base64
 import javax.servlet.http.{HttpServletResponse, HttpServletRequest}
@@ -44,6 +43,7 @@ import org.apache.activemq.apollo.util.S
 import org.apache.activemq.apollo.util.Failure._
 import org.apache.activemq.apollo.util._
 import javax.management.remote.rmi._RMIConnection_Stub
+import java.net.{InetSocketAddress, URI}
 
 object Resource {
 
@@ -175,8 +175,8 @@ abstract class Resource(parent:Resource=
       func(security_context)
     } else {
       security_context = new SecurityContext
-      security_context.local_address = http_request.getLocalAddr+":"+http_request.getLocalPort
-      security_context.remote_address = http_request.getRemoteAddr+":"+http_request.getRemotePort
+      security_context.local_address = new InetSocketAddress(http_request.getLocalAddr, http_request.getLocalPort)
+      security_context.remote_address = new InetSocketAddress(http_request.getRemoteAddr, http_request.getRemotePort)
 
       var auth_header = http_request.getHeader(HEADER_AUTHORIZATION)
       if (auth_header != null && auth_header.length > 0) {