You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by js...@apache.org on 2007/11/17 11:39:55 UTC

svn commit: r595933 - in /activemq/camel/trunk: camel-core/src/main/java/org/apache/camel/model/ camel-core/src/main/java/org/apache/camel/model/loadbalancer/ camel-core/src/main/java/org/apache/camel/processor/loadbalancer/ camel-core/src/main/resourc...

Author: jstrachan
Date: Sat Nov 17 02:39:53 2007
New Revision: 595933

URL: http://svn.apache.org/viewvc?rev=595933&view=rev
Log:
applied patch for https://issues.apache.org/activemq/browse/CAMEL-229 with thanks!

Added:
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/LoadBalancerType.java   (with props)
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/RandomLoadBalanceStrategy.java   (with props)
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/RoundRobinLoadBalanceStrategy.java   (with props)
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/StickyLoadBalanceStrategy.java   (with props)
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/TopicLoadBalanceStrategy.java   (with props)
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/package-info.java   (with props)
    activemq/camel/trunk/camel-core/src/main/resources/org/apache/camel/model/loadbalancer/
    activemq/camel/trunk/camel-core/src/main/resources/org/apache/camel/model/loadbalancer/jaxb.index
      - copied, changed from r594843, activemq/camel/trunk/camel-core/src/main/resources/org/apache/camel/model/jaxb.index
    activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routeWithLoadBalance.xml   (with props)
    activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routeWithStickyLoadBalance.xml   (with props)
    activemq/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringLoadBalanceTest.java   (with props)
    activemq/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/loadBalance.xml   (with props)
Modified:
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/LoadBalanceType.java
    activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/loadbalancer/StickyLoadBalancer.java
    activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/model/XmlParseTest.java
    activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/model/XmlTestSupport.java
    activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/handler/CamelNamespaceHandler.java

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/LoadBalanceType.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/LoadBalanceType.java?rev=595933&r1=595932&r2=595933&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/LoadBalanceType.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/LoadBalanceType.java Sat Nov 17 02:39:53 2007
@@ -24,7 +24,10 @@
 
 import javax.xml.bind.annotation.XmlAccessType;
 import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlElementRef;
+import javax.xml.bind.annotation.XmlElements;
 import javax.xml.bind.annotation.XmlRootElement;
 
 import org.apache.camel.Exchange;
@@ -32,6 +35,12 @@
 import org.apache.camel.Predicate;
 import org.apache.camel.Processor;
 import org.apache.camel.impl.RouteContext;
+
+import org.apache.camel.model.loadbalancer.LoadBalancerType;
+import org.apache.camel.model.loadbalancer.RandomLoadBalanceStrategy;
+import org.apache.camel.model.loadbalancer.RoundRobinLoadBalanceStrategy;
+import org.apache.camel.model.loadbalancer.StickyLoadBalanceStrategy;
+import org.apache.camel.model.loadbalancer.TopicLoadBalanceStrategy;
 import org.apache.camel.processor.ChoiceProcessor;
 import org.apache.camel.processor.FilterProcessor;
 import org.apache.camel.processor.SendProcessor;
@@ -41,18 +50,48 @@
 import org.apache.camel.processor.loadbalancer.TopicLoadBalancer;
 import org.apache.camel.processor.loadbalancer.RandomLoadBalancer;
 import org.apache.camel.util.CollectionStringBuffer;
+import org.apache.camel.util.ObjectHelper;
 
 @XmlRootElement(name = "loadBalance")
-@XmlAccessorType(XmlAccessType.NONE)
+@XmlAccessorType(XmlAccessType.FIELD)
 public class LoadBalanceType extends OutputType<LoadBalanceType> {
-   
-    // how to define it in XML    
-    private LoadBalancer loadBalancer;
+    @XmlAttribute(required = false)
+    private String ref;
+    
+    @XmlElements({
+        @XmlElement(required = false, name = "roundRobin", type = RoundRobinLoadBalanceStrategy.class),
+        @XmlElement(required = false, name = "random", type = RandomLoadBalanceStrategy.class),
+        @XmlElement(required = false, name = "sticky", type = StickyLoadBalanceStrategy.class),
+        @XmlElement(required = false, name = "topic", type = TopicLoadBalanceStrategy.class)}
+        )
+    private LoadBalancerType loadBalancerType;
+    
+    
+          
+    public LoadBalanceType() {
+        
+    }
+    
+    public String getRef() {
+        return ref;
+    }
+    
+    public void setRef(String ref) {
+        this.ref = ref;
+    }
+    
+    public LoadBalancerType getLoadBalancerType() {
+        return loadBalancerType;
+    }
+    
+    public void setLoadBalancerType(LoadBalancerType loadbalancer) {
+        loadBalancerType = loadbalancer;
+    }
     
-       
     protected Processor createOutputsProcessor(RouteContext routeContext, Collection<ProcessorType<?>> outputs)
         throws Exception {
-        assert loadBalancer != null;
+        ObjectHelper.notNull(loadBalancerType, "loadBalancerType");
+        LoadBalancer loadBalancer = loadBalancerType.getLoadBalancer(routeContext);
         for (ProcessorType processorType : outputs) {
             //The outputs should be the SendProcessor
             SendProcessor processor =(SendProcessor) processorType.createProcessor(routeContext);
@@ -65,7 +104,8 @@
     // when this method will be called
     @Override
     public Processor createProcessor(RouteContext routeContext) throws Exception {
-        assert loadBalancer != null;        
+        ObjectHelper.notNull(loadBalancerType, "loadBalancerType");
+        LoadBalancer loadBalancer = loadBalancerType.getLoadBalancer(routeContext);
         for (ProcessorType processorType : getOutputs()) {
             //The outputs should be the SendProcessor
             SendProcessor processor =(SendProcessor) processorType.createProcessor(routeContext);
@@ -79,27 +119,27 @@
     // Fluent API
     // -------------------------------------------------------------------------
     public LoadBalanceType setLoadBalancer(LoadBalancer loadBalancer) {
-        this.loadBalancer = loadBalancer;
+        loadBalancerType = new LoadBalancerType(loadBalancer);
         return this;
     }
     
     public LoadBalanceType roundRobin() {
-        loadBalancer = new RoundRobinLoadBalancer();
+        loadBalancerType = new LoadBalancerType(new RoundRobinLoadBalancer());
         return this;        
     }
     
     public LoadBalanceType random() {
-        loadBalancer = new RandomLoadBalancer();
+        loadBalancerType = new LoadBalancerType(new RandomLoadBalancer());
         return this;
     }
     
     public LoadBalanceType sticky(Expression<Exchange> correlationExpression) {
-        loadBalancer = new StickyLoadBalancer(correlationExpression);
+        loadBalancerType = new LoadBalancerType(new StickyLoadBalancer(correlationExpression));
         return this;
     }
     
     public LoadBalanceType topic() {
-        loadBalancer = new TopicLoadBalancer();
+        loadBalancerType = new LoadBalancerType(new TopicLoadBalancer());
         return this;
     }
     
@@ -116,9 +156,17 @@
     
     @Override
     public String toString() {
-        return "LoadBanlance[ " + getOutputs() + "]";
+        String result;
+        if (loadBalancerType != null) {
+            result = "LoadBalanceType[" + loadBalancerType + ", ";
+        } else {
+            result =  "LoadBalanceType[" + ref + ", ";
+        } 
+        result = result + getOutputs() + "]";
+        return result;
     }
 
+    
     
 
     

Added: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/LoadBalancerType.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/LoadBalancerType.java?rev=595933&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/LoadBalancerType.java (added)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/LoadBalancerType.java Sat Nov 17 02:39:53 2007
@@ -0,0 +1,133 @@
+/**
+ *
+ * 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.camel.model.loadbalancer;
+
+import static org.apache.camel.util.ObjectHelper.notNull;
+
+import java.util.List;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlTransient;
+import javax.xml.bind.annotation.XmlType;
+
+import org.apache.camel.Exchange;
+import org.apache.camel.Processor;
+import org.apache.camel.impl.RouteContext;
+import org.apache.camel.model.IdentifiedType;
+import org.apache.camel.processor.loadbalancer.LoadBalancer;
+import org.apache.camel.util.IntrospectionSupport;
+import org.apache.camel.util.ObjectHelper;
+
+@XmlType(name = "loadBalancerType")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class LoadBalancerType extends IdentifiedType implements LoadBalancer {
+
+    @XmlTransient
+    private LoadBalancer loadBalancer;
+    @XmlTransient
+    private String loadBalancerTypeName;
+    
+    public static LoadBalancer getLoadBalancer(RouteContext routeContext, LoadBalancerType type, String ref) {
+        if (type == null) {
+            notNull(ref, "ref or LoadBalancerType");
+            LoadBalancer loadBalancer = routeContext.lookup(ref, LoadBalancer.class);
+            if (loadBalancer instanceof LoadBalancerType) {
+                type = (LoadBalancerType) loadBalancer;
+            }
+            else {
+                return loadBalancer;
+            }
+        }
+        return type.getLoadBalancer(routeContext);
+    }
+    
+    public LoadBalancerType () {        
+    }
+    
+    public LoadBalancerType(LoadBalancer loadBalancer) {
+        this.loadBalancer = loadBalancer;
+    }
+    
+    protected LoadBalancerType(String loadBalancerTypeName) {
+        this.loadBalancerTypeName = loadBalancerTypeName;
+    }
+    
+    /**
+     * Sets a named property on the data format instance using introspection
+     */
+    protected void setProperty(Object bean, String name, Object value) {
+        try {
+            IntrospectionSupport.setProperty(bean, name, value);
+        }
+        catch (Exception e) {
+            throw new IllegalArgumentException("Failed to set property " + name + " on " + bean + ". Reason: " + e, e);
+        }
+    }
+    
+    /**
+     * Allows derived classes to customize the load balancer
+     */
+    protected void configureLoadBalancer(LoadBalancer loadBalancer) {
+    }
+    
+    public LoadBalancer getLoadBalancer(RouteContext routeContext) {
+        if (loadBalancer == null) {
+            loadBalancer = createLoadBalancer(routeContext);
+            ObjectHelper.notNull(loadBalancer, "loadBalancer");
+            configureLoadBalancer(loadBalancer);
+        }
+        return loadBalancer;
+    }
+
+    /**
+     * Factory method to create the load balancer instance
+     */
+    protected LoadBalancer createLoadBalancer(RouteContext routeContext) {
+        if (loadBalancerTypeName != null) {
+            Class type = ObjectHelper.loadClass(loadBalancerTypeName, getClass().getClassLoader());
+            if (type == null) {
+                throw new IllegalArgumentException("The class " + loadBalancerTypeName + " is not on the classpath! Cannot use the loadBalancer " + this);
+            }
+            return (LoadBalancer) ObjectHelper.newInstance(type);
+        }
+        return null;
+    }
+
+    
+    public void addProcessor(Processor processor) {
+        ObjectHelper.notNull(loadBalancer, "loadBalancer");
+        loadBalancer.addProcessor(processor);
+    }
+
+    public List<Processor> getProcessors() {
+        ObjectHelper.notNull(loadBalancer, "loadBalancer");
+        return loadBalancer.getProcessors();
+    }
+
+    public void removeProcessor(Processor processor) {
+        ObjectHelper.notNull(loadBalancer, "loadBalancer");
+        loadBalancer.removeProcessor(processor);
+    }
+
+    public void process(Exchange exchange) throws Exception {
+        ObjectHelper.notNull(loadBalancer, "loadBalancer");
+        loadBalancer.process(exchange);
+    }
+
+}

Propchange: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/LoadBalancerType.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/RandomLoadBalanceStrategy.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/RandomLoadBalanceStrategy.java?rev=595933&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/RandomLoadBalanceStrategy.java (added)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/RandomLoadBalanceStrategy.java Sat Nov 17 02:39:53 2007
@@ -0,0 +1,34 @@
+/**
+ *
+ * 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.camel.model.loadbalancer;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.camel.impl.RouteContext;
+import org.apache.camel.processor.loadbalancer.LoadBalancer;
+
+
+@XmlRootElement(name = "random")
+public class RandomLoadBalanceStrategy extends LoadBalancerType {
+    
+    @Override
+    protected LoadBalancer createLoadBalancer (RouteContext routeContext) {
+        return new org.apache.camel.processor.loadbalancer.RandomLoadBalancer();
+    }
+
+}

Propchange: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/RandomLoadBalanceStrategy.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/RoundRobinLoadBalanceStrategy.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/RoundRobinLoadBalanceStrategy.java?rev=595933&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/RoundRobinLoadBalanceStrategy.java (added)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/RoundRobinLoadBalanceStrategy.java Sat Nov 17 02:39:53 2007
@@ -0,0 +1,33 @@
+/**
+ *
+ * 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.camel.model.loadbalancer;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.camel.impl.RouteContext;
+import org.apache.camel.processor.loadbalancer.LoadBalancer;
+
+@XmlRootElement(name = "roundRobin")
+public class RoundRobinLoadBalanceStrategy extends LoadBalancerType{
+    
+    @Override
+    protected LoadBalancer createLoadBalancer (RouteContext routeContext) {
+        return new org.apache.camel.processor.loadbalancer.RoundRobinLoadBalancer();
+    }
+
+}

Propchange: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/RoundRobinLoadBalanceStrategy.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/StickyLoadBalanceStrategy.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/StickyLoadBalanceStrategy.java?rev=595933&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/StickyLoadBalanceStrategy.java (added)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/StickyLoadBalanceStrategy.java Sat Nov 17 02:39:53 2007
@@ -0,0 +1,86 @@
+/**
+ *
+ * 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.camel.model.loadbalancer;
+
+import javax.xml.bind.annotation.XmlAccessType;
+import javax.xml.bind.annotation.XmlAccessorType;
+import javax.xml.bind.annotation.XmlAttribute;
+import javax.xml.bind.annotation.XmlElement;
+import javax.xml.bind.annotation.XmlRootElement;
+import javax.xml.bind.annotation.XmlValue;
+
+import org.apache.camel.model.dataformat.ArtixDSContentType;
+import org.apache.camel.model.language.ExpressionType;
+import org.apache.camel.processor.loadbalancer.LoadBalancer;
+
+@XmlRootElement(name = "sticky")
+@XmlAccessorType(XmlAccessType.FIELD)
+public class StickyLoadBalanceStrategy extends LoadBalancerType {
+    @XmlElement(required = true, name = "expression", type = ExpressionType.class)
+    private ExpressionType expressionType;
+    @XmlElement(required = false, name = "loadBalancer", type = ExpressionType.class)
+    private LoadBalancerType loadBalancerType;
+    
+    public StickyLoadBalanceStrategy() {
+        super("org.apache.camel.processor.loadbalancer.StickyLoadBalancer");
+    }
+    
+    public StickyLoadBalanceStrategy(ExpressionType expressionType) {
+        this();
+        this.expressionType = expressionType;
+    }
+    
+    public StickyLoadBalanceStrategy(ExpressionType expressionType, LoadBalancerType loadBalancerTyep) {
+        this();
+        this.expressionType = expressionType;
+        this.loadBalancerType = loadBalancerType;
+    }
+    
+    public void setExpressionType(ExpressionType expressionType) {
+        this.expressionType = expressionType;
+    }
+    
+    public ExpressionType getExpressionType() {
+        return expressionType;
+    }
+    
+    public void setLoadBalancerType(LoadBalancerType loadBalancerType) {
+        this.loadBalancerType = loadBalancerType;
+    }
+    
+    public LoadBalancerType getLoadBalancerType() {
+        return loadBalancerType;
+    }
+    
+    @Override
+    protected void configureLoadBalancer(LoadBalancer loadBalancer) {
+        ExpressionType expression = getExpressionType();
+        if (expression != null) {
+            setProperty(loadBalancer, "correlationExpression", expression);
+        }
+        LoadBalancerType type = getLoadBalancerType();
+        if (type != null) {
+            setProperty(loadBalancer, "loadBalancer", type);
+        }
+    }
+    
+    @Override
+    public String toString() {
+        return "StickyLoadBalanceStrategy[" + expressionType + ", " + loadBalancerType + "]"; 
+    }
+}

Propchange: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/StickyLoadBalanceStrategy.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/TopicLoadBalanceStrategy.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/TopicLoadBalanceStrategy.java?rev=595933&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/TopicLoadBalanceStrategy.java (added)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/TopicLoadBalanceStrategy.java Sat Nov 17 02:39:53 2007
@@ -0,0 +1,32 @@
+/**
+ *
+ * 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.camel.model.loadbalancer;
+
+import javax.xml.bind.annotation.XmlRootElement;
+
+import org.apache.camel.impl.RouteContext;
+import org.apache.camel.processor.loadbalancer.LoadBalancer;
+
+@XmlRootElement(name = "topic")
+public class TopicLoadBalanceStrategy extends LoadBalancerType {
+    @Override
+    protected LoadBalancer createLoadBalancer (RouteContext routeContext) {
+        return new org.apache.camel.processor.loadbalancer.TopicLoadBalancer();
+    }
+
+}

Propchange: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/TopicLoadBalanceStrategy.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/package-info.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/package-info.java?rev=595933&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/package-info.java (added)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/package-info.java Sat Nov 17 02:39:53 2007
@@ -0,0 +1,18 @@
+/**
+ * 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.
+ */
+@javax.xml.bind.annotation.XmlSchema(namespace = "http://activemq.apache.org/camel/schema/spring", elementFormDefault = javax.xml.bind.annotation.XmlNsForm.QUALIFIED)
+package org.apache.camel.model.loadbalancer;

Propchange: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/model/loadbalancer/package-info.java
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/loadbalancer/StickyLoadBalancer.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/loadbalancer/StickyLoadBalancer.java?rev=595933&r1=595932&r2=595933&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/loadbalancer/StickyLoadBalancer.java (original)
+++ activemq/camel/trunk/camel-core/src/main/java/org/apache/camel/processor/loadbalancer/StickyLoadBalancer.java Sat Nov 17 02:39:53 2007
@@ -38,6 +38,18 @@
     private int numberOfHashGroups = 64 * 1024;
     private Map<Object, Processor> stickyMap = new HashMap<Object, Processor>();
 
+    public StickyLoadBalancer() {
+        this.loadBalancer = new RoundRobinLoadBalancer();
+    }
+    
+    public void setCorrelationExpression(Expression<Exchange> correlationExpression) {
+        this.correlationExpression = correlationExpression;
+    }
+    
+    public void setLoadBalancer(QueueLoadBalancer loadBalancer) {
+        this.loadBalancer = loadBalancer;
+    }
+    
     public StickyLoadBalancer(Expression<Exchange> correlationExpression) {
         this(correlationExpression, new RoundRobinLoadBalancer());
     }

Copied: activemq/camel/trunk/camel-core/src/main/resources/org/apache/camel/model/loadbalancer/jaxb.index (from r594843, activemq/camel/trunk/camel-core/src/main/resources/org/apache/camel/model/jaxb.index)
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/main/resources/org/apache/camel/model/loadbalancer/jaxb.index?p2=activemq/camel/trunk/camel-core/src/main/resources/org/apache/camel/model/loadbalancer/jaxb.index&p1=activemq/camel/trunk/camel-core/src/main/resources/org/apache/camel/model/jaxb.index&r1=594843&r2=595933&rev=595933&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/main/resources/org/apache/camel/model/jaxb.index (original)
+++ activemq/camel/trunk/camel-core/src/main/resources/org/apache/camel/model/loadbalancer/jaxb.index Sat Nov 17 02:39:53 2007
@@ -14,35 +14,8 @@
 ## See the License for the specific language governing permissions and
 ## limitations under the License.
 ## ------------------------------------------------------------------------
-AggregatorType
-BeanRef
-CatchType
-ChoiceType
-CompletedPredicate
-DelayerType
-ExceptionType
-FilterType
-FinallyType
-FromType
-IdempotentConsumerType
-InterceptType
-InterceptorRef
-LoadBalanceType
-MarshalType
-MulticastType
-OtherwiseType
-PolicyRef
-ProceedType
-ProcessorRef
-RecipientListType
-RedeliveryPolicyType
-ResequencerType
-RouteType
-RoutesType
-ServiceActivationType
-SplitterType
-ThrottlerType
-ToType
-TryType
-UnmarshalType
-WhenType
\ No newline at end of file
+LoadBalancerType
+RandomLoadBalanceStrategy
+RoundRobinLoadBalanceStrategy
+StickyLoadBalanceStrategy
+TopicLoadBalanceStrategy
\ No newline at end of file

Modified: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/model/XmlParseTest.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/model/XmlParseTest.java?rev=595933&r1=595932&r2=595933&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/model/XmlParseTest.java (original)
+++ activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/model/XmlParseTest.java Sat Nov 17 02:39:53 2007
@@ -17,6 +17,9 @@
 package org.apache.camel.model;
 
 import org.apache.camel.model.language.ExpressionType;
+import org.apache.camel.model.loadbalancer.LoadBalancerType;
+import org.apache.camel.model.loadbalancer.RoundRobinLoadBalanceStrategy;
+import org.apache.camel.model.loadbalancer.StickyLoadBalanceStrategy;
 
 import javax.xml.bind.JAXBException;
 import java.util.List;
@@ -107,6 +110,24 @@
         assertExpression(splitter.getExpression(), "xpath", "/foo/bar");
         assertChildTo("to", splitter, "seda:b");
     }
+    
+    public void testParseLoadBalance() throws Exception {
+        RouteType route = assertOneRoute("routeWithLoadBalance.xml");        
+        assertFrom(route, "seda:a");        
+        LoadBalanceType loadBalance = assertLoadBalancer(route);
+        assertEquals("Here should have 3 output here", 3, loadBalance.getOutputs().size());
+        assertTrue("The loadBalancer shoud be RoundRobinLoadBalanceStrategy", loadBalance.getLoadBalancerType() instanceof RoundRobinLoadBalanceStrategy);
+    }
+    
+    public void testParseStickyLoadBalance() throws Exception {
+        RouteType route = assertOneRoute("routeWithStickyLoadBalance.xml");        
+        assertFrom(route, "seda:a");        
+        LoadBalanceType loadBalance = assertLoadBalancer(route);
+        assertEquals("Here should have 3 output here", 3, loadBalance.getOutputs().size());
+        assertTrue("The loadBalancer shoud be StickyLoadBalanceStrategy", loadBalance.getLoadBalancerType() instanceof StickyLoadBalanceStrategy);
+        StickyLoadBalanceStrategy strategy = (StickyLoadBalanceStrategy)loadBalance.getLoadBalancerType();
+        assertNotNull("the expression should not be null ", strategy.getExpressionType());
+    }
 
     public void testParseBatchResequencerXml() throws Exception {
         RouteType route = assertOneRoute("resequencerBatch.xml");
@@ -187,6 +208,11 @@
     protected SplitterType assertSplitter(ProcessorType<?> route) {
         ProcessorType<?> processor = assertOneElement(route.getOutputs());
         return assertIsInstanceOf(SplitterType.class, processor);
+    }
+    
+    protected LoadBalanceType assertLoadBalancer(ProcessorType<?> route) {
+        ProcessorType<?> processor = assertOneElement(route.getOutputs());
+        return assertIsInstanceOf(LoadBalanceType.class, processor);
     }
 
     protected ResequencerType assertResequencer(ProcessorType<?> route) {

Modified: activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/model/XmlTestSupport.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/model/XmlTestSupport.java?rev=595933&r1=595932&r2=595933&view=diff
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/model/XmlTestSupport.java (original)
+++ activemq/camel/trunk/camel-core/src/test/java/org/apache/camel/model/XmlTestSupport.java Sat Nov 17 02:39:53 2007
@@ -51,6 +51,6 @@
     @Override
     protected void setUp() throws Exception {
         super.setUp();
-        jaxbContext = JAXBContext.newInstance("org.apache.camel.model:org.apache.camel.model.config:org.apache.camel.model.dataformat:org.apache.camel.model.language");
+        jaxbContext = JAXBContext.newInstance("org.apache.camel.model:org.apache.camel.model.config:org.apache.camel.model.dataformat:org.apache.camel.model.language:org.apache.camel.model.loadbalancer");
     }
 }

Added: activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routeWithLoadBalance.xml
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routeWithLoadBalance.xml?rev=595933&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routeWithLoadBalance.xml (added)
+++ activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routeWithLoadBalance.xml Sat Nov 17 02:39:53 2007
@@ -0,0 +1,28 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<routes id="camel" xmlns="http://activemq.apache.org/camel/schema/spring">
+  <route>
+    <from uri="seda:a"/>
+    <loadBalance>
+       <roundRobin/>
+       <to uri="seda:b"/>      
+       <to uri="seda:c"/>      
+       <to uri="seda:d"/>       
+    </loadBalance>    
+  </route>
+</routes>

Propchange: activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routeWithLoadBalance.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Added: activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routeWithStickyLoadBalance.xml
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routeWithStickyLoadBalance.xml?rev=595933&view=auto
==============================================================================
--- activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routeWithStickyLoadBalance.xml (added)
+++ activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routeWithStickyLoadBalance.xml Sat Nov 17 02:39:53 2007
@@ -0,0 +1,30 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<routes id="camel" xmlns="http://activemq.apache.org/camel/schema/spring">
+  <route>
+    <from uri="seda:a"/>
+    <loadBalance>
+       <sticky>
+          <expression language="juel">in.header.foo == 'bar'</expression>
+       </sticky>
+       <to uri="seda:b"/>      
+       <to uri="seda:c"/>      
+       <to uri="seda:d"/>       
+    </loadBalance>    
+  </route>
+</routes>

Propchange: activemq/camel/trunk/camel-core/src/test/resources/org/apache/camel/model/routeWithStickyLoadBalance.xml
------------------------------------------------------------------------------
    svn:eol-style = native

Modified: activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/handler/CamelNamespaceHandler.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/handler/CamelNamespaceHandler.java?rev=595933&r1=595932&r2=595933&view=diff
==============================================================================
--- activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/handler/CamelNamespaceHandler.java (original)
+++ activemq/camel/trunk/components/camel-spring/src/main/java/org/apache/camel/spring/handler/CamelNamespaceHandler.java Sat Nov 17 02:39:53 2007
@@ -30,6 +30,10 @@
 import org.apache.camel.model.dataformat.JaxbDataFormat;
 import org.apache.camel.model.dataformat.SerializationDataFormat;
 import org.apache.camel.model.dataformat.XMLBeansDataFormat;
+import org.apache.camel.model.loadbalancer.RandomLoadBalanceStrategy;
+import org.apache.camel.model.loadbalancer.RoundRobinLoadBalanceStrategy;
+import org.apache.camel.model.loadbalancer.StickyLoadBalanceStrategy;
+import org.apache.camel.model.loadbalancer.TopicLoadBalanceStrategy;
 import org.apache.camel.spi.NamespaceAware;
 import org.apache.camel.spring.CamelBeanPostProcessor;
 import org.apache.camel.spring.CamelContextFactoryBean;
@@ -49,7 +53,7 @@
 import org.w3c.dom.NodeList;
 
 public class CamelNamespaceHandler extends NamespaceHandlerSupport {
-    public static final String JAXB_PACKAGES = "org.apache.camel.spring:org.apache.camel.model:org.apache.camel.model.config:org.apache.camel.model.dataformat:org.apache.camel.model.language";
+    public static final String JAXB_PACKAGES = "org.apache.camel.spring:org.apache.camel.model:org.apache.camel.model.config:org.apache.camel.model.dataformat:org.apache.camel.model.language:org.apache.camel.model.loadbalancer";
     protected BeanDefinitionParser endpointParser = new BeanDefinitionParser(EndpointFactoryBean.class);
     protected BeanDefinitionParser beanPostProcessorParser = new BeanDefinitionParser(CamelBeanPostProcessor.class);
     protected Set<String> parserElementNames = new HashSet<String>();
@@ -68,6 +72,12 @@
         addBeanDefinitionParser("serialization", SerializationDataFormat.class);
         addBeanDefinitionParser("xmlBeans", XMLBeansDataFormat.class);
 
+        //load balancers
+        addBeanDefinitionParser("roundRobin", RoundRobinLoadBalanceStrategy.class);
+        addBeanDefinitionParser("random", RandomLoadBalanceStrategy.class);
+        addBeanDefinitionParser("sticky", StickyLoadBalanceStrategy.class);
+        addBeanDefinitionParser("topic", TopicLoadBalanceStrategy.class);
+        
         // TODO switch to use the above mechanism?
         registerParser("endpoint", endpointParser);
 

Added: activemq/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringLoadBalanceTest.java
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringLoadBalanceTest.java?rev=595933&view=auto
==============================================================================
--- activemq/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringLoadBalanceTest.java (added)
+++ activemq/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringLoadBalanceTest.java Sat Nov 17 02:39:53 2007
@@ -0,0 +1,28 @@
+/**
+ * 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.camel.spring.processor;
+
+import static org.apache.camel.spring.processor.SpringTestHelper.createSpringCamelContext;
+import org.apache.camel.CamelContext;
+import org.apache.camel.processor.LoadBalanceTest;
+
+public class SpringLoadBalanceTest extends LoadBalanceTest {
+    protected CamelContext createCamelContext() throws Exception {
+        return createSpringCamelContext(this, "org/apache/camel/spring/processor/loadBalance.xml");
+    }
+
+}

Propchange: activemq/camel/trunk/components/camel-spring/src/test/java/org/apache/camel/spring/processor/SpringLoadBalanceTest.java
------------------------------------------------------------------------------
    svn:eol-style = native

Added: activemq/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/loadBalance.xml
URL: http://svn.apache.org/viewvc/activemq/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/loadBalance.xml?rev=595933&view=auto
==============================================================================
--- activemq/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/loadBalance.xml (added)
+++ activemq/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/loadBalance.xml Sat Nov 17 02:39:53 2007
@@ -0,0 +1,39 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+    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.
+-->
+<beans xmlns="http://www.springframework.org/schema/beans"
+       xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+       xsi:schemaLocation="
+       http://www.springframework.org/schema/beans http://www.springframework.org/schema/beans/spring-beans-2.0.xsd
+       http://activemq.apache.org/camel/schema/spring http://activemq.apache.org/camel/schema/spring/camel-spring.xsd
+    ">
+
+  <!-- START SNIPPET: example -->
+  <camelContext id="camel" xmlns="http://activemq.apache.org/camel/schema/spring">
+    <route>
+      <from uri="direct:start"/>
+      <loadBalance>
+          <to uri="mock:x"/>        
+          <to uri="mock:y"/>       
+          <to uri="mock:z"/>
+          <roundRobin/>
+      </loadBalance>
+    </route>
+  </camelContext>
+  <!-- END SNIPPET: example -->
+
+</beans>

Propchange: activemq/camel/trunk/components/camel-spring/src/test/resources/org/apache/camel/spring/processor/loadBalance.xml
------------------------------------------------------------------------------
    svn:eol-style = native