You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by da...@apache.org on 2023/05/17 05:44:33 UTC

[camel] branch camel-3.x updated: CAMEL-19361: camel-yaml-dsl/camel-jbang - Running KameletBinding should read properties/environment traits

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

davsclaus pushed a commit to branch camel-3.x
in repository https://gitbox.apache.org/repos/asf/camel.git


The following commit(s) were added to refs/heads/camel-3.x by this push:
     new 017c5482217 CAMEL-19361: camel-yaml-dsl/camel-jbang - Running KameletBinding should read properties/environment traits
017c5482217 is described below

commit 017c54822170b0a65026c04734980e1b1a111778
Author: Claus Ibsen <cl...@gmail.com>
AuthorDate: Wed May 17 07:44:02 2023 +0200

    CAMEL-19361: camel-yaml-dsl/camel-jbang - Running KameletBinding should read properties/environment traits
---
 .../camel/dsl/yaml/YamlRoutesBuilderLoader.java    | 310 ++++++++++++++-------
 .../camel/dsl/yaml/KameletBindingLoaderTest.groovy |  32 +++
 2 files changed, 236 insertions(+), 106 deletions(-)

diff --git a/dsl/camel-yaml-dsl/camel-yaml-dsl/src/main/java/org/apache/camel/dsl/yaml/YamlRoutesBuilderLoader.java b/dsl/camel-yaml-dsl/camel-yaml-dsl/src/main/java/org/apache/camel/dsl/yaml/YamlRoutesBuilderLoader.java
index 4137d1888c2..720fdeb5616 100644
--- a/dsl/camel-yaml-dsl/camel-yaml-dsl/src/main/java/org/apache/camel/dsl/yaml/YamlRoutesBuilderLoader.java
+++ b/dsl/camel-yaml-dsl/camel-yaml-dsl/src/main/java/org/apache/camel/dsl/yaml/YamlRoutesBuilderLoader.java
@@ -66,6 +66,7 @@ import org.apache.camel.spi.annotations.RoutesLoader;
 import org.apache.camel.support.ObjectHelper;
 import org.apache.camel.support.PropertyBindingSupport;
 import org.apache.camel.util.FileUtil;
+import org.apache.camel.util.StringQuoteHelper;
 import org.apache.camel.util.URISupport;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -305,9 +306,9 @@ public class YamlRoutesBuilderLoader extends YamlRoutesBuilderLoaderSupport {
                     anyTupleMatches(mn.getValue(), "kind", "KameletBinding");
             if (integration) {
                 target = preConfigureIntegration(root, ctx, target, preParse);
-            } else if (binding && !preParse) {
+            } else if (binding) {
                 // kamelet binding does not take part in pre-parse phase
-                target = preConfigureKameletBinding(root, ctx, target);
+                target = preConfigureKameletBinding(root, ctx, target, preParse);
             }
         }
 
@@ -538,6 +539,84 @@ public class YamlRoutesBuilderLoader extends YamlRoutesBuilderLoaderSupport {
         return answer;
     }
 
+    private List<CamelContextCustomizer> preConfigureTraitConfigurationBinding(Resource resource, Map<String, Object> map) {
+        List<CamelContextCustomizer> answer = new ArrayList<>();
+
+        if (map == null || map.isEmpty()) {
+            return null;
+        }
+        Object value = map.get("trait.camel.apache.org/camel.properties");
+        if (value == null || value.toString().isEmpty()) {
+            return null;
+        }
+        final String[] properties = StringQuoteHelper.splitSafeQuote(value.toString(), ',', true);
+
+        answer.add(new CamelContextCustomizer() {
+            @Override
+            public void configure(CamelContext camelContext) {
+                try {
+                    org.apache.camel.component.properties.PropertiesComponent pc
+                            = (org.apache.camel.component.properties.PropertiesComponent) camelContext.getPropertiesComponent();
+                    IntegrationConfigurationPropertiesSource ps
+                            = (IntegrationConfigurationPropertiesSource) pc
+                                    .getPropertiesSource("binding-trait-configuration");
+                    if (ps == null) {
+                        ps = new IntegrationConfigurationPropertiesSource(
+                                pc, new PropertiesLocation(resource.getLocation()), "binding-trait-configuration");
+                        pc.addPropertiesSource(ps);
+                    }
+
+                    for (String line : properties) {
+                        ps.parseConfigurationValue(line);
+                    }
+                } catch (Exception e) {
+                    throw new RuntimeCamelException("Error adding properties from metadata/annotations/", e);
+                }
+            }
+        });
+
+        return answer;
+    }
+
+    private List<CamelContextCustomizer> preConfigureTraitEnvironmentBinding(Resource resource, Map<String, Object> map) {
+        List<CamelContextCustomizer> answer = new ArrayList<>();
+
+        if (map == null || map.isEmpty()) {
+            return null;
+        }
+        Object value = map.get("trait.camel.apache.org/environment.vars");
+        if (value == null || value.toString().isEmpty()) {
+            return null;
+        }
+        final String[] properties = StringQuoteHelper.splitSafeQuote(value.toString(), ',', true);
+
+        answer.add(new CamelContextCustomizer() {
+            @Override
+            public void configure(CamelContext camelContext) {
+                try {
+                    org.apache.camel.component.properties.PropertiesComponent pc
+                            = (org.apache.camel.component.properties.PropertiesComponent) camelContext.getPropertiesComponent();
+                    IntegrationConfigurationPropertiesSource ps
+                            = (IntegrationConfigurationPropertiesSource) pc
+                                    .getPropertiesSource("environment-trait-configuration");
+                    if (ps == null) {
+                        ps = new IntegrationConfigurationPropertiesSource(
+                                pc, new PropertiesLocation(resource.getLocation()), "environment-trait-configuration");
+                        pc.addPropertiesSource(ps);
+                    }
+
+                    for (String line : properties) {
+                        ps.parseConfigurationValue(line);
+                    }
+                } catch (Exception e) {
+                    throw new RuntimeCamelException("Error adding properties from metadata/annotations/", e);
+                }
+            }
+        });
+
+        return answer;
+    }
+
     private List<CamelContextCustomizer> preConfigureSources(Node node) {
         List<CamelContextCustomizer> answer = new ArrayList<>();
 
@@ -571,132 +650,151 @@ public class YamlRoutesBuilderLoader extends YamlRoutesBuilderLoaderSupport {
     /**
      * Camel K Kamelet Binding file
      */
-    private Object preConfigureKameletBinding(Node root, YamlDeserializationContext ctx, Object target) {
-        // start with a route
-        final RouteDefinition route = new RouteDefinition();
-        String routeId = asText(nodeAt(root, "/metadata/name"));
-        if (routeId != null) {
-            route.routeId(routeId);
+    private Object preConfigureKameletBinding(Node root, YamlDeserializationContext ctx, Object target, boolean preParse) {
+        // when in pre-parse phase then we only want to gather spec/dependencies,spec/configuration,spec/traits
+
+        List<Object> answer = new ArrayList<>();
+
+        MappingNode ann = asMappingNode(nodeAt(root, "/metadata/annotations"));
+        Map<String, Object> params = asMap(ann);
+        if (params != null) {
+            var list = preConfigureTraitConfigurationBinding(ctx.getResource(), params);
+            if (list != null) {
+                answer.addAll(list);
+            }
+            list = preConfigureTraitEnvironmentBinding(ctx.getResource(), params);
+            if (list != null) {
+                answer.addAll(list);
+            }
         }
 
-        // kamelet binding is a bit more complex, so grab the source and sink
-        // and map those to Camel route definitions
-        MappingNode source = asMappingNode(nodeAt(root, "/spec/source"));
-        MappingNode sink = asMappingNode(nodeAt(root, "/spec/sink"));
-        if (source != null && sink != null) {
-            int line = -1;
-            if (source.getStartMark().isPresent()) {
-                line = source.getStartMark().get().getLine();
+        if (!preParse) {
+            // start with a route
+            final RouteDefinition route = new RouteDefinition();
+            String routeId = asText(nodeAt(root, "/metadata/name"));
+            if (routeId != null) {
+                route.routeId(routeId);
             }
 
-            // source at the beginning (mandatory)
-            String uri = extractCamelEndpointUri(source);
-            route.from(uri);
+            // kamelet binding is a bit more complex, so grab the source and sink
+            // and map those to Camel route definitions
+            MappingNode source = asMappingNode(nodeAt(root, "/spec/source"));
+            MappingNode sink = asMappingNode(nodeAt(root, "/spec/sink"));
+            if (source != null && sink != null) {
+                int line = -1;
+                if (source.getStartMark().isPresent()) {
+                    line = source.getStartMark().get().getLine();
+                }
+
+                // source at the beginning (mandatory)
+                String uri = extractCamelEndpointUri(source);
+                route.from(uri);
 
-            // enrich model with line number
-            if (line != -1) {
-                route.getInput().setLineNumber(line);
-                if (ctx != null) {
-                    route.getInput().setLocation(ctx.getResource().getLocation());
+                // enrich model with line number
+                if (line != -1) {
+                    route.getInput().setLineNumber(line);
+                    if (ctx != null) {
+                        route.getInput().setLocation(ctx.getResource().getLocation());
+                    }
                 }
-            }
 
-            // steps in the middle (optional)
-            Node steps = nodeAt(root, "/spec/steps");
-            if (steps != null) {
-                SequenceNode sn = asSequenceNode(steps);
-                for (Node node : sn.getValue()) {
-                    MappingNode step = asMappingNode(node);
-                    uri = extractCamelEndpointUri(step);
-                    if (uri != null) {
-                        line = -1;
-                        if (node.getStartMark().isPresent()) {
-                            line = node.getStartMark().get().getLine();
-                        }
+                // steps in the middle (optional)
+                Node steps = nodeAt(root, "/spec/steps");
+                if (steps != null) {
+                    SequenceNode sn = asSequenceNode(steps);
+                    for (Node node : sn.getValue()) {
+                        MappingNode step = asMappingNode(node);
+                        uri = extractCamelEndpointUri(step);
+                        if (uri != null) {
+                            line = -1;
+                            if (node.getStartMark().isPresent()) {
+                                line = node.getStartMark().get().getLine();
+                            }
 
-                        ProcessorDefinition<?> out;
-                        // if kamelet then use kamelet eip instead of to
-                        boolean kamelet = uri.startsWith("kamelet:");
-                        if (kamelet) {
-                            uri = uri.substring(8);
-                            out = new KameletDefinition(uri);
-                        } else {
-                            out = new ToDefinition(uri);
-                        }
-                        route.addOutput(out);
-                        // enrich model with line number
-                        if (line != -1) {
-                            out.setLineNumber(line);
-                            if (ctx != null) {
-                                out.setLocation(ctx.getResource().getLocation());
+                            ProcessorDefinition<?> out;
+                            // if kamelet then use kamelet eip instead of to
+                            boolean kamelet = uri.startsWith("kamelet:");
+                            if (kamelet) {
+                                uri = uri.substring(8);
+                                out = new KameletDefinition(uri);
+                            } else {
+                                out = new ToDefinition(uri);
+                            }
+                            route.addOutput(out);
+                            // enrich model with line number
+                            if (line != -1) {
+                                out.setLineNumber(line);
+                                if (ctx != null) {
+                                    out.setLocation(ctx.getResource().getLocation());
+                                }
                             }
                         }
                     }
                 }
-            }
 
-            // sink is at the end (mandatory)
-            line = -1;
-            if (sink.getStartMark().isPresent()) {
-                line = sink.getStartMark().get().getLine();
-            }
-            uri = extractCamelEndpointUri(sink);
-            ToDefinition to = new ToDefinition(uri);
-            route.addOutput(to);
-
-            // enrich model with line number
-            if (line != -1) {
-                to.setLineNumber(line);
-                if (ctx != null) {
-                    to.setLocation(ctx.getResource().getLocation());
+                // sink is at the end (mandatory)
+                line = -1;
+                if (sink.getStartMark().isPresent()) {
+                    line = sink.getStartMark().get().getLine();
                 }
-            }
-
-            // is there any error handler?
-            MappingNode errorHandler = asMappingNode(nodeAt(root, "/spec/errorHandler"));
-            if (errorHandler != null) {
-                // there are 5 different error handlers, which one is it
-                NodeTuple nt = errorHandler.getValue().get(0);
-                String ehName = asText(nt.getKeyNode());
-
-                ErrorHandlerFactory ehf = null;
-                if ("sink".equals(ehName)) {
-                    // a sink is a dead letter queue
-                    DeadLetterChannelDefinition dlcd = new DeadLetterChannelDefinition();
-                    MappingNode endpoint = asMappingNode(nodeAt(nt.getValueNode(), "/endpoint"));
-                    String dlq = extractCamelEndpointUri(endpoint);
-                    dlcd.setDeadLetterUri(dlq);
-                    ehf = dlcd;
-                } else if ("log".equals(ehName)) {
-                    // log is the default error handler
-                    ehf = new DefaultErrorHandlerDefinition();
-                } else if ("none".equals(ehName)) {
-                    route.errorHandler(new NoErrorHandlerDefinition());
+                uri = extractCamelEndpointUri(sink);
+                ToDefinition to = new ToDefinition(uri);
+                route.addOutput(to);
+
+                // enrich model with line number
+                if (line != -1) {
+                    to.setLineNumber(line);
+                    if (ctx != null) {
+                        to.setLocation(ctx.getResource().getLocation());
+                    }
                 }
 
-                // some error handlers support additional parameters
-                if (ehf != null) {
-                    // properties that are general for all kind of error handlers
-                    MappingNode prop = asMappingNode(nodeAt(nt.getValueNode(), "/parameters"));
-                    Map<String, Object> params = asMap(prop);
-                    if (params != null) {
-                        PropertyBindingSupport.build()
-                                .withIgnoreCase(true)
-                                .withFluentBuilder(true)
-                                .withRemoveParameters(true)
-                                .withCamelContext(getCamelContext())
-                                .withTarget(ehf)
-                                .withProperties(params)
-                                .bind();
+                // is there any error handler?
+                MappingNode errorHandler = asMappingNode(nodeAt(root, "/spec/errorHandler"));
+                if (errorHandler != null) {
+                    // there are 5 different error handlers, which one is it
+                    NodeTuple nt = errorHandler.getValue().get(0);
+                    String ehName = asText(nt.getKeyNode());
+
+                    ErrorHandlerFactory ehf = null;
+                    if ("sink".equals(ehName)) {
+                        // a sink is a dead letter queue
+                        DeadLetterChannelDefinition dlcd = new DeadLetterChannelDefinition();
+                        MappingNode endpoint = asMappingNode(nodeAt(nt.getValueNode(), "/endpoint"));
+                        String dlq = extractCamelEndpointUri(endpoint);
+                        dlcd.setDeadLetterUri(dlq);
+                        ehf = dlcd;
+                    } else if ("log".equals(ehName)) {
+                        // log is the default error handler
+                        ehf = new DefaultErrorHandlerDefinition();
+                    } else if ("none".equals(ehName)) {
+                        route.errorHandler(new NoErrorHandlerDefinition());
+                    }
+
+                    // some error handlers support additional parameters
+                    if (ehf != null) {
+                        // properties that are general for all kind of error handlers
+                        MappingNode prop = asMappingNode(nodeAt(nt.getValueNode(), "/parameters"));
+                        params = asMap(prop);
+                        if (params != null) {
+                            PropertyBindingSupport.build()
+                                    .withIgnoreCase(true)
+                                    .withFluentBuilder(true)
+                                    .withRemoveParameters(true)
+                                    .withCamelContext(getCamelContext())
+                                    .withTarget(ehf)
+                                    .withProperties(params)
+                                    .bind();
+                        }
+                        route.errorHandler(ehf);
                     }
-                    route.errorHandler(ehf);
                 }
             }
 
-            target = route;
+            answer.add(route);
         }
 
-        return target;
+        return answer;
     }
 
     private String extractCamelEndpointUri(MappingNode node) {
diff --git a/dsl/camel-yaml-dsl/camel-yaml-dsl/src/test/groovy/org/apache/camel/dsl/yaml/KameletBindingLoaderTest.groovy b/dsl/camel-yaml-dsl/camel-yaml-dsl/src/test/groovy/org/apache/camel/dsl/yaml/KameletBindingLoaderTest.groovy
index f30ec14cb1c..8a339e14ac9 100644
--- a/dsl/camel-yaml-dsl/camel-yaml-dsl/src/test/groovy/org/apache/camel/dsl/yaml/KameletBindingLoaderTest.groovy
+++ b/dsl/camel-yaml-dsl/camel-yaml-dsl/src/test/groovy/org/apache/camel/dsl/yaml/KameletBindingLoaderTest.groovy
@@ -589,4 +589,36 @@ class KameletBindingLoaderTest extends YamlTestSupport {
         }
     }
 
+    def "kamelet binding with trait properties"() {
+        when:
+        loadBindings('''
+                apiVersion: camel.apache.org/v1alpha1
+                kind: KameletBinding
+                metadata:
+                  name: timer-event-source
+                  annotations:
+                    trait.camel.apache.org/camel.properties: "foo=howdy,bar=123"                  
+                    trait.camel.apache.org/environment.vars: "MY_ENV=cheese"                  
+                spec:
+                  source:
+                    ref:
+                      kind: Kamelet
+                      apiVersion: camel.apache.org/v1
+                      name: timer-source
+                    properties:
+                      message: "Hello world!"
+                  sink:
+                    ref:
+                      kind: Kamelet
+                      apiVersion: camel.apache.org/v1
+                      name: log-sink
+            ''')
+        then:
+        context.routeDefinitions.size() == 3
+
+        context.resolvePropertyPlaceholders("{{foo}}") == "howdy"
+        context.resolvePropertyPlaceholders("{{bar}}") == "123"
+        context.resolvePropertyPlaceholders("{{MY_ENV}}") == "cheese"
+    }
+
 }