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 2019/08/13 06:25:05 UTC

[camel] 03/09: CAMEL-13850: Optimize model classes to provide changeable properties that support property placeholders to avoid reflection. Work in progress.

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

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

commit 3ed317216d9de5b1acdda1c81504638c8371e0c2
Author: Claus Ibsen <cl...@gmail.com>
AuthorDate: Mon Aug 12 18:00:21 2019 +0200

    CAMEL-13850: Optimize model classes to provide changeable properties that support property placeholders to avoid reflection. Work in progress.
---
 .../model/DefinitionPropertiesProviderHelper.java  |  18 ++-
 ...DefinitionPropertyPlaceholderConfigurable.java} |  14 +-
 .../model/FromDefinitionPropertiesProvider.java    |  50 -------
 .../model/LogDefinitionPropertiesProvider.java     |  56 --------
 .../camel/model/OptionalIdentifiedDefinition.java  |   2 +-
 .../camel/model/ProcessorDefinitionHelper.java     |  36 +----
 .../java/org/apache/camel/model/ToDefinition.java  |   3 +-
 .../model/ToDefinitionPropertiesProvider.java      |  50 -------
 .../apt/CoreEipAnnotationProcessorHelper.java      | 158 +++++++++++++++++++++
 9 files changed, 182 insertions(+), 205 deletions(-)

diff --git a/core/camel-core/src/main/java/org/apache/camel/model/DefinitionPropertiesProviderHelper.java b/core/camel-core/src/main/java/org/apache/camel/model/DefinitionPropertiesProviderHelper.java
index 7da3ffd..86cb92e 100644
--- a/core/camel-core/src/main/java/org/apache/camel/model/DefinitionPropertiesProviderHelper.java
+++ b/core/camel-core/src/main/java/org/apache/camel/model/DefinitionPropertiesProviderHelper.java
@@ -21,19 +21,23 @@ import java.util.Map;
 import java.util.Optional;
 import java.util.function.Function;
 
+import org.apache.camel.model.placeholder.FromDefinitionPropertyPlaceholderProvider;
+import org.apache.camel.model.placeholder.LogDefinitionPropertyPlaceholderProvider;
+import org.apache.camel.model.placeholder.ToDefinitionPropertyPlaceholderProvider;
+
 public class DefinitionPropertiesProviderHelper {
 
-    private static final Map<Class, Function<Object, PropertyPlaceholderAware>> MAP;
+    private static final Map<Class, Function<Object, DefinitionPropertyPlaceholderConfigurable>> MAP;
     static {
-        Map<Class, Function<Object, PropertyPlaceholderAware>> map = new HashMap<>();
-        map.put(FromDefinition.class, FromDefinitionPropertiesProvider::new);
-        map.put(LogDefinition.class, LogDefinitionPropertiesProvider::new);
-        map.put(ToDefinition.class, ToDefinitionPropertiesProvider::new);
+        Map<Class, Function<Object, DefinitionPropertyPlaceholderConfigurable>> map = new HashMap<>();
+        map.put(FromDefinition.class, FromDefinitionPropertyPlaceholderProvider::new);
+        map.put(LogDefinition.class, LogDefinitionPropertyPlaceholderProvider::new);
+        map.put(ToDefinition.class, ToDefinitionPropertyPlaceholderProvider::new);
         MAP = map;
     }
 
-    public static Optional<PropertyPlaceholderAware> provider(Object definition) {
-        Function<Object, PropertyPlaceholderAware> func = MAP.get(definition.getClass());
+    public static Optional<DefinitionPropertyPlaceholderConfigurable> provider(Object definition) {
+        Function<Object, DefinitionPropertyPlaceholderConfigurable> func = MAP.get(definition.getClass());
         if (func != null) {
             return Optional.of(func.apply(definition));
         }
diff --git a/core/camel-core/src/main/java/org/apache/camel/model/PropertyPlaceholderAware.java b/core/camel-core/src/main/java/org/apache/camel/model/DefinitionPropertyPlaceholderConfigurable.java
similarity index 75%
rename from core/camel-core/src/main/java/org/apache/camel/model/PropertyPlaceholderAware.java
rename to core/camel-core/src/main/java/org/apache/camel/model/DefinitionPropertyPlaceholderConfigurable.java
index c4fb8b7..9c82f6b 100644
--- a/core/camel-core/src/main/java/org/apache/camel/model/PropertyPlaceholderAware.java
+++ b/core/camel-core/src/main/java/org/apache/camel/model/DefinitionPropertyPlaceholderConfigurable.java
@@ -22,24 +22,28 @@ import java.util.function.Supplier;
 
 import org.apache.camel.CamelContext;
 
-// TODO: Rename interface
-public interface PropertyPlaceholderAware {
+/**
+ * To be used for configuring property placeholder options on the EIP models.
+ */
+public interface DefinitionPropertyPlaceholderConfigurable {
 
     /**
      * Gets the options on the model definition which supports property placeholders and can be resolved.
+     * This will be all the string based options.
      *
      * @return key/values of options
      */
     default Map<String, Supplier<String>> getReadPropertyPlaceholderOptions(CamelContext camelContext) {
-        PropertyPlaceholderAware aware = DefinitionPropertiesProviderHelper.provider(this).orElse(null);
+        DefinitionPropertyPlaceholderConfigurable aware = DefinitionPropertiesProviderHelper.provider(this).orElse(null);
         return aware != null ? aware.getReadPropertyPlaceholderOptions(camelContext) : null;
     }
 
     /**
-     * To update an existing property using the function with the ket/value and returning the changed value
+     * To update an existing property using the function with the key/value and returning the changed value
+     * This will be all the string based options.
      */
     default Map<String, Consumer<String>> getWritePropertyPlaceholderOptions(CamelContext camelContext) {
-        PropertyPlaceholderAware aware = DefinitionPropertiesProviderHelper.provider(this).orElse(null);
+        DefinitionPropertyPlaceholderConfigurable aware = DefinitionPropertiesProviderHelper.provider(this).orElse(null);
         return aware != null ? aware.getWritePropertyPlaceholderOptions(camelContext) : null;
     }
 
diff --git a/core/camel-core/src/main/java/org/apache/camel/model/FromDefinitionPropertiesProvider.java b/core/camel-core/src/main/java/org/apache/camel/model/FromDefinitionPropertiesProvider.java
deleted file mode 100644
index 5953f84..0000000
--- a/core/camel-core/src/main/java/org/apache/camel/model/FromDefinitionPropertiesProvider.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/**
- * 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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.camel.model;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.function.Consumer;
-import java.util.function.Supplier;
-
-import org.apache.camel.CamelContext;
-
-public class FromDefinitionPropertiesProvider implements PropertyPlaceholderAware {
-
-    private final FromDefinition definition;
-    private final Map<String, Supplier<String>> readPlaceholders = new HashMap<>();
-    private final Map<String, Consumer<String>> writePlaceholders = new HashMap<>();
-
-    public FromDefinitionPropertiesProvider(Object obj) {
-        this.definition = (FromDefinition) obj;
-
-        readPlaceholders.put("id", definition::getId);
-        readPlaceholders.put("uri", definition::getUri);
-        writePlaceholders.put("id", definition::setId);
-        writePlaceholders.put("uri", definition::setUri);
-    }
-
-    @Override
-    public Map<String, Supplier<String>> getReadPropertyPlaceholderOptions(CamelContext camelContext) {
-        return readPlaceholders;
-    }
-
-    @Override
-    public Map<String, Consumer<String>> getWritePropertyPlaceholderOptions(CamelContext camelContext) {
-        return writePlaceholders;
-    }
-}
diff --git a/core/camel-core/src/main/java/org/apache/camel/model/LogDefinitionPropertiesProvider.java b/core/camel-core/src/main/java/org/apache/camel/model/LogDefinitionPropertiesProvider.java
deleted file mode 100644
index a60ffa8..0000000
--- a/core/camel-core/src/main/java/org/apache/camel/model/LogDefinitionPropertiesProvider.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/**
- * 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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.camel.model;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.function.Consumer;
-import java.util.function.Supplier;
-
-import org.apache.camel.CamelContext;
-
-public class LogDefinitionPropertiesProvider implements PropertyPlaceholderAware {
-
-    private final LogDefinition definition;
-    private final Map<String, Supplier<String>> readPlaceholders = new HashMap<>();
-    private final Map<String, Consumer<String>> writePlaceholders = new HashMap<>();
-
-    public LogDefinitionPropertiesProvider(Object obj) {
-        this.definition = (LogDefinition) obj;
-
-        readPlaceholders.put("id", definition::getId);
-        readPlaceholders.put("message", definition::getMessage);
-        readPlaceholders.put("logName", definition::getLogName);
-        readPlaceholders.put("marker", definition::getMarker);
-        readPlaceholders.put("loggerRef", definition::getLoggerRef);
-        writePlaceholders.put("id", definition::setId);
-        writePlaceholders.put("message", definition::setMessage);
-        writePlaceholders.put("logName", definition::setLogName);
-        writePlaceholders.put("marker", definition::setMarker);
-        writePlaceholders.put("loggerRef", definition::setLoggerRef);
-    }
-
-    @Override
-    public Map<String, Supplier<String>> getReadPropertyPlaceholderOptions(CamelContext camelContext) {
-        return readPlaceholders;
-    }
-
-    @Override
-    public Map<String, Consumer<String>> getWritePropertyPlaceholderOptions(CamelContext camelContext) {
-        return writePlaceholders;
-    }
-}
diff --git a/core/camel-core/src/main/java/org/apache/camel/model/OptionalIdentifiedDefinition.java b/core/camel-core/src/main/java/org/apache/camel/model/OptionalIdentifiedDefinition.java
index 5f520f5..9ca6bff 100644
--- a/core/camel-core/src/main/java/org/apache/camel/model/OptionalIdentifiedDefinition.java
+++ b/core/camel-core/src/main/java/org/apache/camel/model/OptionalIdentifiedDefinition.java
@@ -31,7 +31,7 @@ import org.apache.camel.spi.NodeIdFactory;
 @XmlType(name = "optionalIdentifiedDefinition")
 @XmlAccessorType(XmlAccessType.PROPERTY)
 // must use XmlAccessType.PROPERTY which is required by camel-spring / camel-blueprint for their namespace parsers
-public abstract class OptionalIdentifiedDefinition<T extends OptionalIdentifiedDefinition<T>> implements NamedNode, PropertyPlaceholderAware {
+public abstract class OptionalIdentifiedDefinition<T extends OptionalIdentifiedDefinition<T>> implements NamedNode, DefinitionPropertyPlaceholderConfigurable {
 
     private String id;
     private Boolean customId;
diff --git a/core/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinitionHelper.java b/core/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinitionHelper.java
index ff809f6..b57f51a 100644
--- a/core/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinitionHelper.java
+++ b/core/camel-core/src/main/java/org/apache/camel/model/ProcessorDefinitionHelper.java
@@ -28,18 +28,12 @@ import java.util.concurrent.ExecutorService;
 import java.util.concurrent.ScheduledExecutorService;
 import java.util.function.Consumer;
 import java.util.function.Supplier;
-import javax.xml.namespace.QName;
 
 import org.apache.camel.CamelContext;
-import org.apache.camel.Exchange;
 import org.apache.camel.NamedNode;
 import org.apache.camel.spi.ExecutorServiceManager;
-import org.apache.camel.spi.PropertiesComponent;
 import org.apache.camel.spi.RouteContext;
-import org.apache.camel.support.IntrospectionSupport;
-import org.apache.camel.support.PropertyBindingSupport;
 import org.apache.camel.util.ObjectHelper;
-import org.apache.camel.util.StringHelper;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
@@ -655,32 +649,6 @@ public final class ProcessorDefinitionHelper {
         });
     }
 
-    @Deprecated
-    private static void addRestoreAction(final CamelContext context, final Object target, final Map<String, Object> properties) {
-        if (properties.isEmpty()) {
-            return;
-        }
-
-        RestoreAction restoreAction = CURRENT_RESTORE_ACTION.get();
-        if (restoreAction == null) {
-            return;
-        }
-
-        restoreAction.actions.add(new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    // do not use property placeholders as we want to preserve the text as-is when we restore
-                    PropertyBindingSupport.build()
-                            .withPlaceholder(false).withNesting(false).withReference(false)
-                            .bind(context, target, properties);
-                } catch (Exception e) {
-                    LOG.warn("Cannot restore definition properties. This exception is ignored.", e);
-                }
-            }
-        });
-    }
-
     public static void addPropertyPlaceholdersChangeRevertAction(Runnable action) {
         RestoreAction restoreAction = CURRENT_RESTORE_ACTION.get();
         if (restoreAction == null) {
@@ -706,11 +674,11 @@ public final class ProcessorDefinitionHelper {
         LOG.trace("Resolving property placeholders for: {}", definition);
 
         // only do this for models that supports property placeholders
-        if (!(definition instanceof PropertyPlaceholderAware)) {
+        if (!(definition instanceof DefinitionPropertyPlaceholderConfigurable)) {
             return;
         }
 
-        PropertyPlaceholderAware ppa = (PropertyPlaceholderAware) definition;
+        DefinitionPropertyPlaceholderConfigurable ppa = (DefinitionPropertyPlaceholderConfigurable) definition;
 
         // find all getter/setter which we can use for property placeholders
         Map<String, String> changedProperties = new HashMap<>();
diff --git a/core/camel-core/src/main/java/org/apache/camel/model/ToDefinition.java b/core/camel-core/src/main/java/org/apache/camel/model/ToDefinition.java
index 3a0ba35..10b61d8 100644
--- a/core/camel-core/src/main/java/org/apache/camel/model/ToDefinition.java
+++ b/core/camel-core/src/main/java/org/apache/camel/model/ToDefinition.java
@@ -18,7 +18,6 @@ package org.apache.camel.model;
 
 import java.util.HashMap;
 import java.util.Map;
-import java.util.function.BiFunction;
 import java.util.function.Consumer;
 import java.util.function.Supplier;
 import javax.xml.bind.annotation.XmlAccessType;
@@ -39,7 +38,7 @@ import org.apache.camel.spi.PropertiesComponent;
 @Metadata(label = "eip,endpoint,routing")
 @XmlRootElement(name = "to")
 @XmlAccessorType(XmlAccessType.FIELD)
-public class ToDefinition extends SendDefinition<ToDefinition> implements PropertyPlaceholderAware {
+public class ToDefinition extends SendDefinition<ToDefinition> implements DefinitionPropertyPlaceholderConfigurable {
     @XmlAttribute
     private ExchangePattern pattern;
 
diff --git a/core/camel-core/src/main/java/org/apache/camel/model/ToDefinitionPropertiesProvider.java b/core/camel-core/src/main/java/org/apache/camel/model/ToDefinitionPropertiesProvider.java
deleted file mode 100644
index e0e64a5..0000000
--- a/core/camel-core/src/main/java/org/apache/camel/model/ToDefinitionPropertiesProvider.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/**
- * 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
- * <p>
- * http://www.apache.org/licenses/LICENSE-2.0
- * <p>
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package org.apache.camel.model;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.function.Consumer;
-import java.util.function.Supplier;
-
-import org.apache.camel.CamelContext;
-
-public class ToDefinitionPropertiesProvider implements PropertyPlaceholderAware {
-
-    private final ToDefinition definition;
-    private final Map<String, Supplier<String>> readPlaceholders = new HashMap<>();
-    private final Map<String, Consumer<String>> writePlaceholders = new HashMap<>();
-
-    public ToDefinitionPropertiesProvider(Object obj) {
-        this.definition = (ToDefinition) obj;
-
-        readPlaceholders.put("id", definition::getId);
-        readPlaceholders.put("uri", definition::getUri);
-        writePlaceholders.put("id", definition::setId);
-        writePlaceholders.put("uri", definition::setUri);
-    }
-
-    @Override
-    public Map<String, Supplier<String>> getReadPropertyPlaceholderOptions(CamelContext camelContext) {
-        return readPlaceholders;
-    }
-
-    @Override
-    public Map<String, Consumer<String>> getWritePropertyPlaceholderOptions(CamelContext camelContext) {
-        return writePlaceholders;
-    }
-}
diff --git a/tooling/apt/src/main/java/org/apache/camel/tools/apt/CoreEipAnnotationProcessorHelper.java b/tooling/apt/src/main/java/org/apache/camel/tools/apt/CoreEipAnnotationProcessorHelper.java
index df08f15..edb48d7 100644
--- a/tooling/apt/src/main/java/org/apache/camel/tools/apt/CoreEipAnnotationProcessorHelper.java
+++ b/tooling/apt/src/main/java/org/apache/camel/tools/apt/CoreEipAnnotationProcessorHelper.java
@@ -17,12 +17,14 @@
 package org.apache.camel.tools.apt;
 
 import java.io.PrintWriter;
+import java.io.Writer;
 import java.util.Comparator;
 import java.util.HashSet;
 import java.util.LinkedHashSet;
 import java.util.List;
 import java.util.Set;
 import java.util.TreeSet;
+import java.util.function.Consumer;
 import java.util.stream.Collectors;
 
 import javax.annotation.processing.ProcessingEnvironment;
@@ -34,6 +36,8 @@ import javax.lang.model.element.VariableElement;
 import javax.lang.model.type.TypeMirror;
 import javax.lang.model.util.ElementFilter;
 import javax.lang.model.util.Elements;
+import javax.tools.Diagnostic;
+import javax.tools.JavaFileObject;
 import javax.xml.bind.annotation.XmlAttribute;
 import javax.xml.bind.annotation.XmlElement;
 import javax.xml.bind.annotation.XmlElementRef;
@@ -42,11 +46,15 @@ import javax.xml.bind.annotation.XmlRootElement;
 import javax.xml.bind.annotation.XmlType;
 import javax.xml.bind.annotation.XmlValue;
 
+import jdk.nashorn.internal.ir.debug.ClassHistogramElement;
 import org.apache.camel.spi.AsPredicate;
 import org.apache.camel.spi.Metadata;
+import org.apache.camel.tools.apt.helper.IOHelper;
 import org.apache.camel.tools.apt.helper.JsonSchemaHelper;
 import org.apache.camel.tools.apt.helper.Strings;
 
+import static org.apache.camel.tools.apt.AnnotationProcessorHelper.dumpExceptionToErrorFile;
+import static org.apache.camel.tools.apt.AnnotationProcessorHelper.error;
 import static org.apache.camel.tools.apt.AnnotationProcessorHelper.findJavaDoc;
 import static org.apache.camel.tools.apt.AnnotationProcessorHelper.findTypeElement;
 import static org.apache.camel.tools.apt.AnnotationProcessorHelper.findTypeElementChildren;
@@ -124,6 +132,7 @@ public class CoreEipAnnotationProcessorHelper {
         processFile(processingEnv, packageName, fileName, writer -> writeJSonSchemeDocumentation(processingEnv, writer, roundEnv, classElement, rootElement, javaTypeName, name));
     }
 
+
     protected void writeJSonSchemeDocumentation(ProcessingEnvironment processingEnv, PrintWriter writer, RoundEnvironment roundEnv, TypeElement classElement,
                                                 XmlRootElement rootElement, String javaTypeName, String modelName) {
         // gather eip information
@@ -141,6 +150,155 @@ public class CoreEipAnnotationProcessorHelper {
 
         String json = createParameterJsonSchema(eipModel, eipOptions);
         writer.println(json);
+
+        // write property placeholder source code
+        writePropertyPlaceholderProviderSource(processingEnv, writer, roundEnv, classElement, eipModel, eipOptions);
+    }
+
+    protected void writePropertyPlaceholderProviderSource(ProcessingEnvironment processingEnv, PrintWriter writer, RoundEnvironment roundEnv, TypeElement classElement,
+                                                          EipModel eipModel, Set<EipOption> options) {
+
+        // the following are valid class elements which we want to generate
+        boolean rest = classElement.getQualifiedName().toString().startsWith("org.apache.camel.model.rest");
+        boolean processor = hasSuperClass(processingEnv, roundEnv, classElement, "org.apache.camel.model.ProcessorDefinition");
+        boolean language = hasSuperClass(processingEnv, roundEnv, classElement, "org.apache.camel.model.language.ExpressionDefinition");
+        boolean dataformat = hasSuperClass(processingEnv, roundEnv, classElement, "org.apache.camel.model.DataFormatDefinition");
+
+        if (!rest && !processor && !language && !dataformat) {
+            return;
+        }
+
+        TypeElement parent = findTypeElement(processingEnv, roundEnv, "org.apache.camel.model.DefinitionPropertyPlaceholderConfigurable");
+        String def = classElement.getSimpleName().toString();
+        String fqnDef = classElement.getQualifiedName().toString();
+        String cn = def + "PropertyPlaceholderProvider";
+        String fqn = "org.apache.camel.model.placeholder." + cn;
+
+        doWritePropertyPlaceholderProviderSource(processingEnv, parent, def, fqnDef, cn, fqn, options);
+
+        // we also need to generate from when we generate route as from can also configure property placeholders
+        if ("RouteDefinition".equals(def)) {
+            def = "FromDefinition";
+            fqnDef = "org.apache.camel.model.FromDefinition";
+            cn = "FromDefinitionPropertyPlaceholderProvider";
+            fqn = "org.apache.camel.model.placeholder.FromDefinitionPropertyPlaceholderProvider";
+
+            options.clear();
+            options.add(new EipOption("id", null, null, "java.lang.String", false, null, null, false, null, false, null, false, null, false));
+            options.add(new EipOption("uri", null, null, "java.lang.String", false, null, null, false, null, false, null, false, null, false));
+
+            doWritePropertyPlaceholderProviderSource(processingEnv, parent, def, fqnDef, cn, fqn, options);
+        }
+    }
+
+    private void doWritePropertyPlaceholderProviderSource(ProcessingEnvironment processingEnv, TypeElement parent,
+                                                          String def, String fqnDef, String cn, String fqn,
+                                                          Set<EipOption> options) {
+
+        Writer w = null;
+        try {
+            JavaFileObject src = processingEnv.getFiler().createSourceFile(fqn, parent);
+            w = src.openWriter();
+
+            w.write("/* Generated by camel-apt */\n");
+            w.write("package org.apache.camel.model.placeholder;\n");
+            w.write("\n");
+            w.write("import java.util.HashMap;\n");
+            w.write("import java.util.Map;\n");
+            w.write("import java.util.function.Consumer;\n");
+            w.write("import java.util.function.Supplier;\n");
+            w.write("\n");
+            w.write("import org.apache.camel.CamelContext;\n");
+            w.write("import " + fqnDef + ";\n");
+            w.write("import org.apache.camel.model.DefinitionPropertyPlaceholderConfigurable;\n");
+            w.write("\n");
+            w.write("public class " + cn + " implements DefinitionPropertyPlaceholderConfigurable {\n");
+            w.write("\n");
+            w.write("    private final Map<String, Supplier<String>> readPlaceholders = new HashMap<>();\n");
+            w.write("    private final Map<String, Consumer<String>> writePlaceholders = new HashMap<>();\n");
+            w.write("\n");
+
+            // add constructor
+            w.write("    public " + cn + "(Object obj) {\n");
+            w.write("        " + def + " definition = (" + def + ") obj;\n");
+            w.write("\n");
+
+            // only include string types as they are the only ones we can use for property placeholders
+            boolean found = false;
+            for (EipOption option : options) {
+                if ("java.lang.String".equals(option.getType())) {
+                    found = true;
+                    String getOrSet = sanitizeOptionName(def, option);
+                    getOrSet = Character.toUpperCase(getOrSet.charAt(0)) + getOrSet.substring(1);
+                    w.write("        readPlaceholders.put(\"" + option.getName() + "\", definition::get" + getOrSet + ");\n");
+                    w.write("        writePlaceholders.put(\"" + option.getName() + "\", definition::set" + getOrSet + ");\n");
+                }
+            }
+            if (!found) {
+                w.write("\n");
+            }
+
+            w.write("    }\n");
+            w.write("\n");
+            w.write("    @Override\n");
+            w.write("    public Map<String, Supplier<String>> getReadPropertyPlaceholderOptions(CamelContext camelContext) {\n");
+            w.write("        return readPlaceholders;\n");
+            w.write("    }\n");
+            w.write("\n");
+            w.write("    @Override\n");
+            w.write("    public Map<String, Consumer<String>> getWritePropertyPlaceholderOptions(CamelContext camelContext) {\n");
+            w.write("        return writePlaceholders;\n");
+            w.write("    }\n");
+            w.write("\n");
+            w.write("}\n");
+            w.write("\n");
+        } catch (Exception e) {
+            processingEnv.getMessager().printMessage(Diagnostic.Kind.ERROR, "Unable to process annotated elements in " + getClass().getSimpleName() + ": " + e.getMessage());
+            dumpExceptionToErrorFile("camel-apt-error.log", "Error processing annotation in " + getClass().getSimpleName(), e);
+        } finally {
+            IOHelper.close(w);
+        }
+    }
+
+    public String sanitizeOptionName(String def, EipOption option) {
+        // some elements have different setter/getter names vs the xml dsl
+        if ("SimpleExpression".equals(def) || "JsonPathExpression".equals(def)) {
+            if ("resultType".equals(option.getName())) {
+                return "resultTypeName";
+            }
+        } else if ("EnrichDefinition".equals(def) || "PollEnrichDefinition".equals(def) || "ClaimCheckDefinition".equals(def)) {
+            if ("strategyRef".equals(option.getName())) {
+                return "aggregationStrategyRef";
+            } else if ("strategyMethodName".equals(option.getName())) {
+                return "aggregationStrategyMethodName";
+            } else if ("strategyMethodAllowNull".equals(option.getName())) {
+                return "aggregationStrategyMethodAllowNull";
+            }
+        } else if ("MethodCallExpression".equals(def)) {
+            if ("beanType".equals(option.getName())) {
+                return "beanTypeName";
+            }
+        } else if ("XPathExpression".equals(def)) {
+            if ("documentType".equals(option.getName())) {
+                return "documentTypeName";
+            } else if ("resultType".equals(option.getName())) {
+                return "resultTypeName";
+            }
+        } else if ("WireTapDefinition".equals(def)) {
+            if ("processorRef".equals(option.getName())) {
+                return "newExchangeProcessorRef";
+            }
+        } else if ("TidyMarkupDataFormat".equals(def)) {
+            if ("dataObjectType".equals(option.getName())) {
+                return "dataObjectTypeName";
+            }
+        } else if ("BindyDataFormat".equals(def)) {
+            if ("classType".equals(option.getName())) {
+                return "classTypeAsString";
+            }
+        }
+
+        return option.getName();
     }
 
     public String createParameterJsonSchema(EipModel eipModel, Set<EipOption> options) {