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 2010/07/18 16:55:04 UTC

svn commit: r965240 - in /activemq/sandbox/activemq-apollo-actor: apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/protocol/ apollo-stomp/src/main/resources/META-INF/services/org.apache.activemq.apollo/ apollo-stomp/src/main/scala/org/apa...

Author: chirino
Date: Sun Jul 18 14:55:03 2010
New Revision: 965240

URL: http://svn.apache.org/viewvc?rev=965240&view=rev
Log:
Did a all java version of the ClassFinder.. the protocol codec factory is now using it.

Added:
    activemq/sandbox/activemq-apollo-actor/apollo-stomp/src/main/resources/META-INF/services/org.apache.activemq.apollo/protocol-codec-factory.index
    activemq/sandbox/activemq-apollo-actor/apollo-util/src/main/scala/org/apache/activemq/apollo/util/JavaClassFinder.java
Modified:
    activemq/sandbox/activemq-apollo-actor/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/protocol/MultiProtocol.scala
    activemq/sandbox/activemq-apollo-actor/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/protocol/ProtocolFactory.scala
    activemq/sandbox/activemq-apollo-actor/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/StompProtocol.scala
    activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/ObjectStreamProtocolCodecFactory.java
    activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/ProtocolCodecFactory.java
    activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/TransportFactory.java
    activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/TransportFactorySupport.java
    activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/pipe/PipeTransportServer.java

Modified: activemq/sandbox/activemq-apollo-actor/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/protocol/MultiProtocol.scala
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/protocol/MultiProtocol.scala?rev=965240&r1=965239&r2=965240&view=diff
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/protocol/MultiProtocol.scala (original)
+++ activemq/sandbox/activemq-apollo-actor/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/protocol/MultiProtocol.scala Sun Jul 18 14:55:03 2010
@@ -65,6 +65,8 @@ class MultiProtocol(val func: ()=>Array[
 
   lazy val protocols: Array[Protocol] = func()
 
+  def protocol = "multi"
+
   def createProtocolCodec = new MultiProtocolCodec(protocols)
 
   def createProtocolHandler = new MultiProtocolHandler

Modified: activemq/sandbox/activemq-apollo-actor/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/protocol/ProtocolFactory.scala
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/protocol/ProtocolFactory.scala?rev=965240&r1=965239&r2=965240&view=diff
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/protocol/ProtocolFactory.scala (original)
+++ activemq/sandbox/activemq-apollo-actor/apollo-broker/src/main/scala/org/apache/activemq/apollo/broker/protocol/ProtocolFactory.scala Sun Jul 18 14:55:03 2010
@@ -22,6 +22,7 @@ import org.apache.activemq.apollo.util.C
 import org.apache.activemq.apollo.store.MessageRecord
 import org.apache.activemq.apollo.transport._
 import org.apache.activemq.apollo.broker.{Delivery, Message, BrokerConnection}
+import com.sun.corba.se.pept.protocol.ProtocolHandler
 
 /**
  * <p>
@@ -54,7 +55,7 @@ object ProtocolFactory {
   }
 }
 
-trait Protocol extends ProtocolCodecFactory {
+trait Protocol extends ProtocolCodecFactory.Provider {
 
   def createProtocolHandler:ProtocolHandler
   def encode(message:Message):MessageRecord

Added: activemq/sandbox/activemq-apollo-actor/apollo-stomp/src/main/resources/META-INF/services/org.apache.activemq.apollo/protocol-codec-factory.index
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/apollo-stomp/src/main/resources/META-INF/services/org.apache.activemq.apollo/protocol-codec-factory.index?rev=965240&view=auto
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/apollo-stomp/src/main/resources/META-INF/services/org.apache.activemq.apollo/protocol-codec-factory.index (added)
+++ activemq/sandbox/activemq-apollo-actor/apollo-stomp/src/main/resources/META-INF/services/org.apache.activemq.apollo/protocol-codec-factory.index Sun Jul 18 14:55:03 2010
@@ -0,0 +1,17 @@
+## ---------------------------------------------------------------------------
+## 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.
+## ---------------------------------------------------------------------------
+org.apache.activemq.apollo.stomp.StompProtocolCodecFactory
\ No newline at end of file

Modified: activemq/sandbox/activemq-apollo-actor/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/StompProtocol.scala
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/StompProtocol.scala?rev=965240&r1=965239&r2=965240&view=diff
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/StompProtocol.scala (original)
+++ activemq/sandbox/activemq-apollo-actor/apollo-stomp/src/main/scala/org/apache/activemq/apollo/stomp/StompProtocol.scala Sun Jul 18 14:55:03 2010
@@ -66,10 +66,12 @@ object StompConstants {
  *
  * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
  */
-class StompProtocolCodecFactory extends ProtocolCodecFactory {
+class StompProtocolCodecFactory extends ProtocolCodecFactory.Provider {
   import Stomp.Commands.CONNECT
   import Stomp.Commands.STOMP
 
+  def protocol = StompConstants.PROTOCOL
+
   def createProtocolCodec() = new StompCodec();
 
   def isIdentifiable() = true

Modified: activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/ObjectStreamProtocolCodecFactory.java
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/ObjectStreamProtocolCodecFactory.java?rev=965240&r1=965239&r2=965240&view=diff
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/ObjectStreamProtocolCodecFactory.java (original)
+++ activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/ObjectStreamProtocolCodecFactory.java Sun Jul 18 14:55:03 2010
@@ -19,9 +19,13 @@ package org.apache.activemq.apollo.trans
 
 import org.fusesource.hawtbuf.Buffer;
 
-public class ObjectStreamProtocolCodecFactory implements ProtocolCodecFactory {
+public class ObjectStreamProtocolCodecFactory implements ProtocolCodecFactory.Provider {
 
-	public ProtocolCodec createProtocolCodec() {
+    public String protocol() {
+        return "object";
+    }
+
+    public ProtocolCodec createProtocolCodec() {
 		return new ObjectStreamProtocolCodec();
 	}	
 

Modified: activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/ProtocolCodecFactory.java
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/ProtocolCodecFactory.java?rev=965240&r1=965239&r2=965240&view=diff
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/ProtocolCodecFactory.java (original)
+++ activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/ProtocolCodecFactory.java Sun Jul 18 14:55:03 2010
@@ -16,36 +16,64 @@
  */
 package org.apache.activemq.apollo.transport;
 
+import org.apache.activemq.apollo.util.JavaClassFinder;
 import org.fusesource.hawtbuf.Buffer;
 
-public interface ProtocolCodecFactory {
-    
-    /**
-     * @return an instance of the wire format. 
-     * 
-     */
-    ProtocolCodec createProtocolCodec();
-    
-    /**
-     * @return true if this wire format factory is identifiable. An identifiable
-     * protocol will first write a easy to identify header to the stream
-     */
-    boolean isIdentifiable();
-    
-    /**
-     * @return Returns the maximum length of the header used to discriminate the wire format if it
-     * {@link #isIdentifiable()}
-     * @throws UnsupportedOperationException If {@link #isIdentifiable()} is false
-     */
-    int maxIdentificaionLength();
+import java.util.HashMap;
+import java.util.List;
+
+public class ProtocolCodecFactory {
+
+    public static interface Provider {
+
+        String protocol();
+
+        /**
+         * @return an instance of the wire format.
+         *
+         */
+        ProtocolCodec createProtocolCodec();
+
+        /**
+         * @return true if this wire format factory is identifiable. An identifiable
+         * protocol will first write a easy to identify header to the stream
+         */
+        boolean isIdentifiable();
+
+        /**
+         * @return Returns the maximum length of the header used to discriminate the wire format if it
+         * {@link #isIdentifiable()}
+         * @throws UnsupportedOperationException If {@link #isIdentifiable()} is false
+         */
+        int maxIdentificaionLength();
+
+        /**
+         * Called to test if this protocol matches the identification header.
+         *
+         * @param buffer The byte buffer representing the header data read so far.
+         * @return true if the Buffer matches the protocol format header.
+         */
+        boolean matchesIdentification(Buffer buffer);
+
+    }
+
+    static public HashMap<String, Provider> providers = new HashMap<String, Provider>();
+
+    static {
+        JavaClassFinder<Provider> finder = new JavaClassFinder<Provider>("META-INF/services/org.apache.activemq.apollo/protocol-codec-factory.index");
+        for( Provider provider: finder.new_instances() ) {
+            providers.put(provider.protocol(), provider);
+        }
+    }
 
     /**
-     * Called to test if this protocol matches the identification header.
-     * 
-     * @param buffer The byte buffer representing the header data read so far.
-     * @return true if the Buffer matches the protocol format header.
+     * Gets the provider.
      */
-    boolean matchesIdentification(Buffer buffer);
+    public static ProtocolCodecFactory.Provider get(String name) {
+        return providers.get(name);
+    }
+
 
-    
 }
+
+

Modified: activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/TransportFactory.java
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/TransportFactory.java?rev=965240&r1=965239&r2=965240&view=diff
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/TransportFactory.java (original)
+++ activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/TransportFactory.java Sun Jul 18 14:55:03 2010
@@ -16,9 +16,9 @@
  */
 package org.apache.activemq.apollo.transport;
 
-import org.apache.activemq.apollo.util.ClassFinder;
+import org.apache.activemq.apollo.util.JavaClassFinder;
 
-import java.util.ArrayList;
+import java.util.List;
 
 /**
  *
@@ -31,19 +31,11 @@ public class TransportFactory {
         public Transport connect(String location) throws Exception;
     }
 
-    static public ArrayList<Provider> providers;
+    static public List<Provider> providers;
 
     static {
-        ClassFinder<Provider> finder = new ClassFinder<Provider>("META-INF/services/org.apache.activemq.apollo/transport-factory.index");
-        ArrayList<Provider> t = new ArrayList<Provider>();
-        for( Class<Provider> clazz: finder.findArray() ) {
-            try {
-              t.add( clazz.newInstance() );
-            } catch(Throwable e) {
-              e.printStackTrace();
-            }
-        }
-        providers = t;
+        JavaClassFinder<Provider> finder = new JavaClassFinder<Provider>("META-INF/services/org.apache.activemq.apollo/transport-factory.index");
+        providers = finder.new_instances();
     }
 
     /**

Modified: activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/TransportFactorySupport.java
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/TransportFactorySupport.java?rev=965240&r1=965239&r2=965240&view=diff
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/TransportFactorySupport.java (original)
+++ activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/TransportFactorySupport.java Sun Jul 18 14:55:03 2010
@@ -32,7 +32,6 @@ import java.util.Map;
  */
 public class  TransportFactorySupport {
 
-    private static final FactoryFinder PROTOCOL_CODEC_FACTORY_FINDER = new FactoryFinder("META-INF/services/org/apache/activemq/wireformat/");
 
     static public Transport configure(Transport transport, Map<String, String> options) throws IOException {
         ProtocolCodec wf = createProtocolCodec(options);
@@ -54,7 +53,7 @@ public class  TransportFactorySupport {
     }
 
     static public ProtocolCodec createProtocolCodec(Map<String, String> options) throws IOException {
-        ProtocolCodecFactory factory = createProtocolCodecFactory(options);
+        ProtocolCodecFactory.Provider factory = createProtocolCodecFactory(options);
         if( factory == null ) {
             return null;
         }
@@ -62,7 +61,7 @@ public class  TransportFactorySupport {
         return protocolCodec;
     }
 
-    static public ProtocolCodecFactory createProtocolCodecFactory(Map<String, String> options) throws IOException {
+    static public ProtocolCodecFactory.Provider createProtocolCodecFactory(Map<String, String> options) throws IOException {
         String protocolName = (String)options.remove("protocol");
         if (protocolName == null) {
             protocolName = getDefaultProtocolName();
@@ -70,16 +69,15 @@ public class  TransportFactorySupport {
         if( "null".equals(protocolName) ) {
             return null;
         }
-
-        try {
-            ProtocolCodecFactory wff = (ProtocolCodecFactory) PROTOCOL_CODEC_FACTORY_FINDER.newInstance(protocolName);
-            IntrospectionSupport.setProperties(wff, options, "protocol.");
-            return wff;
-        } catch (Throwable e) {
-            throw IOExceptionSupport.create("Could not create protocol codec for: " + protocolName + ", reason: " + e, e);
+        ProtocolCodecFactory.Provider provider = ProtocolCodecFactory.get(protocolName);
+        if( provider==null ) {
+            throw new IOException("Could not create protocol codec for: " + protocolName );
         }
+        
+        return provider;
     }
-    static public ProtocolCodecFactory createProtocolCodecFactory(String location) throws IOException, URISyntaxException {
+
+    static public ProtocolCodecFactory.Provider createProtocolCodecFactory(String location) throws IOException, URISyntaxException {
         URI uri = new URI(location);
         Map<String, String> options = new HashMap<String, String>(URISupport.parseParamters(uri));
 
@@ -88,13 +86,11 @@ public class  TransportFactorySupport {
             return null;
         }
 
-        try {
-            ProtocolCodecFactory wff = (ProtocolCodecFactory) PROTOCOL_CODEC_FACTORY_FINDER.newInstance(protocolName);
-            IntrospectionSupport.setProperties(wff, options);
-            return wff;
-        } catch (Throwable e) {
-            throw IOExceptionSupport.create("Could not protocol codec for: " + protocolName + ", reason: " + e, e);
+        ProtocolCodecFactory.Provider provider = ProtocolCodecFactory.get(protocolName);
+        if( provider==null ) {
+            throw new IOException("Could not create protocol codec for: " + protocolName);
         }
+        return provider;
     }
 
     static protected String getDefaultProtocolName() {

Modified: activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/pipe/PipeTransportServer.java
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/pipe/PipeTransportServer.java?rev=965240&r1=965239&r2=965240&view=diff
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/pipe/PipeTransportServer.java (original)
+++ activemq/sandbox/activemq-apollo-actor/apollo-transport/src/main/java/org/apache/activemq/apollo/transport/pipe/PipeTransportServer.java Sun Jul 18 14:55:03 2010
@@ -38,7 +38,7 @@ public class PipeTransportServer impleme
     protected URI connectURI;
     protected TransportAcceptListener listener;
     protected String name;
-    protected ProtocolCodecFactory protocolCodecFactory;
+    protected ProtocolCodecFactory.Provider protocolCodecFactory;
     protected boolean marshal;
     protected final AtomicInteger connectionCounter = new AtomicInteger();
     DispatchQueue dispatchQueue;
@@ -143,7 +143,7 @@ public class PipeTransportServer impleme
         return new PipeTransport(this);
     }
 
-    public void setProtocolCodecFactory(ProtocolCodecFactory protocolCodecFactory) {
+    public void setProtocolCodecFactory(ProtocolCodecFactory.Provider protocolCodecFactory) {
         this.protocolCodecFactory = protocolCodecFactory;
     }
 

Added: activemq/sandbox/activemq-apollo-actor/apollo-util/src/main/scala/org/apache/activemq/apollo/util/JavaClassFinder.java
URL: http://svn.apache.org/viewvc/activemq/sandbox/activemq-apollo-actor/apollo-util/src/main/scala/org/apache/activemq/apollo/util/JavaClassFinder.java?rev=965240&view=auto
==============================================================================
--- activemq/sandbox/activemq-apollo-actor/apollo-util/src/main/scala/org/apache/activemq/apollo/util/JavaClassFinder.java (added)
+++ activemq/sandbox/activemq-apollo-actor/apollo-util/src/main/scala/org/apache/activemq/apollo/util/JavaClassFinder.java Sun Jul 18 14:55:03 2010
@@ -0,0 +1,108 @@
+/**
+ * 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.util;
+
+import scala.collection.immutable.Nil;
+
+import java.io.IOException;
+import java.io.InputStream;
+import java.net.URL;
+import java.util.*;
+
+/**
+ * <p>
+ * Used to discover classes using the META-INF discovery trick.
+ * </p>
+ *
+ * @author <a href="http://hiramchirino.com">Hiram Chirino</a>
+ */
+public class JavaClassFinder<T> {
+    private final String path;
+    private final ClassLoader[] loaders;
+
+    public JavaClassFinder(String path) {
+        this(path, new ClassLoader[]{Thread.currentThread().getContextClassLoader()});
+    }
+
+    public JavaClassFinder(String path, ClassLoader[] loaders) {
+        this.path = path;
+        this.loaders = loaders;
+    }
+
+    public List<Class<T>> find() {
+
+        HashSet<Class<T>> classes = new HashSet<Class<T>>();
+        for (ClassLoader loader : loaders) {
+
+            try {
+                Enumeration<URL> resources = loader.getResources(path);
+                HashSet<String> classNames = new HashSet<String>();
+
+                while (resources.hasMoreElements()) {
+                    URL url = resources.nextElement();
+                    Properties p = loadProperties(url.openStream());
+                    Enumeration<Object> keys = p.keys();
+                    while (keys.hasMoreElements()) {
+                        classNames.add((String) keys.nextElement());
+                    }
+                }
+                
+                for (String name : classNames) {
+                    try {
+                        classes.add((Class<T>) loader.loadClass(name));
+                    } catch (ClassNotFoundException e) {
+                        e.printStackTrace();
+                    }
+                }
+
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+        }
+        return new ArrayList<Class<T>>(classes);
+    }
+
+    public List<T> new_instances() {
+        ArrayList<T> t = new ArrayList<T>();
+        for (Class<T> clazz : find()) {
+            try {
+                t.add(clazz.newInstance());
+            } catch (Throwable e) {
+                e.printStackTrace();
+            }
+        }
+        return t;
+    }
+
+    private Properties loadProperties(InputStream is) {
+        if (is == null) {
+            return null;
+        }
+        try {
+            Properties p = new Properties();
+            p.load(is);
+            return p;
+        } catch (Exception e) {
+            return null;
+        } finally {
+            try {
+                is.close();
+            } catch (Throwable e) {
+            }
+        }
+    }
+}
\ No newline at end of file