You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by lb...@apache.org on 2019/11/26 10:42:56 UTC

[camel-k-runtime] branch master updated (89d6bcf -> 4a9ebf3)

This is an automated email from the ASF dual-hosted git repository.

lburgazzoli pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git.


    from 89d6bcf  Allow to use any class as configuration source
     new ee06153  YAML: fix ToDynamicStepParser
     new 52fec19  YAML: improve filter dsl test
     new 6361af1  YAML: add support for Aggregate EIP
     new ad1a50c  YAML: add support for Bean EIP
     new 1689b7c  YAML: add support for Enrich EIP
     new 1ccd4cb  YAML: add support for Loop EIP
     new 45f114f  YAML: add support for Multicast EIP
     new 4d8723b  YAML: add support for PollEnrich EIP
     new 013c0f7  YAML: add support for RecipientList EIP
     new 882799a  YAML: add support for Resequence EIP
     new ea687b8  YAML: add support for RoutingSlip EIP
     new 9c89eae  YAML: add support for Sample EIP
     new a9d530a  YAML: add support for Sort EIP
     new 09f3a38  YAML: add support for Stop EIP
     new 5662013  YAML: add support for Threads EIP
     new 32c9d3c  YAML: add support for Throttle EIP
     new a51e8be  YAML: add support for Validate EIP
     new 4a9ebf3  YAML: add support for WireTap EIP

The 18 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../k/loader/yaml/parser/AggregateStepParser.java  |  64 ++++++++++++
 .../{LogStepParser.java => BeanStepParser.java}    |   8 +-
 ...HeaderStepParser.java => EnrichStepParser.java} |  30 +++---
 .../{FilterStepParser.java => LoopStepParser.java} |  12 +--
 ...terStepParser.java => MulticastStepParser.java} |  12 +--
 ...erStepParser.java => PollEnrichStepParser.java} |  30 +++---
 ...tepParser.java => RecipientListStepParser.java} |  12 +--
 .../k/loader/yaml/parser/ResequenceStepParser.java |  75 ++++++++++++++
 ...yStepParser.java => RoutingSlipStepParser.java} |  16 +--
 ...FilterStepParser.java => SampleStepParser.java} |  12 +--
 .../{DelayStepParser.java => SortStepParser.java}  |  12 +--
 .../{LogStepParser.java => StopStepParser.java}    |   8 +-
 ...ocessStepParser.java => ThreadsStepParser.java} |   8 +-
 ...BodyStepParser.java => ThrottleStepParser.java} |  12 +--
 .../k/loader/yaml/parser/ToDynamicStepParser.java  |  13 ++-
 ...BodyStepParser.java => ValidateStepParser.java} |  12 +--
 .../k/loader/yaml/parser/WireTapStepParser.java    | 110 +++++++++++++++++++++
 ...efinitionTest.groovy => DefinitionsTest.groovy} |   2 +-
 .../yaml/{RouteTest.groovy => RoutesTest.groovy}   |  70 ++++++-------
 .../apache/camel/k/loader/yaml/TestSupport.groovy  |  38 ++++++-
 .../k/loader/yaml/parser/AggregateTest.groovy      |  75 ++++++++++++++
 .../{RemovePropertyTest.groovy => BeanTest.groovy} |  15 +--
 .../{FilterTest.groovy => EnrichTest.groovy}       |  30 +++---
 .../camel/k/loader/yaml/parser/FilterTest.groovy   |  18 ++--
 .../{SetHeaderTest.groovy => LoopTest.groovy}      |  36 ++++---
 .../{ProcessTest.groovy => MulticastTest.groovy}   |  18 ++--
 .../{DelayTest.groovy => PollEnrichTest.groovy}    |  34 ++++---
 ...cRouterTest.groovy => RecipientListTest.groovy} |  42 +++++---
 .../k/loader/yaml/parser/ResequenceTest.groovy     |  83 ++++++++++++++++
 ...tPropertyTest.groovy => RoutingSlipTest.groovy} |  38 +++----
 .../{ProcessTest.groovy => SampleTest.groovy}      |  12 +--
 .../parser/{ProcessTest.groovy => SortTest.groovy} |  12 +--
 .../{ProcessTest.groovy => ThreadsTest.groovy}     |  12 +--
 .../{SetBodyTest.groovy => ThrottleTest.groovy}    |  33 ++++---
 .../{SetHeaderTest.groovy => ValidateTest.groovy}  |  32 +++---
 .../camel/k/loader/yaml/parser/WireTapTest.groovy  |  60 +++++++++++
 .../resources/routes/RoutesTest_aggregator.yaml    |  12 ++-
 .../test/resources/routes/RoutesTest_filter.yaml   |  17 +++-
 .../test/resources/routes/RoutesTest_split.yaml    |  17 +++-
 39 files changed, 867 insertions(+), 285 deletions(-)
 create mode 100644 camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/AggregateStepParser.java
 copy camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/{LogStepParser.java => BeanStepParser.java} (85%)
 copy camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/{RemoveHeaderStepParser.java => EnrichStepParser.java} (59%)
 copy camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/{FilterStepParser.java => LoopStepParser.java} (79%)
 copy camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/{FilterStepParser.java => MulticastStepParser.java} (79%)
 copy camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/{RemoveHeaderStepParser.java => PollEnrichStepParser.java} (59%)
 copy camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/{DynamicRouterStepParser.java => RecipientListStepParser.java} (78%)
 create mode 100644 camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/ResequenceStepParser.java
 copy camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/{SetPropertyStepParser.java => RoutingSlipStepParser.java} (71%)
 copy camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/{FilterStepParser.java => SampleStepParser.java} (79%)
 copy camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/{DelayStepParser.java => SortStepParser.java} (76%)
 copy camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/{LogStepParser.java => StopStepParser.java} (85%)
 copy camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/{ProcessStepParser.java => ThreadsStepParser.java} (84%)
 copy camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/{SetBodyStepParser.java => ThrottleStepParser.java} (76%)
 copy camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/{SetBodyStepParser.java => ValidateStepParser.java} (76%)
 create mode 100644 camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/WireTapStepParser.java
 rename camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/{RouteDefinitionTest.groovy => DefinitionsTest.groovy} (99%)
 rename camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/{RouteTest.groovy => RoutesTest.groovy} (63%)
 create mode 100644 camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/AggregateTest.groovy
 copy camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/{RemovePropertyTest.groovy => BeanTest.groovy} (75%)
 copy camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/{FilterTest.groovy => EnrichTest.groovy} (62%)
 copy camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/{SetHeaderTest.groovy => LoopTest.groovy} (61%)
 copy camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/{ProcessTest.groovy => MulticastTest.groovy} (66%)
 copy camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/{DelayTest.groovy => PollEnrichTest.groovy} (61%)
 copy camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/{DynamicRouterTest.groovy => RecipientListTest.groovy} (51%)
 create mode 100644 camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/ResequenceTest.groovy
 copy camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/{SetPropertyTest.groovy => RoutingSlipTest.groovy} (57%)
 copy camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/{ProcessTest.groovy => SampleTest.groovy} (78%)
 copy camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/{ProcessTest.groovy => SortTest.groovy} (77%)
 copy camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/{ProcessTest.groovy => ThreadsTest.groovy} (80%)
 copy camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/{SetBodyTest.groovy => ThrottleTest.groovy} (56%)
 copy camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/{SetHeaderTest.groovy => ValidateTest.groovy} (62%)
 create mode 100644 camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/WireTapTest.groovy
 copy camel-k-loader-knative/src/test/resources/routes.yaml => camel-k-loader-yaml/src/test/resources/routes/RoutesTest_aggregator.yaml (78%)
 copy camel-k-loader-knative/src/test/resources/routes.yaml => camel-k-loader-yaml/src/test/resources/routes/RoutesTest_filter.yaml (73%)
 copy camel-k-loader-knative/src/test/resources/routes.yaml => camel-k-loader-yaml/src/test/resources/routes/RoutesTest_split.yaml (77%)


[camel-k-runtime] 07/18: YAML: add support for Multicast EIP

Posted by lb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lburgazzoli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git

commit 45f114fd1023675d300176909f079da25dc9518b
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Sun Nov 24 15:32:01 2019 +0100

    YAML: add support for Multicast EIP
---
 .../k/loader/yaml/parser/MulticastStepParser.java  | 49 ++++++++++++++++++++++
 .../k/loader/yaml/parser/MulticastTest.groovy      | 42 +++++++++++++++++++
 2 files changed, 91 insertions(+)

diff --git a/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/MulticastStepParser.java b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/MulticastStepParser.java
new file mode 100644
index 0000000..edc43f2
--- /dev/null
+++ b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/MulticastStepParser.java
@@ -0,0 +1,49 @@
+/*
+ * 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.k.loader.yaml.parser;
+
+import java.util.List;
+
+import org.apache.camel.k.annotation.yaml.YAMLStepParser;
+import org.apache.camel.k.loader.yaml.model.Step;
+import org.apache.camel.model.MulticastDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.reifier.MulticastReifier;
+import org.apache.camel.reifier.ProcessorReifier;
+
+@YAMLStepParser("multicast")
+public class MulticastStepParser implements ProcessorStepParser {
+    static {
+        ProcessorReifier.registerReifier(Definition.class, MulticastReifier::new);
+    }
+
+    @Override
+    public ProcessorDefinition<?> toProcessor(Context context) {
+        Definition definition = context.node(Definition.class);
+
+        return StepParserSupport.convertSteps(
+            context,
+            definition,
+            definition.steps
+        );
+    }
+
+    public static final class Definition extends MulticastDefinition implements Step.Definition {
+        public List<Step> steps;
+    }
+}
+
diff --git a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/MulticastTest.groovy b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/MulticastTest.groovy
new file mode 100644
index 0000000..d411503
--- /dev/null
+++ b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/MulticastTest.groovy
@@ -0,0 +1,42 @@
+/*
+ * 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.k.loader.yaml.parser
+
+import org.apache.camel.k.loader.yaml.TestSupport
+import org.apache.camel.model.MulticastDefinition
+
+class MulticastTest extends TestSupport {
+
+    def "definition"() {
+        given:
+            def stepContext = stepContext('''
+                 stop-on-exception: true
+                 parallel-processing: true
+                 steps:
+                   - to: "direct:a"
+                   - to: "direct:b"
+            ''')
+        when:
+            def processor = new MulticastStepParser().toProcessor(stepContext)
+        then:
+            with (processor, MulticastDefinition) {
+                stopOnException == true
+                parallelProcessing == true
+                outputs.size() == 2
+            }
+    }
+}


[camel-k-runtime] 02/18: YAML: improve filter dsl test

Posted by lb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lburgazzoli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git

commit 52fec199a5604cd3a792ab20740f9c06b16c1bf5
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Sun Nov 24 21:31:35 2019 +0100

    YAML: improve filter dsl test
---
 .../camel/k/loader/yaml/parser/FilterTest.groovy       | 18 +++++++++++-------
 1 file changed, 11 insertions(+), 7 deletions(-)

diff --git a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/FilterTest.groovy b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/FilterTest.groovy
index 9249a3b..6e78d63 100644
--- a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/FilterTest.groovy
+++ b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/FilterTest.groovy
@@ -29,10 +29,12 @@ class FilterTest extends TestSupport {
         when:
             def processor = new FilterStepParser().toProcessor(stepContext)
         then:
-            def p = processor as FilterDefinition
-
-            p.expression.language == 'simple'
-            p.expression.expression == '${body}'
+            with (processor, FilterDefinition) {
+                with (expression) {
+                    language == 'simple'
+                    expression == '${body}'
+                }
+            }
     }
 
     def "definition with expression block"() {
@@ -44,9 +46,11 @@ class FilterTest extends TestSupport {
         when:
             def processor = new FilterStepParser().toProcessor(stepContext)
         then:
-            with(processor, FilterDefinition) {
-                expression.language == 'simple'
-                expression.expression == '${body}'
+            with (processor, FilterDefinition) {
+                with (expression) {
+                    language == 'simple'
+                    expression == '${body}'
+                }
             }
     }
 


[camel-k-runtime] 10/18: YAML: add support for Resequence EIP

Posted by lb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lburgazzoli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git

commit 882799a8730861e1830b4e588d01ee56cab66cd3
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Sun Nov 24 16:44:46 2019 +0100

    YAML: add support for Resequence EIP
---
 .../k/loader/yaml/parser/ResequenceStepParser.java | 75 +++++++++++++++++++
 .../k/loader/yaml/parser/ResequenceTest.groovy     | 83 ++++++++++++++++++++++
 2 files changed, 158 insertions(+)

diff --git a/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/ResequenceStepParser.java b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/ResequenceStepParser.java
new file mode 100644
index 0000000..400fffe
--- /dev/null
+++ b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/ResequenceStepParser.java
@@ -0,0 +1,75 @@
+/*
+ * 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.k.loader.yaml.parser;
+
+import java.util.List;
+
+import com.fasterxml.jackson.annotation.JsonAlias;
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import org.apache.camel.Expression;
+import org.apache.camel.k.annotation.yaml.YAMLStepParser;
+import org.apache.camel.k.loader.yaml.model.Step;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.ResequenceDefinition;
+import org.apache.camel.model.config.BatchResequencerConfig;
+import org.apache.camel.model.config.StreamResequencerConfig;
+import org.apache.camel.reifier.ProcessorReifier;
+import org.apache.camel.reifier.ResequenceReifier;
+
+@YAMLStepParser("resequence")
+public class ResequenceStepParser implements ProcessorStepParser {
+    static {
+        ProcessorReifier.registerReifier(Definition.class, ResequenceReifier::new);
+    }
+
+    @Override
+    public ProcessorDefinition<?> toProcessor(Context context) {
+        Definition definition = context.node(Definition.class);
+
+        return StepParserSupport.convertSteps(
+            context,
+            definition,
+            definition.steps
+        );
+    }
+
+    public static final class Definition extends ResequenceDefinition implements HasExpression, Step.Definition {
+        public List<Step> steps;
+
+        @JsonIgnore
+        public void setExpression(Expression expression) {
+            super.setExpression(expression);
+        }
+
+        @JsonAlias("batch-config")
+        public void setBatchConfig(BatchResequencerConfig config) {
+            if (getResequencerConfig() != null) {
+                throw new IllegalArgumentException("And resequencer config has already been set");
+            }
+            setResequencerConfig(config);
+        }
+
+        @JsonAlias("stream-config")
+        public void setStreamConfig(StreamResequencerConfig config) {
+            if (getResequencerConfig() != null) {
+                throw new IllegalArgumentException("And resequencer config has already been set");
+            }
+            setResequencerConfig(config);
+        }
+    }
+}
+
diff --git a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/ResequenceTest.groovy b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/ResequenceTest.groovy
new file mode 100644
index 0000000..78b4c7a
--- /dev/null
+++ b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/ResequenceTest.groovy
@@ -0,0 +1,83 @@
+/*
+ * 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.k.loader.yaml.parser
+
+import org.apache.camel.k.loader.yaml.TestSupport
+import org.apache.camel.model.ResequenceDefinition
+import org.apache.camel.model.config.StreamResequencerConfig
+import org.apache.camel.model.language.SimpleExpression
+
+class ResequenceTest extends TestSupport {
+
+    def "definition with expression"() {
+        given:
+            def stepContext = stepContext('''
+                simple: "${in.header.seqnum}"
+                stream-config:
+                    capacity: 5000
+                    timeout: 4000  
+                steps:
+                  - to: "direct:a"
+                  - to: "direct:b"
+            ''')
+        when:
+            def processor = new ResequenceStepParser().toProcessor(stepContext)
+        then:
+            with (processor, ResequenceDefinition) {
+                with (expression, SimpleExpression) {
+                    language == 'simple'
+                    expression == '${in.header.seqnum}'
+                }
+                with (streamConfig, StreamResequencerConfig) {
+                    capacity == 5000
+                    timeout == 4000
+                }
+
+                outputs.size() == 2
+            }
+    }
+
+    def "definition with expression block"() {
+        given:
+            def stepContext = stepContext('''
+                expression:
+                    simple: "${in.header.seqnum}"
+                stream-config:
+                    capacity: 5000
+                    timeout: 4000
+                steps:
+                  - to: "direct:a"
+                  - to: "direct:b"
+            ''')
+        when:
+            def processor = new ResequenceStepParser().toProcessor(stepContext)
+        then:
+            with (processor, ResequenceDefinition) {
+                with (expression, SimpleExpression) {
+                    language == 'simple'
+                    expression == '${in.header.seqnum}'
+                }
+                with (streamConfig, StreamResequencerConfig) {
+                    capacity == 5000
+                    timeout == 4000
+                }
+
+                outputs.size() == 2
+            }
+    }
+
+}


[camel-k-runtime] 09/18: YAML: add support for RecipientList EIP

Posted by lb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lburgazzoli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git

commit 013c0f72688adb4992db93cd50cd9a9f30a049db
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Sun Nov 24 15:44:40 2019 +0100

    YAML: add support for RecipientList EIP
---
 .../yaml/parser/RecipientListStepParser.java       | 40 +++++++++++++
 .../k/loader/yaml/parser/RecipientListTest.groovy  | 67 ++++++++++++++++++++++
 2 files changed, 107 insertions(+)

diff --git a/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/RecipientListStepParser.java b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/RecipientListStepParser.java
new file mode 100644
index 0000000..10eeebe
--- /dev/null
+++ b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/RecipientListStepParser.java
@@ -0,0 +1,40 @@
+/*
+ * 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.k.loader.yaml.parser;
+
+import org.apache.camel.k.annotation.yaml.YAMLStepParser;
+import org.apache.camel.k.loader.yaml.model.Step;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.RecipientListDefinition;
+import org.apache.camel.reifier.ProcessorReifier;
+import org.apache.camel.reifier.RecipientListReifier;
+
+@YAMLStepParser("recipient-list")
+public class RecipientListStepParser implements ProcessorStepParser {
+    static {
+        ProcessorReifier.registerReifier(Definition.class, RecipientListReifier::new);
+    }
+
+    @Override
+    public ProcessorDefinition<?> toProcessor(Context context) {
+        return context.node(Definition.class);
+    }
+
+    public static final class Definition extends RecipientListDefinition implements HasExpression, Step.Definition {
+    }
+}
+
diff --git a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/RecipientListTest.groovy b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/RecipientListTest.groovy
new file mode 100644
index 0000000..bd33b87
--- /dev/null
+++ b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/RecipientListTest.groovy
@@ -0,0 +1,67 @@
+/*
+ * 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.k.loader.yaml.parser
+
+import org.apache.camel.k.loader.yaml.TestSupport
+import org.apache.camel.model.RecipientListDefinition
+import org.apache.camel.model.language.ConstantExpression
+
+class RecipientListTest extends TestSupport {
+
+    def "definition with expression"() {
+        given:
+            def stepContext = stepContext('''
+                constant: "direct:a,direct:b"
+                stop-on-exception: true
+                parallel-processing: true
+            ''')
+        when:
+            def processor = new RecipientListStepParser().toProcessor(stepContext)
+        then:
+            with (processor, RecipientListDefinition) {
+                stopOnException == true
+                parallelProcessing == true
+
+                with(expression, ConstantExpression) {
+                    language == 'constant'
+                    expression == 'direct:a,direct:b'
+                }
+            }
+    }
+
+    def "definition with expression block"() {
+        given:
+            def stepContext = stepContext('''
+                expression:
+                    constant: "direct:a,direct:b"
+                stop-on-exception: true
+                parallel-processing: true
+            ''')
+        when:
+            def processor = new RecipientListStepParser().toProcessor(stepContext)
+        then:
+            with (processor, RecipientListDefinition) {
+                stopOnException == true
+                parallelProcessing == true
+
+                with(expression, ConstantExpression) {
+                    language == 'constant'
+                    expression == 'direct:a,direct:b'
+                }
+            }
+    }
+}


[camel-k-runtime] 17/18: YAML: add support for Validate EIP

Posted by lb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lburgazzoli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git

commit a51e8bef2a728551591272316547ba76df8cbe7a
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Sun Nov 24 21:34:51 2019 +0100

    YAML: add support for Validate EIP
---
 .../k/loader/yaml/parser/ValidateStepParser.java   | 40 +++++++++++++++
 .../camel/k/loader/yaml/parser/ValidateTest.groovy | 57 ++++++++++++++++++++++
 2 files changed, 97 insertions(+)

diff --git a/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/ValidateStepParser.java b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/ValidateStepParser.java
new file mode 100644
index 0000000..a6e325c
--- /dev/null
+++ b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/ValidateStepParser.java
@@ -0,0 +1,40 @@
+/*
+ * 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.k.loader.yaml.parser;
+
+import org.apache.camel.k.annotation.yaml.YAMLStepParser;
+import org.apache.camel.k.loader.yaml.model.Step;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.ValidateDefinition;
+import org.apache.camel.reifier.ProcessorReifier;
+import org.apache.camel.reifier.ValidateReifier;
+
+@YAMLStepParser("validate")
+public class ValidateStepParser implements ProcessorStepParser {
+    static {
+        ProcessorReifier.registerReifier(Definition.class, ValidateReifier::new);
+    }
+
+    @Override
+    public ProcessorDefinition<?> toProcessor(Context context) {
+        return context.node(Definition.class);
+    }
+
+    public static final class Definition extends ValidateDefinition implements HasExpression, Step.Definition {
+    }
+}
+
diff --git a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/ValidateTest.groovy b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/ValidateTest.groovy
new file mode 100644
index 0000000..58f405e
--- /dev/null
+++ b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/ValidateTest.groovy
@@ -0,0 +1,57 @@
+/*
+ * 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.k.loader.yaml.parser
+
+import org.apache.camel.k.loader.yaml.TestSupport
+import org.apache.camel.model.ValidateDefinition
+
+class ValidateTest extends TestSupport {
+
+    def "definition with expression"() {
+        given:
+            def stepContext = stepContext('''
+                 simple: "${in.header.bar} == 100"
+            ''')
+        when:
+            def processor = new ValidateStepParser().toProcessor(stepContext)
+        then:
+            with (processor, ValidateDefinition) {
+                with (expression) {
+                    language == 'simple'
+                    expression == '${in.header.bar} == 100'
+                }
+            }
+    }
+
+    def "definition with expression block"() {
+        given:
+            def stepContext = stepContext('''
+                 expression:
+                   simple: "${in.header.bar} == 100"
+            ''')
+        when:
+            def processor = new ValidateStepParser().toProcessor(stepContext)
+        then:
+            with (processor, ValidateDefinition) {
+                with (expression) {
+                    language == 'simple'
+                    expression == '${in.header.bar} == 100'
+                }
+            }
+    }
+
+}


[camel-k-runtime] 06/18: YAML: add support for Loop EIP

Posted by lb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lburgazzoli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git

commit 1ccd4cb2e42d1838e0bbcff7c823d37f940764a1
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Sun Nov 24 15:23:49 2019 +0100

    YAML: add support for Loop EIP
---
 .../camel/k/loader/yaml/parser/LoopStepParser.java | 49 +++++++++++++++++
 .../camel/k/loader/yaml/parser/LoopTest.groovy     | 61 ++++++++++++++++++++++
 2 files changed, 110 insertions(+)

diff --git a/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/LoopStepParser.java b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/LoopStepParser.java
new file mode 100644
index 0000000..9735404
--- /dev/null
+++ b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/LoopStepParser.java
@@ -0,0 +1,49 @@
+/*
+ * 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.k.loader.yaml.parser;
+
+import java.util.List;
+
+import org.apache.camel.k.annotation.yaml.YAMLStepParser;
+import org.apache.camel.k.loader.yaml.model.Step;
+import org.apache.camel.model.LoopDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.reifier.LoopReifier;
+import org.apache.camel.reifier.ProcessorReifier;
+
+@YAMLStepParser("loop")
+public class LoopStepParser implements ProcessorStepParser {
+    static {
+        ProcessorReifier.registerReifier(Definition.class, LoopReifier::new);
+    }
+
+    @Override
+    public ProcessorDefinition<?> toProcessor(Context context) {
+        Definition definition = context.node(Definition.class);
+
+        return StepParserSupport.convertSteps(
+            context,
+            definition,
+            definition.steps
+        );
+    }
+
+    public static final class Definition extends LoopDefinition implements HasExpression, Step.Definition {
+        public List<Step> steps;
+    }
+}
+
diff --git a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/LoopTest.groovy b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/LoopTest.groovy
new file mode 100644
index 0000000..6ef474f
--- /dev/null
+++ b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/LoopTest.groovy
@@ -0,0 +1,61 @@
+/*
+ * 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.k.loader.yaml.parser
+
+import org.apache.camel.k.loader.yaml.TestSupport
+import org.apache.camel.model.LoopDefinition
+
+class LoopTest extends TestSupport {
+
+    def "definition with expression"() {
+        given:
+            def stepContext = stepContext('''
+                 constant: "3"
+                 do-while: true
+            ''')
+        when:
+            def processor = new LoopStepParser().toProcessor(stepContext)
+        then:
+            with(processor, LoopDefinition) {
+                with(expression) {
+                    language == 'constant'
+                    expression == '3'
+                }
+                doWhile == true
+            }
+    }
+
+    def "definition with expression block"() {
+        given:
+            def stepContext = stepContext('''
+                 expression:
+                   constant: "3"
+                 do-while: true
+            ''')
+        when:
+            def processor = new LoopStepParser().toProcessor(stepContext)
+        then:
+            with(processor, LoopDefinition) {
+                with(expression) {
+                    language == 'constant'
+                    expression == '3'
+                }
+                doWhile == true
+            }
+    }
+
+}


[camel-k-runtime] 03/18: YAML: add support for Aggregate EIP

Posted by lb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lburgazzoli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git

commit 6361af15ab3ed0acfb4c92aa043daa57b079e872
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Sun Nov 24 11:20:04 2019 +0100

    YAML: add support for Aggregate EIP
---
 .../k/loader/yaml/parser/AggregateStepParser.java  | 64 ++++++++++++++++++
 ...efinitionTest.groovy => DefinitionsTest.groovy} |  2 +-
 .../yaml/{RouteTest.groovy => RoutesTest.groovy}   | 70 +++++++++-----------
 .../apache/camel/k/loader/yaml/TestSupport.groovy  | 38 ++++++++++-
 .../k/loader/yaml/parser/AggregateTest.groovy      | 75 ++++++++++++++++++++++
 .../resources/routes/RoutesTest_aggregator.yaml    | 25 ++++++++
 .../test/resources/routes/RoutesTest_filter.yaml   | 30 +++++++++
 .../test/resources/routes/RoutesTest_split.yaml    | 30 +++++++++
 8 files changed, 293 insertions(+), 41 deletions(-)

diff --git a/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/AggregateStepParser.java b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/AggregateStepParser.java
new file mode 100644
index 0000000..52b192b
--- /dev/null
+++ b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/AggregateStepParser.java
@@ -0,0 +1,64 @@
+/*
+ * 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.k.loader.yaml.parser;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import org.apache.camel.Expression;
+import org.apache.camel.k.annotation.yaml.YAMLStepParser;
+import org.apache.camel.k.loader.yaml.model.Step;
+import org.apache.camel.model.AggregateDefinition;
+import org.apache.camel.model.ExpressionSubElementDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.language.ExpressionDefinition;
+import org.apache.camel.reifier.AggregateReifier;
+import org.apache.camel.reifier.ProcessorReifier;
+
+@YAMLStepParser("aggregate")
+public class AggregateStepParser implements ProcessorStepParser {
+    static {
+        ProcessorReifier.registerReifier(Definition.class, AggregateReifier::new);
+    }
+
+    @Override
+    public ProcessorDefinition<?> toProcessor(Context context) {
+        return context.node(Definition.class);
+    }
+
+    public static final class Definition extends AggregateDefinition implements HasExpression, Step.Definition {
+        @JsonIgnore
+        public void setExpression(Expression expression) {
+            super.setExpression(expression);
+        }
+
+        public void setCorrelationExpression(CorrelationExpression correlationExpression) {
+            super.setCorrelationExpression(correlationExpression);
+        }
+    }
+
+    public static final class CorrelationExpression extends ExpressionSubElementDefinition implements HasExpression {
+        @Override
+        public void setExpression(ExpressionDefinition expressionDefinition) {
+            super.setExpressionType(expressionDefinition);
+        }
+
+        @Override
+        public ExpressionDefinition getExpression() {
+            return super.getExpressionType();
+        }
+    }
+}
+
diff --git a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/RouteDefinitionTest.groovy b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/DefinitionsTest.groovy
similarity index 99%
rename from camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/RouteDefinitionTest.groovy
rename to camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/DefinitionsTest.groovy
index b63d4ba..0fe243d 100644
--- a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/RouteDefinitionTest.groovy
+++ b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/DefinitionsTest.groovy
@@ -27,7 +27,7 @@ import org.apache.commons.io.IOUtils
 
 import java.nio.charset.StandardCharsets
 
-class RouteDefinitionTest extends TestSupport {
+class DefinitionsTest extends TestSupport {
 
     def "route with id"() {
         given:
diff --git a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/RouteTest.groovy b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/RoutesTest.groovy
similarity index 63%
rename from camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/RouteTest.groovy
rename to camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/RoutesTest.groovy
index 72dd75a..23fd49b 100644
--- a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/RouteTest.groovy
+++ b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/RoutesTest.groovy
@@ -18,27 +18,13 @@ package org.apache.camel.k.loader.yaml
 
 
 import org.apache.camel.component.mock.MockEndpoint
+import org.apache.camel.processor.aggregate.UseLatestAggregationStrategy
 
-class RouteTest extends TestSupport {
+class RoutesTest extends TestSupport {
 
-    def 'test split'() {
+    def 'split'() {
         setup:
-            def context = startContext('''
-                - from:
-                    uri: "direct:route"
-                    steps:
-                      - split:
-                          tokenize: ","
-                          steps:
-                            - to: "mock:split"
-                      - to: "mock:route"
-                - from:
-                    uri: "direct:flow"
-                    steps:
-                      - split:
-                          tokenize: ","
-                      - to: "mock:flow"
-            ''')
+            def context = startContext()
 
             mockEndpoint(context,'mock:split') {
                 expectedMessageCount = 3
@@ -63,35 +49,20 @@ class RouteTest extends TestSupport {
             context?.stop()
     }
 
-    def 'test filter'() {
+    def 'filter'() {
         setup:
-            def context = startContext('''
-                - from:
-                    uri: "direct:route"
-                    steps:
-                      - filter:
-                          simple: "${body.startsWith(\\"a\\")}"
-                          steps:
-                            - to: "mock:filter"
-                      - to: "mock:route"
-                - from:
-                    uri: "direct:flow"
-                    steps:
-                      - filter:
-                          simple: "${body.startsWith(\\"a\\")}"
-                      - to: "mock:flow"
-            ''')
+            def context = startContext()
 
             mockEndpoint(context, 'mock:route') {
-                expectedMessageCount = 2
+                expectedMessageCount 2
                 expectedBodiesReceived 'a', 'b'
             }
             mockEndpoint(context, 'mock:filter') {
-                expectedMessageCount = 1
+                expectedMessageCount 1
                 expectedBodiesReceived 'a'
             }
             mockEndpoint(context,'mock:flow') {
-                expectedMessageCount = 1
+                expectedMessageCount 1
                 expectedBodiesReceived 'a'
             }
         when:
@@ -106,4 +77,27 @@ class RouteTest extends TestSupport {
         cleanup:
             context?.stop()
     }
+
+    def 'aggregator'() {
+        setup:
+            def context = startContext([
+                'aggregatorStrategy': new UseLatestAggregationStrategy()
+            ])
+
+            mockEndpoint(context, 'mock:route') {
+                expectedMessageCount 2
+                expectedBodiesReceived '2', '4'
+            }
+        when:
+            context.createProducerTemplate().with {
+                sendBodyAndHeader('direct:route', '1', 'StockSymbol', 1)
+                sendBodyAndHeader('direct:route', '2', 'StockSymbol', 1)
+                sendBodyAndHeader('direct:route', '3', 'StockSymbol', 2)
+                sendBodyAndHeader('direct:route', '4', 'StockSymbol', 2)
+            }
+        then:
+            MockEndpoint.assertIsSatisfied(context)
+        cleanup:
+            context?.stop()
+    }
 }
diff --git a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/TestSupport.groovy b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/TestSupport.groovy
index 400c59f..fea7a41 100644
--- a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/TestSupport.groovy
+++ b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/TestSupport.groovy
@@ -43,9 +43,29 @@ class TestSupport extends Specification {
     }
 
     static CamelContext startContext(String content) {
+        return startContext(content, [:])
+    }
+
+    static CamelContext startContext(String content, Map<String, Object> beans) {
+        return startContext(
+                IOUtils.toInputStream(content.stripMargin(), StandardCharsets.UTF_8),
+                beans
+        )
+    }
+
+    static CamelContext startContext(InputStream content) {
+        return startContext(content, [:])
+    }
+
+    static CamelContext startContext(InputStream content, Map<String, Object> beans) {
         def context = new DefaultCamelContext()
-        def istream = IOUtils.toInputStream(content.stripMargin(), StandardCharsets.UTF_8)
-        def builder = new YamlSourceLoader().builder(istream)
+        def builder = new YamlSourceLoader().builder(content)
+
+        if (beans) {
+            beans.each {
+                k, v -> context.registry.bind(k, v)
+            }
+        }
 
         context.disableJMX()
         context.setStreamCaching(true)
@@ -55,6 +75,20 @@ class TestSupport extends Specification {
         return context
     }
 
+    CamelContext startContext() {
+        return startContext([:])
+    }
+
+    CamelContext startContext(Map<String, Object> beans) {
+        def name = specificationContext.currentIteration.name.replace(' ', '_')
+        def path = "/routes/${specificationContext.currentSpec.name}_${name}.yaml"
+
+        return startContext(
+                TestSupport.class.getResourceAsStream(path) as InputStream,
+                beans
+        )
+    }
+
     static MockEndpoint mockEndpoint(
             CamelContext context,
             String uri,
diff --git a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/AggregateTest.groovy b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/AggregateTest.groovy
new file mode 100644
index 0000000..8335dcf
--- /dev/null
+++ b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/AggregateTest.groovy
@@ -0,0 +1,75 @@
+/*
+ * 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.k.loader.yaml.parser
+
+import org.apache.camel.k.loader.yaml.TestSupport
+import org.apache.camel.model.AggregateDefinition
+import org.apache.camel.model.language.SimpleExpression
+
+class AggregateTest extends TestSupport {
+
+    def "definition"() {
+        given:
+            def stepContext = stepContext('''
+                 expression:
+                     simple: "${header.ID}"
+                 correlation-expression:
+                     simple: "${header.Count}"
+                 strategy-ref: "myAppender"
+                 completion-size: 10
+            ''')
+        when:
+            def processor = new AggregateStepParser().toProcessor(stepContext)
+        then:
+            with(processor, AggregateDefinition) {
+                strategyRef == 'myAppender'
+                completionSize == 10
+
+                with(expression, SimpleExpression) {
+                    expression ==  '${header.ID}'
+                }
+                with(correlationExpression?.expressionType, SimpleExpression) {
+                    expression == '${header.Count}'
+                }
+            }
+    }
+
+    def "compact efinition"() {
+        given:
+            def stepContext = stepContext('''
+                 simple: "${header.ID}"
+                 correlation-expression:
+                     simple: "${header.Count}"
+                 strategy-ref: "myAppender"
+                 completion-size: 10
+            ''')
+        when:
+            def processor = new AggregateStepParser().toProcessor(stepContext)
+        then:
+            with(processor, AggregateDefinition) {
+                strategyRef == 'myAppender'
+                completionSize == 10
+
+                with(expression, SimpleExpression) {
+                    expression ==  '${header.ID}'
+                }
+                with(correlationExpression?.expressionType, SimpleExpression) {
+                    expression == '${header.Count}'
+                }
+            }
+    }
+}
diff --git a/camel-k-loader-yaml/src/test/resources/routes/RoutesTest_aggregator.yaml b/camel-k-loader-yaml/src/test/resources/routes/RoutesTest_aggregator.yaml
new file mode 100644
index 0000000..1634826
--- /dev/null
+++ b/camel-k-loader-yaml/src/test/resources/routes/RoutesTest_aggregator.yaml
@@ -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.
+#
+- from:
+    uri: "direct:route"
+    steps:
+      - aggregate:
+          strategy-ref: "aggregatorStrategy"
+          completion-size: 2
+          correlation-expression:
+            simple: "${header.StockSymbol}"
+      - to: "mock:route"
\ No newline at end of file
diff --git a/camel-k-loader-yaml/src/test/resources/routes/RoutesTest_filter.yaml b/camel-k-loader-yaml/src/test/resources/routes/RoutesTest_filter.yaml
new file mode 100644
index 0000000..e34b1ff
--- /dev/null
+++ b/camel-k-loader-yaml/src/test/resources/routes/RoutesTest_filter.yaml
@@ -0,0 +1,30 @@
+#
+# 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.
+#
+- from:
+    uri: "direct:route"
+    steps:
+      - filter:
+          simple: "${body.startsWith(\"a\")}"
+          steps:
+            - to: "mock:filter"
+      - to: "mock:route"
+- from:
+    uri: "direct:flow"
+    steps:
+      - filter:
+          simple: "${body.startsWith(\"a\")}"
+      - to: "mock:flow"
\ No newline at end of file
diff --git a/camel-k-loader-yaml/src/test/resources/routes/RoutesTest_split.yaml b/camel-k-loader-yaml/src/test/resources/routes/RoutesTest_split.yaml
new file mode 100644
index 0000000..5011482
--- /dev/null
+++ b/camel-k-loader-yaml/src/test/resources/routes/RoutesTest_split.yaml
@@ -0,0 +1,30 @@
+#
+# 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.
+#
+- from:
+    uri: "direct:route"
+    steps:
+      - split:
+          tokenize: ","
+          steps:
+            - to: "mock:split"
+      - to: "mock:route"
+- from:
+    uri: "direct:flow"
+    steps:
+      - split:
+          tokenize: ","
+      - to: "mock:flow"
\ No newline at end of file


[camel-k-runtime] 15/18: YAML: add support for Threads EIP

Posted by lb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lburgazzoli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git

commit 5662013907a4b7659a118ebcc2460efbd1cff40a
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Sun Nov 24 21:23:47 2019 +0100

    YAML: add support for Threads EIP
---
 .../k/loader/yaml/parser/ThreadsStepParser.java    | 30 ++++++++++++++++++
 .../camel/k/loader/yaml/parser/ThreadsTest.groovy  | 36 ++++++++++++++++++++++
 2 files changed, 66 insertions(+)

diff --git a/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/ThreadsStepParser.java b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/ThreadsStepParser.java
new file mode 100644
index 0000000..a21fc33
--- /dev/null
+++ b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/ThreadsStepParser.java
@@ -0,0 +1,30 @@
+/*
+ * 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.k.loader.yaml.parser;
+
+import org.apache.camel.k.annotation.yaml.YAMLStepParser;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.ThreadsDefinition;
+
+@YAMLStepParser("threads")
+public class ThreadsStepParser implements ProcessorStepParser {
+    @Override
+    public ProcessorDefinition<?> toProcessor(Context context) {
+        return context.node(ThreadsDefinition.class);
+    }
+}
+
diff --git a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/ThreadsTest.groovy b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/ThreadsTest.groovy
new file mode 100644
index 0000000..2573bce
--- /dev/null
+++ b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/ThreadsTest.groovy
@@ -0,0 +1,36 @@
+/*
+ * 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.k.loader.yaml.parser
+
+import org.apache.camel.k.loader.yaml.TestSupport
+import org.apache.camel.model.ThreadsDefinition
+
+class ThreadsTest extends TestSupport {
+
+    def "definition"() {
+        given:
+            def stepContext = stepContext('''
+                 pool-size: 5
+            ''')
+        when:
+            def processor = new ThreadsStepParser().toProcessor(stepContext)
+        then:
+            with (processor, ThreadsDefinition) {
+                poolSize == 5
+            }
+    }
+}


[camel-k-runtime] 14/18: YAML: add support for Stop EIP

Posted by lb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lburgazzoli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git

commit 09f3a382f0f440584b16093a7625ca0dca77d466
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Sun Nov 24 21:23:35 2019 +0100

    YAML: add support for Stop EIP
---
 .../camel/k/loader/yaml/parser/StopStepParser.java | 30 ++++++++++++++++++++++
 1 file changed, 30 insertions(+)

diff --git a/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/StopStepParser.java b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/StopStepParser.java
new file mode 100644
index 0000000..7e147bc
--- /dev/null
+++ b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/StopStepParser.java
@@ -0,0 +1,30 @@
+/*
+ * 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.k.loader.yaml.parser;
+
+import org.apache.camel.k.annotation.yaml.YAMLStepParser;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.StopDefinition;
+
+@YAMLStepParser("stop")
+public class StopStepParser implements ProcessorStepParser {
+    @Override
+    public ProcessorDefinition<?> toProcessor(Context context) {
+        return context.node(StopDefinition.class);
+    }
+}
+


[camel-k-runtime] 08/18: YAML: add support for PollEnrich EIP

Posted by lb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lburgazzoli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git

commit 4d8723be22a5b08bfa357057e123a1cf227a6c8d
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Sun Nov 24 15:36:08 2019 +0100

    YAML: add support for PollEnrich EIP
---
 .../k/loader/yaml/parser/PollEnrichStepParser.java | 54 +++++++++++++++++++
 .../k/loader/yaml/parser/PollEnrichTest.groovy     | 61 ++++++++++++++++++++++
 2 files changed, 115 insertions(+)

diff --git a/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/PollEnrichStepParser.java b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/PollEnrichStepParser.java
new file mode 100644
index 0000000..4f24db3
--- /dev/null
+++ b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/PollEnrichStepParser.java
@@ -0,0 +1,54 @@
+/*
+ * 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.k.loader.yaml.parser;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import org.apache.camel.Expression;
+import org.apache.camel.k.annotation.yaml.YAMLStepParser;
+import org.apache.camel.k.loader.yaml.model.Step;
+import org.apache.camel.model.PollEnrichDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.reifier.PollEnrichReifier;
+import org.apache.camel.reifier.ProcessorReifier;
+
+@YAMLStepParser("poll-enrich")
+public class PollEnrichStepParser implements ProcessorStepParser {
+    static {
+        ProcessorReifier.registerReifier(Definition.class, PollEnrichReifier::new);
+    }
+
+    @Override
+    public ProcessorDefinition<?> toProcessor(Context context) {
+        return context.node(Definition.class);
+    }
+
+    public static final class Definition extends PollEnrichDefinition implements HasExpression, Step.Definition {
+        @JsonIgnore
+        public void setExpression(Expression expression) {
+            super.setExpression(expression);
+        }
+
+        public void setStrategyRef(String aggregationStrategyRef) {
+            super.setAggregationStrategyRef(aggregationStrategyRef);
+        }
+
+        public String getStrategyMethodName() {
+            return super.getAggregationStrategyRef();
+        }
+    }
+}
+
diff --git a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/PollEnrichTest.groovy b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/PollEnrichTest.groovy
new file mode 100644
index 0000000..1e2b3dc
--- /dev/null
+++ b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/PollEnrichTest.groovy
@@ -0,0 +1,61 @@
+/*
+ * 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.k.loader.yaml.parser
+
+import org.apache.camel.k.loader.yaml.TestSupport
+import org.apache.camel.model.PollEnrichDefinition
+
+class PollEnrichTest extends TestSupport {
+
+    def "definition with expression"() {
+        given:
+            def stepContext = stepContext('''
+                 simple: "${body}"
+                 strategy-ref: "myStrategy"
+            ''')
+        when:
+            def processor = new PollEnrichStepParser().toProcessor(stepContext)
+        then:
+            with(processor, PollEnrichDefinition) {
+                with(expression) {
+                    language == 'simple'
+                    expression == '${body}'
+                }
+                aggregationStrategyRef == 'myStrategy'
+            }
+    }
+
+    def "definition with expression block"() {
+        given:
+            def stepContext = stepContext('''
+                 expression:
+                   simple: "${body}"
+                 strategy-ref: "myStrategy"
+            ''')
+        when:
+            def processor = new PollEnrichStepParser().toProcessor(stepContext)
+        then:
+            with(processor, PollEnrichDefinition) {
+                with(expression) {
+                    language == 'simple'
+                    expression == '${body}'
+                }
+                aggregationStrategyRef == 'myStrategy'
+            }
+    }
+
+}


[camel-k-runtime] 13/18: YAML: add support for Sort EIP

Posted by lb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lburgazzoli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git

commit a9d530a26678506952c7b216fe524664d18c17fb
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Sun Nov 24 21:16:58 2019 +0100

    YAML: add support for Sort EIP
---
 .../camel/k/loader/yaml/parser/SortStepParser.java | 40 ++++++++++++++++++++++
 .../camel/k/loader/yaml/parser/SortTest.groovy     | 36 +++++++++++++++++++
 2 files changed, 76 insertions(+)

diff --git a/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/SortStepParser.java b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/SortStepParser.java
new file mode 100644
index 0000000..449b275
--- /dev/null
+++ b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/SortStepParser.java
@@ -0,0 +1,40 @@
+/*
+ * 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.k.loader.yaml.parser;
+
+import org.apache.camel.k.annotation.yaml.YAMLStepParser;
+import org.apache.camel.k.loader.yaml.model.Step;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.SortDefinition;
+import org.apache.camel.reifier.ProcessorReifier;
+import org.apache.camel.reifier.SortReifier;
+
+@YAMLStepParser("sort")
+public class SortStepParser implements ProcessorStepParser {
+    static {
+        ProcessorReifier.registerReifier(Definition.class, SortReifier::new);
+    }
+
+    @Override
+    public ProcessorDefinition<?> toProcessor(Context context) {
+        return context.node(Definition.class);
+    }
+
+    public static final class Definition extends SortDefinition implements HasExpression, Step.Definition {
+    }
+}
+
diff --git a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/SortTest.groovy b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/SortTest.groovy
new file mode 100644
index 0000000..c3b5345
--- /dev/null
+++ b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/SortTest.groovy
@@ -0,0 +1,36 @@
+/*
+ * 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.k.loader.yaml.parser
+
+import org.apache.camel.k.loader.yaml.TestSupport
+import org.apache.camel.model.SortDefinition
+
+class SortTest extends TestSupport {
+
+    def "definition"() {
+        given:
+            def stepContext = stepContext('''
+                 comparator-ref: "myComparator"
+            ''')
+        when:
+            def processor = new SortStepParser().toProcessor(stepContext)
+        then:
+            with (processor, SortDefinition) {
+                comparatorRef == 'myComparator'
+            }
+    }
+}


[camel-k-runtime] 11/18: YAML: add support for RoutingSlip EIP

Posted by lb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lburgazzoli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git

commit ea687b8ef08bb4502e3cd0267f14a453904d126a
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Sun Nov 24 17:02:05 2019 +0100

    YAML: add support for RoutingSlip EIP
---
 .../loader/yaml/parser/RoutingSlipStepParser.java  | 44 ++++++++++++++++
 .../k/loader/yaml/parser/RoutingSlipTest.groovy    | 61 ++++++++++++++++++++++
 2 files changed, 105 insertions(+)

diff --git a/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/RoutingSlipStepParser.java b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/RoutingSlipStepParser.java
new file mode 100644
index 0000000..83522b4
--- /dev/null
+++ b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/RoutingSlipStepParser.java
@@ -0,0 +1,44 @@
+/*
+ * 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.k.loader.yaml.parser;
+
+import org.apache.camel.Expression;
+import org.apache.camel.k.annotation.yaml.YAMLStepParser;
+import org.apache.camel.k.loader.yaml.model.Step;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.RoutingSlipDefinition;
+import org.apache.camel.reifier.ProcessorReifier;
+import org.apache.camel.reifier.RoutingSlipReifier;
+
+@YAMLStepParser("routing-slip")
+public class RoutingSlipStepParser implements ProcessorStepParser {
+    static {
+        ProcessorReifier.registerReifier(Definition.class, RoutingSlipReifier::new);
+    }
+
+    @Override
+    public ProcessorDefinition<?> toProcessor(Context context) {
+        return context.node(Definition.class);
+    }
+
+    public static final class Definition extends RoutingSlipDefinition implements HasExpression, Step.Definition {
+        public Definition() {
+            super((Expression) null, RoutingSlipDefinition.DEFAULT_DELIMITER);
+        }
+    }
+}
+
diff --git a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/RoutingSlipTest.groovy b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/RoutingSlipTest.groovy
new file mode 100644
index 0000000..332a88c
--- /dev/null
+++ b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/RoutingSlipTest.groovy
@@ -0,0 +1,61 @@
+/*
+ * 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.k.loader.yaml.parser
+
+import org.apache.camel.k.loader.yaml.TestSupport
+import org.apache.camel.model.RoutingSlipDefinition
+import org.apache.camel.model.language.SimpleExpression
+
+class RoutingSlipTest extends TestSupport {
+
+    def "definition with expression"() {
+        given:
+            def stepContext = stepContext('''
+                 simple: "${header.MyHeader}"
+                 uri-delimiter: "|"
+            ''')
+        when:
+            def processor = new RoutingSlipStepParser().toProcessor(stepContext)
+        then:
+            with(processor, RoutingSlipDefinition) {
+                with(expression, SimpleExpression) {
+                    language == 'simple'
+                    expression == '${header.MyHeader}'
+                }
+                uriDelimiter == '|'
+            }
+    }
+
+    def "definition with expression block"() {
+        given:
+            def stepContext = stepContext('''
+                 expression:
+                    simple: "${header.MyHeader}"
+                 uri-delimiter: "|"
+            ''')
+        when:
+            def processor = new RoutingSlipStepParser().toProcessor(stepContext)
+        then:
+            with(processor, RoutingSlipDefinition) {
+                with(expression, SimpleExpression) {
+                    language == 'simple'
+                    expression == '${header.MyHeader}'
+                }
+                uriDelimiter == '|'
+            }
+    }
+}


[camel-k-runtime] 04/18: YAML: add support for Bean EIP

Posted by lb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lburgazzoli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git

commit ad1a50cd6a2b43fdeee37747ea132601c9151b57
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Sun Nov 24 15:00:38 2019 +0100

    YAML: add support for Bean EIP
---
 .../camel/k/loader/yaml/parser/BeanStepParser.java | 30 ++++++++++++++++
 .../camel/k/loader/yaml/parser/BeanTest.groovy     | 40 ++++++++++++++++++++++
 2 files changed, 70 insertions(+)

diff --git a/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/BeanStepParser.java b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/BeanStepParser.java
new file mode 100644
index 0000000..a60f712
--- /dev/null
+++ b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/BeanStepParser.java
@@ -0,0 +1,30 @@
+/*
+ * 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.k.loader.yaml.parser;
+
+import org.apache.camel.k.annotation.yaml.YAMLStepParser;
+import org.apache.camel.model.BeanDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+
+@YAMLStepParser("bean")
+public class BeanStepParser implements ProcessorStepParser {
+    @Override
+    public ProcessorDefinition<?> toProcessor(Context context) {
+        return context.node(BeanDefinition.class);
+    }
+}
+
diff --git a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/BeanTest.groovy b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/BeanTest.groovy
new file mode 100644
index 0000000..78f1486
--- /dev/null
+++ b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/BeanTest.groovy
@@ -0,0 +1,40 @@
+/*
+ * 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.k.loader.yaml.parser
+
+
+import org.apache.camel.k.loader.yaml.TestSupport
+import org.apache.camel.model.BeanDefinition
+
+class BeanTest extends TestSupport {
+
+    def "definition"() {
+        given:
+            def stepContext = stepContext('''
+                 ref: "myBean"
+                 method: "process"
+            ''')
+        when:
+            def processor = new BeanStepParser().toProcessor(stepContext)
+        then:
+            with(processor, BeanDefinition) {
+                ref == 'myBean'
+                method == 'process'
+            }
+    }
+
+}


[camel-k-runtime] 01/18: YAML: fix ToDynamicStepParser

Posted by lb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lburgazzoli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git

commit ee06153cc808a7fc964dfd33cd31ee09d8b5587f
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Sun Nov 24 23:07:42 2019 +0100

    YAML: fix ToDynamicStepParser
---
 .../camel/k/loader/yaml/parser/ToDynamicStepParser.java     | 13 +++++++++----
 1 file changed, 9 insertions(+), 4 deletions(-)

diff --git a/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/ToDynamicStepParser.java b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/ToDynamicStepParser.java
index 9605824..465bbe3 100644
--- a/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/ToDynamicStepParser.java
+++ b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/ToDynamicStepParser.java
@@ -24,18 +24,23 @@ import com.fasterxml.jackson.annotation.JsonIgnore;
 import org.apache.camel.k.annotation.yaml.YAMLStepParser;
 import org.apache.camel.k.loader.yaml.model.Step;
 import org.apache.camel.model.ProcessorDefinition;
-import org.apache.camel.model.ToDefinition;
 import org.apache.camel.model.ToDynamicDefinition;
+import org.apache.camel.reifier.ProcessorReifier;
+import org.apache.camel.reifier.ToDynamicReifier;
 import org.apache.camel.util.URISupport;
 
 @YAMLStepParser("tod")
 public class ToDynamicStepParser implements ProcessorStepParser {
+    static {
+        ProcessorReifier.registerReifier(Definition.class, ToDynamicReifier::new);
+    }
+
     @Override
     public ProcessorDefinition<?> toProcessor(Context context) {
-        return new ToDynamicDefinition(context.node(Definition.class).getEndpointUri());
+        return context.node(Definition.class);
     }
 
-    public static final class Definition extends ToDefinition implements Step.Definition {
+    public static final class Definition extends ToDynamicDefinition implements Step.Definition {
         public Map<String, Object> parameters;
 
         public Definition() {
@@ -47,7 +52,7 @@ public class ToDynamicStepParser implements ProcessorStepParser {
 
         @JsonIgnore
         public String getEndpointUri() {
-            String answer = uri;
+            String answer = getUri();
 
             if (parameters != null) {
                 try {


[camel-k-runtime] 16/18: YAML: add support for Throttle EIP

Posted by lb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lburgazzoli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git

commit 32c9d3c13f3a74ced1585c736129ea51d1bde2ec
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Sun Nov 24 21:28:52 2019 +0100

    YAML: add support for Throttle EIP
---
 .../k/loader/yaml/parser/ThrottleStepParser.java   | 40 ++++++++++++++
 .../camel/k/loader/yaml/parser/ThrottleTest.groovy | 62 ++++++++++++++++++++++
 2 files changed, 102 insertions(+)

diff --git a/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/ThrottleStepParser.java b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/ThrottleStepParser.java
new file mode 100644
index 0000000..a5ef353
--- /dev/null
+++ b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/ThrottleStepParser.java
@@ -0,0 +1,40 @@
+/*
+ * 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.k.loader.yaml.parser;
+
+import org.apache.camel.k.annotation.yaml.YAMLStepParser;
+import org.apache.camel.k.loader.yaml.model.Step;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.ThrottleDefinition;
+import org.apache.camel.reifier.ProcessorReifier;
+import org.apache.camel.reifier.ThrottleReifier;
+
+@YAMLStepParser("throttle")
+public class ThrottleStepParser implements ProcessorStepParser {
+    static {
+        ProcessorReifier.registerReifier(Definition.class, ThrottleReifier::new);
+    }
+
+    @Override
+    public ProcessorDefinition<?> toProcessor(Context context) {
+        return context.node(Definition.class);
+    }
+
+    public static final class Definition extends ThrottleDefinition implements HasExpression, Step.Definition {
+    }
+}
+
diff --git a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/ThrottleTest.groovy b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/ThrottleTest.groovy
new file mode 100644
index 0000000..c5eccb8
--- /dev/null
+++ b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/ThrottleTest.groovy
@@ -0,0 +1,62 @@
+/*
+ * 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.k.loader.yaml.parser
+
+import org.apache.camel.k.loader.yaml.TestSupport
+import org.apache.camel.model.ThrottleDefinition
+import org.apache.camel.model.language.ConstantExpression
+
+class ThrottleTest extends TestSupport {
+
+    def "definition with expression"() {
+        given:
+            def stepContext = stepContext('''
+                 constant: "5s"
+                 executor-service-ref: "myExecutor"
+            ''')
+        when:
+            def processor = new ThrottleStepParser().toProcessor(stepContext)
+        then:
+            with (processor, ThrottleDefinition) {
+                with (expression, ConstantExpression) {
+                    language == 'constant'
+                    expression == '5s'
+                }
+                executorServiceRef == 'myExecutor'
+            }
+    }
+
+    def "definition with expression block"() {
+        given:
+            def stepContext = stepContext('''
+                 expression:
+                     constant: "5s"
+                 executor-service-ref: "myExecutor"
+            ''')
+        when:
+            def processor = new ThrottleStepParser().toProcessor(stepContext)
+        then:
+            with (processor, ThrottleDefinition) {
+                with (expression, ConstantExpression) {
+                    language == 'constant'
+                    expression == '5s'
+                }
+                executorServiceRef == 'myExecutor'
+            }
+    }
+
+}


[camel-k-runtime] 05/18: YAML: add support for Enrich EIP

Posted by lb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lburgazzoli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git

commit 1689b7c0b71bade8135e35953692856c8fe9ade1
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Sun Nov 24 15:16:26 2019 +0100

    YAML: add support for Enrich EIP
---
 .../k/loader/yaml/parser/EnrichStepParser.java     | 54 +++++++++++++++++++
 .../camel/k/loader/yaml/parser/EnrichTest.groovy   | 61 ++++++++++++++++++++++
 2 files changed, 115 insertions(+)

diff --git a/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/EnrichStepParser.java b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/EnrichStepParser.java
new file mode 100644
index 0000000..6108a8c
--- /dev/null
+++ b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/EnrichStepParser.java
@@ -0,0 +1,54 @@
+/*
+ * 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.k.loader.yaml.parser;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import org.apache.camel.Expression;
+import org.apache.camel.k.annotation.yaml.YAMLStepParser;
+import org.apache.camel.k.loader.yaml.model.Step;
+import org.apache.camel.model.EnrichDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.reifier.EnrichReifier;
+import org.apache.camel.reifier.ProcessorReifier;
+
+@YAMLStepParser("enrich")
+public class EnrichStepParser implements ProcessorStepParser {
+    static {
+        ProcessorReifier.registerReifier(Definition.class, EnrichReifier::new);
+    }
+
+    @Override
+    public ProcessorDefinition<?> toProcessor(Context context) {
+        return context.node(Definition.class);
+    }
+
+    public static final class Definition extends EnrichDefinition implements HasExpression, Step.Definition {
+        @JsonIgnore
+        public void setExpression(Expression expression) {
+            super.setExpression(expression);
+        }
+
+        public void setStrategyRef(String aggregationStrategyRef) {
+            super.setAggregationStrategyRef(aggregationStrategyRef);
+        }
+
+        public String getStrategyMethodName() {
+            return super.getAggregationStrategyRef();
+        }
+    }
+}
+
diff --git a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/EnrichTest.groovy b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/EnrichTest.groovy
new file mode 100644
index 0000000..c6f87b2
--- /dev/null
+++ b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/EnrichTest.groovy
@@ -0,0 +1,61 @@
+/*
+ * 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.k.loader.yaml.parser
+
+import org.apache.camel.k.loader.yaml.TestSupport
+import org.apache.camel.model.EnrichDefinition
+
+class EnrichTest extends TestSupport {
+
+    def "definition with expression"() {
+        given:
+            def stepContext = stepContext('''
+                 simple: "${body}"
+                 strategy-ref: "myStrategy"
+            ''')
+        when:
+            def processor = new EnrichStepParser().toProcessor(stepContext)
+        then:
+            with(processor, EnrichDefinition) {
+                with(expression) {
+                    language == 'simple'
+                    expression == '${body}'
+                }
+                aggregationStrategyRef == 'myStrategy'
+            }
+    }
+
+    def "definition with expression block"() {
+        given:
+            def stepContext = stepContext('''
+                 expression:
+                   simple: "${body}"
+                 strategy-ref: "myStrategy"
+            ''')
+        when:
+            def processor = new EnrichStepParser().toProcessor(stepContext)
+        then:
+            with(processor, EnrichDefinition) {
+                with(expression) {
+                    language == 'simple'
+                    expression == '${body}'
+                }
+                aggregationStrategyRef == 'myStrategy'
+            }
+    }
+
+}


[camel-k-runtime] 18/18: YAML: add support for WireTap EIP

Posted by lb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lburgazzoli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git

commit 4a9ebf38c4940d0e4a753577244d120a7429a54e
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Sun Nov 24 23:07:56 2019 +0100

    YAML: add support for WireTap EIP
---
 .../k/loader/yaml/parser/WireTapStepParser.java    | 110 +++++++++++++++++++++
 .../camel/k/loader/yaml/parser/WireTapTest.groovy  |  60 +++++++++++
 2 files changed, 170 insertions(+)

diff --git a/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/WireTapStepParser.java b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/WireTapStepParser.java
new file mode 100644
index 0000000..b9aa194
--- /dev/null
+++ b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/WireTapStepParser.java
@@ -0,0 +1,110 @@
+/*
+ * 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.k.loader.yaml.parser;
+
+import java.io.UnsupportedEncodingException;
+import java.net.URISyntaxException;
+import java.util.List;
+import java.util.Map;
+
+import com.fasterxml.jackson.annotation.JsonIgnore;
+import org.apache.camel.k.annotation.yaml.YAMLStepParser;
+import org.apache.camel.k.loader.yaml.model.Step;
+import org.apache.camel.model.ExpressionSubElementDefinition;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.SetHeaderDefinition;
+import org.apache.camel.model.ToDynamicDefinition;
+import org.apache.camel.model.WireTapDefinition;
+import org.apache.camel.model.language.ExpressionDefinition;
+import org.apache.camel.reifier.ProcessorReifier;
+import org.apache.camel.reifier.WireTapReifier;
+import org.apache.camel.util.ObjectHelper;
+import org.apache.camel.util.URISupport;
+
+@YAMLStepParser("wiretap")
+public class WireTapStepParser implements ProcessorStepParser {
+    static {
+        ProcessorReifier.registerReifier(Definition.class, WireTapReifier::new);
+    }
+
+    @Override
+    public ProcessorDefinition<?> toProcessor(Context context) {
+        Definition definition = context.node(Definition.class);
+
+        WireTapDefinition answer = new WireTapDefinition();
+        ObjectHelper.ifNotEmpty(definition.processorRef, answer::setNewExchangeProcessorRef);
+        ObjectHelper.ifNotEmpty(definition.executorServiceRef, answer::setExecutorServiceRef);
+        ObjectHelper.ifNotEmpty(definition.onPrepareRef, answer::onPrepareRef);
+        ObjectHelper.ifNotEmpty(definition.copy, answer::setCopy);
+        ObjectHelper.ifNotEmpty(definition.dynamicUri, answer::setDynamicUri);
+
+        if (definition.newExchange != null) {
+            answer.setNewExchangeExpression(definition.newExchange);
+
+            if (definition.newExchange.headers != null) {
+                answer.setHeaders(definition.newExchange.headers);
+            }
+        }
+
+        answer.setUri(definition.getEndpointUri());
+
+        return answer;
+    }
+
+    public static final class Definition extends ToDynamicDefinition implements Step.Definition {
+        public String processorRef;
+        public String executorServiceRef;
+        public String onPrepareRef;
+        public Boolean copy;
+        public Boolean dynamicUri;
+        public NewExchangeDefinition newExchange;
+        public Map<String, Object> parameters;
+
+        @JsonIgnore
+        public String getEndpointUri() {
+            String answer = getUri();
+
+            if (parameters != null) {
+                try {
+                    answer = URISupport.appendParametersToURI(answer, parameters);
+                } catch (URISyntaxException | UnsupportedEncodingException e) {
+                    throw new RuntimeException(e);
+                }
+            }
+
+            return answer;
+        }
+    }
+
+    public static final class NewExchangeDefinition extends ExpressionSubElementDefinition implements HasExpression {
+        public List<HeaderDefinition> headers;
+
+        @Override
+        public void setExpression(ExpressionDefinition expressionDefinition) {
+            super.setExpressionType(expressionDefinition);
+        }
+
+        @Override
+        public ExpressionDefinition getExpression() {
+            return super.getExpressionType();
+        }
+    }
+
+    public static final class HeaderDefinition extends SetHeaderDefinition implements HasExpression, Step.Definition {
+    }
+}
+
diff --git a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/WireTapTest.groovy b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/WireTapTest.groovy
new file mode 100644
index 0000000..d08429d
--- /dev/null
+++ b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/WireTapTest.groovy
@@ -0,0 +1,60 @@
+/*
+ * 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.k.loader.yaml.parser
+
+import org.apache.camel.k.loader.yaml.TestSupport
+import org.apache.camel.model.WireTapDefinition
+import org.apache.camel.model.language.ConstantExpression
+import org.apache.camel.model.language.SimpleExpression
+
+class WireTapTest extends TestSupport {
+
+    def "definition"() {
+        given:
+            def stepContext = stepContext('''
+                 uri: "direct:wt"
+                 new-exchange:
+                     simple: "${body}"
+                     headers:
+                         - name: "Header_1"
+                           simple: "${header.MyHeader1}"
+                         - name: "Header_2"
+                           constant: "test"                 
+                     body:
+            ''')
+        when:
+            def processor = new WireTapStepParser().toProcessor(stepContext)
+        then:
+            with (processor, WireTapDefinition) {
+                with (newExchangeExpression?.expression, SimpleExpression) {
+                    language == 'simple'
+                    expression == '${body}'
+                }
+
+                headers?.size() == 2
+
+                with (headers[0].expression, SimpleExpression) {
+                    language == 'simple'
+                    expression == '${header.MyHeader1}'
+                }
+                with (headers[1].expression, ConstantExpression) {
+                    language == 'constant'
+                    expression == 'test'
+                }
+            }
+    }
+}


[camel-k-runtime] 12/18: YAML: add support for Sample EIP

Posted by lb...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

lburgazzoli pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/camel-k-runtime.git

commit 9c89eae58faec2a1689eba41d38d9c1d084b326b
Author: lburgazzoli <lb...@gmail.com>
AuthorDate: Sun Nov 24 17:07:03 2019 +0100

    YAML: add support for Sample EIP
---
 .../k/loader/yaml/parser/SampleStepParser.java     | 49 ++++++++++++++++++++++
 .../camel/k/loader/yaml/parser/SampleTest.groovy   | 36 ++++++++++++++++
 2 files changed, 85 insertions(+)

diff --git a/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/SampleStepParser.java b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/SampleStepParser.java
new file mode 100644
index 0000000..a65c355
--- /dev/null
+++ b/camel-k-loader-yaml/src/main/java/org/apache/camel/k/loader/yaml/parser/SampleStepParser.java
@@ -0,0 +1,49 @@
+/*
+ * 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.k.loader.yaml.parser;
+
+import java.util.List;
+
+import org.apache.camel.k.annotation.yaml.YAMLStepParser;
+import org.apache.camel.k.loader.yaml.model.Step;
+import org.apache.camel.model.ProcessorDefinition;
+import org.apache.camel.model.SamplingDefinition;
+import org.apache.camel.reifier.ProcessorReifier;
+import org.apache.camel.reifier.SamplingReifier;
+
+@YAMLStepParser("sample")
+public class SampleStepParser implements ProcessorStepParser {
+    static {
+        ProcessorReifier.registerReifier(Definition.class, SamplingReifier::new);
+    }
+
+    @Override
+    public ProcessorDefinition<?> toProcessor(Context context) {
+        Definition definition = context.node(Definition.class);
+
+        return StepParserSupport.convertSteps(
+            context,
+            definition,
+            definition.steps
+        );
+    }
+
+    public static final class Definition extends SamplingDefinition implements Step.Definition {
+        public List<Step> steps;
+    }
+}
+
diff --git a/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/SampleTest.groovy b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/SampleTest.groovy
new file mode 100644
index 0000000..c2036e2
--- /dev/null
+++ b/camel-k-loader-yaml/src/test/groovy/org/apache/camel/k/loader/yaml/parser/SampleTest.groovy
@@ -0,0 +1,36 @@
+/*
+ * 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.k.loader.yaml.parser
+
+import org.apache.camel.k.loader.yaml.TestSupport
+import org.apache.camel.model.SamplingDefinition
+
+class SampleTest extends TestSupport {
+
+    def "definition"() {
+        given:
+            def stepContext = stepContext('''
+                 message-frequency: "5"
+            ''')
+        when:
+            def processor = new SampleStepParser().toProcessor(stepContext)
+        then:
+            with (processor, SamplingDefinition) {
+                messageFrequency == 5
+            }
+    }
+}