You are viewing a plain text version of this content. The canonical link for it is here.
Posted to scm@geronimo.apache.org by jg...@apache.org on 2006/09/17 14:35:47 UTC

svn commit: r447058 [4/4] - in /geronimo/sandbox/gcache/openwire: ./ src/main/java/org/apache/geronimo/openwire/command/ src/main/java/org/apache/geronimo/openwire/state/ src/main/java/org/apache/geronimo/openwire/thread/ src/main/java/org/apache/geron...

Added: geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/FailoverTransport.java
URL: http://svn.apache.org/viewvc/geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/FailoverTransport.java?view=auto&rev=447058
==============================================================================
--- geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/FailoverTransport.java (added)
+++ geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/FailoverTransport.java Sun Sep 17 05:35:45 2006
@@ -0,0 +1,547 @@
+/**
+ *
+ * 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.geronimo.openwire.transport.failover;
+
+import java.io.IOException;
+import java.io.InterruptedIOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Random;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.geronimo.openwire.command.Command;
+import org.apache.geronimo.openwire.command.NodeInfo;
+import org.apache.geronimo.openwire.command.Response;
+import org.apache.geronimo.openwire.state.ConnectionStateTracker;
+import org.apache.geronimo.openwire.state.Tracked;
+import org.apache.geronimo.openwire.thread.DefaultThreadPools;
+import org.apache.geronimo.openwire.thread.Task;
+import org.apache.geronimo.openwire.thread.TaskRunner;
+import org.apache.geronimo.openwire.transport.CompositeTransport;
+import org.apache.geronimo.openwire.transport.FutureResponse;
+import org.apache.geronimo.openwire.transport.ResponseCallback;
+import org.apache.geronimo.openwire.transport.Transport;
+import org.apache.geronimo.openwire.transport.TransportFactory;
+import org.apache.geronimo.openwire.transport.TransportListener;
+import org.apache.geronimo.openwire.util.IOExceptionSupport;
+import org.apache.geronimo.openwire.util.ServiceSupport;
+
+import edu.emory.mathcs.backport.java.util.concurrent.ConcurrentHashMap;
+import edu.emory.mathcs.backport.java.util.concurrent.CopyOnWriteArrayList;
+
+/**
+ * A Transport that is made reliable by being able to fail over to another
+ * transport when a transport failure is detected.
+ * 
+ * @version $Revision$
+ */
+public class FailoverTransport implements CompositeTransport {
+
+    private static final Log log = LogFactory.getLog(FailoverTransport.class);
+
+    private TransportListener transportListener;
+    private boolean disposed;
+    private final CopyOnWriteArrayList uris = new CopyOnWriteArrayList();
+
+    private final Object reconnectMutex = new Object();
+    private final Object sleepMutex = new Object();
+    private final ConnectionStateTracker stateTracker = new ConnectionStateTracker();
+    private final ConcurrentHashMap requestMap = new ConcurrentHashMap();
+
+    private URI connectedTransportURI;
+    private Transport connectedTransport;
+    private final TaskRunner reconnectTask;
+    private boolean started;
+
+    private long initialReconnectDelay = 10;
+    private long maxReconnectDelay = 1000 * 30;
+    private long backOffMultiplier = 2;
+    private boolean useExponentialBackOff = true;
+    private boolean randomize = true;
+    private boolean initialized;
+    private int maxReconnectAttempts;
+    private int connectFailures;
+    private long reconnectDelay = initialReconnectDelay;
+    private Exception connectionFailure;
+
+    private final TransportListener myTransportListener = createTransportListener();
+    
+    TransportListener createTransportListener() {
+    	return new TransportListener() {
+	        public void onCommand(Command command) {
+	            if (command == null) {
+	                return;
+	            }
+	            if (command.isResponse()) {
+                    Object object = requestMap.remove(new Integer(((Response) command).getCorrelationId()));
+                    if( object!=null && object.getClass() == Tracked.class ) {
+                	   ((Tracked)object).onResponses();
+                    }
+	            }
+	            if (!initialized){
+	                if (command.isNodeInfo()){
+	                    NodeInfo info = (NodeInfo)command;
+                        NodeInfo[] peers = info.getPeerNodeInfos();
+	                    if (peers!= null){
+	                        for (int i =0; i < peers.length;i++){
+	                            String nodeString = peers[i].getNodeURL();
+	                            add(nodeString);
+	                        }
+	                    }
+	                initialized = true;
+	                }
+	                
+	            }
+	            if (transportListener != null) {
+	                transportListener.onCommand(command);
+	            }
+	        }
+	
+	        public void onException(IOException error) {
+	            try {
+	                handleTransportFailure(error);
+	            }
+	            catch (InterruptedException e) {
+	                Thread.currentThread().interrupt();
+	                transportListener.onException(new InterruptedIOException());
+	            }
+	        }
+	        
+	        public void transportInterupted(){
+	            if (transportListener != null){
+	                transportListener.transportInterupted();
+	            }
+	        }
+	
+	        public void transportResumed(){
+	            if(transportListener != null){
+	                transportListener.transportResumed();
+	            }
+	        }
+	    };
+    }
+    
+    public FailoverTransport() throws InterruptedIOException {
+
+    	stateTracker.setTrackTransactions(true);
+    	
+        // Setup a task that is used to reconnect the a connection async.
+        reconnectTask = DefaultThreadPools.getDefaultTaskRunnerFactory().createTaskRunner(new Task() {
+
+            public boolean iterate() {
+
+                Exception failure=null;
+                synchronized (reconnectMutex) {
+
+                    if (disposed || connectionFailure!=null) {
+                        reconnectMutex.notifyAll();
+                    }
+
+                    if (connectedTransport != null || disposed || connectionFailure!=null) {
+                        return false;
+                    } else {
+                        ArrayList connectList = getConnectList();
+                        if( connectList.isEmpty() ) {
+                            failure = new IOException("No uris available to connect to.");
+                        } else {
+                            if (!useExponentialBackOff){
+                                reconnectDelay = initialReconnectDelay;
+                            }
+                            Iterator iter = connectList.iterator();
+                            for (int i = 0; iter.hasNext() && connectedTransport == null && !disposed; i++) {
+                                URI uri = (URI) iter.next();
+                                try {
+                                    log.debug("Attempting connect to: " + uri);
+                                    Transport t = TransportFactory.compositeConnect(uri);
+                                    t.setTransportListener(myTransportListener);
+                                    t.start();
+                                    
+                                    if (started) {
+                                        restoreTransport(t);
+                                    }
+                                    
+                                    log.debug("Connection established");
+                                    reconnectDelay = initialReconnectDelay;
+                                    connectedTransportURI = uri;
+                                    connectedTransport = t;
+                                    reconnectMutex.notifyAll();
+                                    connectFailures = 0;
+                                    if (transportListener != null){
+                                        transportListener.transportResumed();
+                                    }
+                                   
+                                    return false;
+                                }
+                                catch (Exception e) {
+                                    failure = e;
+                                    log.debug("Connect fail to: " + uri + ", reason: " + e);
+                                }
+                            }
+                        }
+                    }
+                    
+                    if (maxReconnectAttempts > 0 && ++connectFailures >= maxReconnectAttempts) {
+                        log.error("Failed to connect to transport after: " + connectFailures + " attempt(s)");
+                        connectionFailure = failure;
+                        reconnectMutex.notifyAll();
+                        return false;
+                    }
+                }
+
+                if(!disposed){
+                    
+                        log.debug("Waiting "+reconnectDelay+" ms before attempting connection. ");
+                        synchronized(sleepMutex){
+                            try{
+                                sleepMutex.wait(reconnectDelay);
+                            }catch(InterruptedException e){
+                               Thread.currentThread().interrupt();
+                            }
+                        }
+                        
+                    
+                    if(useExponentialBackOff){
+                        // Exponential increment of reconnect delay.
+                        reconnectDelay*=backOffMultiplier;
+                        if(reconnectDelay>maxReconnectDelay)
+                            reconnectDelay=maxReconnectDelay;
+                    }
+                }
+                return !disposed;
+            }
+
+        }, "ActiveMQ Failover Worker: "+System.identityHashCode(this));
+    }
+
+    private void handleTransportFailure(IOException e) throws InterruptedException {
+        if (transportListener != null){
+            transportListener.transportInterupted();
+        }
+        synchronized (reconnectMutex) {
+            log.info("Transport failed, attempting to automatically reconnect due to: " + e, e);
+            if (connectedTransport != null) {
+                initialized = false;
+                ServiceSupport.dispose(connectedTransport);
+                connectedTransport = null;
+                connectedTransportURI = null;
+            }
+            reconnectTask.wakeup();
+        }
+    }
+
+    public void start() throws Exception {
+        synchronized (reconnectMutex) {
+            log.debug("Started.");
+            if (started)
+                return;
+            started = true;
+            if (connectedTransport != null) {
+                stateTracker.restore(connectedTransport);
+            }
+        }
+    }
+
+    public void stop() throws Exception {
+        synchronized (reconnectMutex) {
+            log.debug("Stopped.");
+            if (!started)
+                return;
+            started = false;
+            disposed = true;
+
+            if (connectedTransport != null) {
+                connectedTransport.stop();
+                connectedTransport=null;
+            }
+            reconnectMutex.notifyAll();
+        }
+        synchronized(sleepMutex){
+            sleepMutex.notifyAll();
+        }
+        reconnectTask.shutdown();
+    }
+
+    public long getInitialReconnectDelay() {
+        return initialReconnectDelay;
+    }
+
+    public void setInitialReconnectDelay(long initialReconnectDelay) {
+        this.initialReconnectDelay = initialReconnectDelay;
+    }
+
+    public long getMaxReconnectDelay() {
+        return maxReconnectDelay;
+    }
+
+    public void setMaxReconnectDelay(long maxReconnectDelay) {
+        this.maxReconnectDelay = maxReconnectDelay;
+    }
+
+    public long getReconnectDelay() {
+        return reconnectDelay;
+    }
+
+    public void setReconnectDelay(long reconnectDelay) {
+        this.reconnectDelay = reconnectDelay;
+    }
+
+    public long getReconnectDelayExponent() {
+        return backOffMultiplier;
+    }
+
+    public void setReconnectDelayExponent(long reconnectDelayExponent) {
+        this.backOffMultiplier = reconnectDelayExponent;
+    }
+
+    public Transport getConnectedTransport() {
+        return connectedTransport;
+    }
+
+    public URI getConnectedTransportURI() {
+        return connectedTransportURI;
+    }
+
+    public int getMaxReconnectAttempts() {
+        return maxReconnectAttempts;
+    }
+
+    public void setMaxReconnectAttempts(int maxReconnectAttempts) {
+        this.maxReconnectAttempts = maxReconnectAttempts;
+    }
+
+    /**
+     * @return Returns the randomize.
+     */
+    public boolean isRandomize(){
+        return randomize;
+    }
+
+    /**
+     * @param randomize The randomize to set.
+     */
+    public void setRandomize(boolean randomize){
+        this.randomize=randomize;
+    }
+
+    public void oneway(Command command) throws IOException {
+        Exception error = null;
+        try {
+
+            synchronized (reconnectMutex) {
+                // Keep trying until the message is sent.
+                for (int i = 0;!disposed; i++) {
+                    try {
+
+                        // Wait for transport to be connected.
+                        while (connectedTransport == null && !disposed && connectionFailure==null ) {
+                            log.trace("Waiting for transport to reconnect.");
+                            try {
+                                reconnectMutex.wait(1000);
+                            }
+                            catch (InterruptedException e) {
+                                Thread.currentThread().interrupt();
+                                log.debug("Interupted: " + e, e);
+                            }
+                        }
+
+                        if( connectedTransport==null ) {
+                            // Previous loop may have exited due to use being
+                            // disposed.
+                            if (disposed) {
+                                error = new IOException("Transport disposed.");
+                            } else if (connectionFailure!=null) {
+                                error = connectionFailure;
+                            } else {
+                                error = new IOException("Unexpected failure.");
+                            }
+                            break;
+                        }
+
+                        // If it was a request and it was not being tracked by
+                        // the state tracker,
+                        // then hold it in the requestMap so that we can replay
+                        // it later.
+                        Tracked tracked = stateTracker.track(command);
+                        if( tracked!=null && tracked.isWaitingForResponse() ) {
+                            requestMap.put(new Integer(command.getCommandId()), tracked);
+                        } else if ( tracked==null && command.isResponseRequired()) {
+                            requestMap.put(new Integer(command.getCommandId()), command);
+                        }
+                                                
+                        // Send the message.
+                        try {
+                            connectedTransport.oneway(command);
+                        } catch (IOException e) {
+                        	
+                        	// If the command was not tracked.. we will retry in this method
+                        	if( tracked==null ) {
+                        		
+                        		// since we will retry in this method.. take it out of the request
+                        		// map so that it is not sent 2 times on recovery
+                            	if( command.isResponseRequired() ) {
+                            		requestMap.remove(new Integer(command.getCommandId()));
+                            	}
+                            	
+                                // Rethrow the exception so it will handled by the outer catch
+                                throw e;
+                        	}
+                        	
+                        }
+                        
+                        return;
+
+                    }
+                    catch (IOException e) {
+                        log.debug("Send oneway attempt: " + i + " failed.");
+                        handleTransportFailure(e);
+                    }
+                }
+            }
+        }
+        catch (InterruptedException e) {
+            // Some one may be trying to stop our thread.
+            Thread.currentThread().interrupt();
+            throw new InterruptedIOException();
+        }
+        if(!disposed){
+            if(error!=null){
+                if(error instanceof IOException)
+                    throw (IOException) error;
+                throw IOExceptionSupport.create(error);
+            }
+        }
+    }
+
+    public FutureResponse asyncRequest(Command command, ResponseCallback responseCallback) throws IOException {
+        throw new AssertionError("Unsupported Method");
+    }
+
+    public Response request(Command command) throws IOException {
+        throw new AssertionError("Unsupported Method");
+    }
+    
+    public Response request(Command command,int timeout) throws IOException {
+        throw new AssertionError("Unsupported Method");
+    }
+
+    public void add(URI u[]) {
+        for (int i = 0; i < u.length; i++) {
+            if( !uris.contains(u[i]) )
+                uris.add(u[i]);
+        }
+        reconnect();
+    }
+
+    public void remove(URI u[]) {
+        for (int i = 0; i < u.length; i++) {
+            uris.remove(u[i]);
+        }
+        reconnect();
+    }
+    
+    public void add(String u){
+        try {
+        URI uri = new URI(u);
+        if (!uris.contains(uri))
+            uris.add(uri);
+
+        reconnect();
+        }catch(Exception e){
+            log.error("Failed to parse URI: " + u);
+        }
+    }
+
+
+    public void reconnect() {
+        log.debug("Waking up reconnect task");
+        try {
+            reconnectTask.wakeup();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    private ArrayList getConnectList(){
+        ArrayList l=new ArrayList(uris);
+        if (randomize){
+            // Randomly, reorder the list by random swapping
+            Random r=new Random();
+            r.setSeed(System.currentTimeMillis());
+            for (int i=0;i<l.size();i++){
+                int p=r.nextInt(l.size());
+                Object t=l.get(p);
+                l.set(p,l.get(i));
+                l.set(i,t);
+            }
+        }
+        return l;
+    }
+
+    public TransportListener getTransportListener() {
+        return transportListener;
+    }
+
+    public void setTransportListener(TransportListener commandListener) {
+        this.transportListener = commandListener;
+    }
+
+    public Object narrow(Class target) {
+
+        if (target.isAssignableFrom(getClass())) {
+            return this;
+        }
+        synchronized (reconnectMutex) {
+            if (connectedTransport != null) {
+                return connectedTransport.narrow(target);
+            }
+        }
+        return null;
+
+    }
+
+    protected void restoreTransport(Transport t) throws Exception, IOException {
+        t.start();
+        stateTracker.restore(t);
+        for (Iterator iter2 = requestMap.values().iterator(); iter2.hasNext();) {
+            Command command = (Command) iter2.next();
+            t.oneway(command);
+        }
+    }
+
+    public boolean isUseExponentialBackOff() {
+        return useExponentialBackOff;
+    }
+
+    public void setUseExponentialBackOff(boolean useExponentialBackOff) {
+        this.useExponentialBackOff = useExponentialBackOff;
+    }
+
+    public String toString() {
+        return connectedTransportURI==null ? "unconnected" : connectedTransportURI.toString();
+    }
+
+	public String getRemoteAddress() {
+		if(connectedTransport != null){
+			return connectedTransport.getRemoteAddress();
+		}
+		return null;
+	}
+
+}

Propchange: geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/FailoverTransport.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/FailoverTransport.java
------------------------------------------------------------------------------
    svn:keywords = Date Revision

Propchange: geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/FailoverTransport.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/FailoverTransportFactory.java
URL: http://svn.apache.org/viewvc/geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/FailoverTransportFactory.java?view=auto&rev=447058
==============================================================================
--- geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/FailoverTransportFactory.java (added)
+++ geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/FailoverTransportFactory.java Sun Sep 17 05:35:45 2006
@@ -0,0 +1,76 @@
+/**
+ *
+ * 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.geronimo.openwire.transport.failover;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.Map;
+
+import org.apache.geronimo.openwire.transport.MutexTransport;
+import org.apache.geronimo.openwire.transport.ResponseCorrelator;
+import org.apache.geronimo.openwire.transport.Transport;
+import org.apache.geronimo.openwire.transport.TransportFactory;
+import org.apache.geronimo.openwire.transport.TransportServer;
+import org.apache.geronimo.openwire.util.IntrospectionSupport;
+import org.apache.geronimo.openwire.util.URISupport;
+import org.apache.geronimo.openwire.util.URISupport.CompositeData;
+
+public class FailoverTransportFactory extends TransportFactory {
+
+    public Transport doConnect(URI location) throws IOException {
+        try {
+            Transport transport = createTransport(URISupport.parseComposite(location));
+            transport =  new MutexTransport(transport);
+            transport = new ResponseCorrelator(transport);
+            return transport;
+        } catch (URISyntaxException e) {
+            throw new IOException("Invalid location: "+location);
+        }
+    }
+    
+    public Transport doCompositeConnect(URI location) throws IOException {
+        try {
+            return createTransport(URISupport.parseComposite(location));
+        } catch (URISyntaxException e) {
+            throw new IOException("Invalid location: "+location);
+        }
+    }
+
+    /**
+     * @param location
+     * @return 
+     * @throws IOException
+     */
+    public Transport createTransport(CompositeData compositData) throws IOException {
+        FailoverTransport transport = createTransport(compositData.getParameters());
+        transport.add(compositData.getComponents());
+        return transport;
+    }
+
+    public FailoverTransport createTransport(Map parameters) throws IOException {
+        FailoverTransport transport = new FailoverTransport();
+        IntrospectionSupport.setProperties(transport, parameters);
+        return transport;
+    }
+
+    public TransportServer doBind(String brokerId,URI location) throws IOException {
+        throw new IOException("Invalid server URI: "+location);
+    }
+
+}

Propchange: geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/FailoverTransportFactory.java
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/FailoverTransportFactory.java
------------------------------------------------------------------------------
    svn:keywords = Date Revision

Propchange: geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/FailoverTransportFactory.java
------------------------------------------------------------------------------
    svn:mime-type = text/plain

Added: geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/package.html
URL: http://svn.apache.org/viewvc/geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/package.html?view=auto&rev=447058
==============================================================================
--- geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/package.html (added)
+++ geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/package.html Sun Sep 17 05:35:45 2006
@@ -0,0 +1,25 @@
+<!--
+    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.
+-->
+<html>
+<head>
+</head>
+<body>
+
+Fail-Over Transport which will automatically reconnect to a failed transport and choose one of a list of possible transport implementations to use.
+
+</body>
+</html>

Propchange: geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/package.html
------------------------------------------------------------------------------
    svn:eol-style = native

Propchange: geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/package.html
------------------------------------------------------------------------------
    svn:keywords = Date Revision

Propchange: geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/failover/package.html
------------------------------------------------------------------------------
    svn:mime-type = text/html

Modified: geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/tcp/TcpTransportServer.java
URL: http://svn.apache.org/viewvc/geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/tcp/TcpTransportServer.java?view=diff&rev=447058&r1=447057&r2=447058
==============================================================================
--- geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/tcp/TcpTransportServer.java (original)
+++ geronimo/sandbox/gcache/openwire/src/main/java/org/apache/geronimo/openwire/transport/tcp/TcpTransportServer.java Sun Sep 17 05:35:45 2006
@@ -44,12 +44,12 @@
 
 /**
  * A TCP based implementation of {@link org.apache.geronimo.openwire.transport.TransportServer}
- * 
+ *
  * @version $Revision$
  */
 
 public class TcpTransportServer extends TransportServerThreadSupport {
-	
+
     private static final Log log = LogFactory.getLog(TcpTransportServer.class);
     protected ServerSocket serverSocket;
     protected int backlog = 5000;
@@ -60,7 +60,7 @@
     protected boolean trace;
     protected Map transportOptions;
     protected final ServerSocketFactory serverSocketFactory;
-    
+
     public TcpTransportServer(TcpTransportFactory transportFactory, URI location, ServerSocketFactory serverSocketFactory) throws IOException, URISyntaxException {
         super(location);
         this.transportFactory=transportFactory;
@@ -69,7 +69,7 @@
 
     public void bind() throws IOException {
     	URI bind = getBindLocation();
-    	
+
         String host = bind.getHost();
         host = (host == null || host.length() == 0) ? "localhost" : host;
         InetAddress addr = InetAddress.getByName(host);
@@ -93,7 +93,7 @@
 			throw IOExceptionSupport.create(e);
 		}
     }
-    
+
     /**
      * @return Returns the wireFormatFactory.
      */
@@ -112,8 +112,8 @@
     /**
      * Associates a broker info with the transport server so that the transport
      * can do discovery advertisements of the broker.
-     * 
-     * @param brokerInfo
+     *
+     * @param nodeInfo
      */
     public void setNodeInfo(NodeInfo nodeInfo) {
     }