You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by "orpiske (via GitHub)" <gi...@apache.org> on 2023/08/29 05:36:36 UTC

[GitHub] [camel] orpiske opened a new pull request, #11225: CAMEL-19398: refactor the core type-converter

orpiske opened a new pull request, #11225:
URL: https://github.com/apache/camel/pull/11225

   This is preliminary work that should simplify moving away from an Exception-based logic for type handling
   
   It refactors the type conversion code so that it is preloaded to a Map, which store type conversion pairs. This map is checked whenever there is a type conversion, by matching the requested pair (from/to) with the known conversions on the Map.
   
   Among other things, it reduces the incidence of runtime type detection and tries to let the compiler decide the method whenever it is possible
   
   This builds on top of the work done on #10271.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel] orpiske commented on pull request #11225: CAMEL-19398: refactor the core type-converter

Posted by "orpiske (via GitHub)" <gi...@apache.org>.
orpiske commented on PR #11225:
URL: https://github.com/apache/camel/pull/11225#issuecomment-1696805803

   Some performance details: 
   - Testing with a content-based-router, routing based on the body of a message, this code was faster than 4.0.0 in 100% of the times 
   - Testing with a routing-slip,  with a bean to dynamically route the exchanges, this code was faster than 4.0.0 in about 83% of the times
   - For seda and controlbus, the performance from previous version seems approximately the same 
   - There seems to be a small performance drop with the disruptor component (but I plan to investigate further)


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel] orpiske commented on a diff in pull request #11225: CAMEL-19398: refactor the core type-converter

Posted by "orpiske (via GitHub)" <gi...@apache.org>.
orpiske commented on code in PR #11225:
URL: https://github.com/apache/camel/pull/11225#discussion_r1308458572


##########
core/camel-base/src/main/java/org/apache/camel/impl/converter/TypeResolverHelper.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.impl.converter;
+
+import java.util.Map;
+import java.util.Optional;
+
+import org.apache.camel.TypeConverter;
+import org.apache.camel.converter.TypeConvertible;
+
+/**
+ * Helper methods for resolving the type conversions. This is an internal API and not meant for public usages.
+ *
+ * In a broader sense: the methods of this code help with traversing the class hierarchy of the types involved in a
+ * conversion, so that the correct TypeConverter can be used. This is a helper class to CoreTypeConverterRegistry.
+ *
+ * In the CoreTypeConverterRegistry class, the registry of types if maintained in a ConcurrentMap that associates a type
+ * pair with the resolver for it (i.e.: it associates pair representing a conversion from String to Integer to a type
+ * converter - such as CamelBaseBulkConverterLoader).
+ *
+ * NOTE 1: a lot of this code is in the hot path of the core engine, so change with extreme caution to prevent
+ * performance issues on the core code.
+ *
+ * NOTE 2: also, a lot of this code runs rather slow operations, so calling these methods should be avoided as much as
+ * possible
+ *
+ */
+final class TypeResolverHelper {
+    private TypeResolverHelper() {
+
+    }
+
+    /**
+     * Lookup the type converter in the registry (given a type to convert to and a type to convert from, along with a
+     * mapping of all known converters)
+     *
+     * @param  toType     the type to convert to
+     * @param  fromType   the type to convert from
+     * @param  converters the map of all known converters
+     * @return            the type converter or null if unknown
+     */
+    static TypeConverter doLookup(
+            Class<?> toType, Class<?> fromType, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+        return doLookup(new TypeConvertible<>(fromType, toType), converters);
+    }
+
+    private static TypeConverter doLookup(
+            TypeConvertible<?, ?> typeConvertible, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+
+        // try with base converters first
+        final TypeConverter typeConverter = converters.get(typeConvertible);
+        if (typeConverter != null) {
+            return typeConverter;
+        }
+
+        final TypeConverter superConverterTc = tryMatch(typeConvertible, converters);
+        if (superConverterTc != null) {
+            return superConverterTc;
+        }
+
+        final TypeConverter arrayConverterTc = tryObjectArrayConverters(typeConvertible, converters);
+        if (arrayConverterTc != null) {
+            return arrayConverterTc;
+        }
+
+        final TypeConverter primitiveAwareConverter = tryPrimitive(typeConvertible, converters);
+        if (primitiveAwareConverter != null) {
+            return primitiveAwareConverter;
+        }
+
+        // only do these tests as fallback and only on the target type
+        if (!typeConvertible.getFrom().equals(Object.class)) {
+
+            final TypeConverter assignableConverter
+                    = tryAssignableFrom(typeConvertible, converters);
+            if (assignableConverter != null) {
+                return assignableConverter;
+            }
+
+            final TypeConverter objConverter = converters.get(new TypeConvertible<>(Object.class, typeConvertible.getTo()));
+            if (objConverter != null) {
+                return objConverter;
+            }
+        }
+
+        // none found
+        return null;
+    }
+
+    /**
+     * Try the base converters. That is, those matching a direct conversion (i.e.: when the from and to types requested
+     * do exist on the converters' map OR when the from and to types requested match for a _primitive type).
+     *
+     * For instance: From String.class, To: int.class (would match a method such as myConverter(String, Integer) or
+     * myConverter(String, int).
+     *
+     * @param  typeConvertible the type converter pair
+     * @param  converters      the map of all known converters
+     * @return                 the type converter or null if unknown
+     */
+    static TypeConverter tryAssignableFrom(
+            TypeConvertible<?, ?> typeConvertible, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+
+        /*
+         Let's try classes derived from this toType: basically it traverses the entries looking for assignable types
+         matching both the "from type" and the "to type" which are NOT Object (we usually try this later).
+         */
+        final Optional<Map.Entry<TypeConvertible<?, ?>, TypeConverter>> first = converters.entrySet().stream()
+                .filter(v -> v.getKey().isAssignableMatch(typeConvertible))
+                .findFirst();
+
+        return first.map(Map.Entry::getValue).orElse(null);
+    }
+
+    /**
+     * Try to resolve a TypeConverter by looking at Array matches for a given "from" type. This also includes evaluating
+     * candidates matching a receiver of Object[] type.
+     *
+     * @param  typeConvertible the type converter pair
+     * @param  converters      the map of all known converters
+     * @return                 the type converter or null if unknown
+     */
+    static TypeConverter tryObjectArrayConverters(
+            TypeConvertible<?, ?> typeConvertible, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+
+        return tryObjectArrayConverters(typeConvertible.getFrom(), typeConvertible.getTo(), converters);
+    }
+
+    private static TypeConverter tryObjectArrayConverters(
+            Class<?> fromType, Class<?> toType, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+        // Let the fallback converters handle the primitive arrays
+        if (fromType.isArray() && !fromType.getComponentType().isPrimitive()) {
+            /* We usually define our converters are receiving an object array (Object[]), however, this won't easily match:
+             * because an object array is not an interface or a super class of other array types (i.e.: not a super class
+             * of String[]). So, we take the direct road here and try check for an object array converter right away.
+             */
+            return converters.get(new TypeConvertible<>(Object[].class, toType));
+        }
+
+        return null;
+    }
+
+    /**
+     * Try to resolve the TypeConverter by forcing a costly and slow recursive check.
+     *
+     * @param  typeConvertible the type converter pair
+     * @param  converters      the map of all known converters
+     * @return                 the type converter or null if unknown
+     */
+    static TypeConverter tryMatch(
+            TypeConvertible<?, ?> typeConvertible, Map<TypeConvertible<?, ?>, TypeConverter> converters) {

Review Comment:
   I'm going to give it a try with a plain for loop and see how it goes.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel] orpiske commented on a diff in pull request #11225: CAMEL-19398: refactor the core type-converter

Posted by "orpiske (via GitHub)" <gi...@apache.org>.
orpiske commented on code in PR #11225:
URL: https://github.com/apache/camel/pull/11225#discussion_r1309931621


##########
core/camel-api/src/main/java/org/apache/camel/converter/ConverterFunction.java:
##########
@@ -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.
+ */
+
+package org.apache.camel.converter;
+
+import org.apache.camel.Exchange;
+
+@FunctionalInterface

Review Comment:
   Thanks. This file was unused - leftover from the first version of this patch. I removed it. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel] orpiske commented on a diff in pull request #11225: CAMEL-19398: refactor the core type-converter

Posted by "orpiske (via GitHub)" <gi...@apache.org>.
orpiske commented on code in PR #11225:
URL: https://github.com/apache/camel/pull/11225#discussion_r1309934391


##########
core/camel-base/src/main/java/org/apache/camel/impl/converter/TypeResolverHelper.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.impl.converter;
+
+import java.util.Map;
+import java.util.Optional;
+
+import org.apache.camel.TypeConverter;
+import org.apache.camel.converter.TypeConvertible;
+
+/**
+ * Helper methods for resolving the type conversions. This is an internal API and not meant for public usages.
+ *
+ * In a broader sense: the methods of this code help with traversing the class hierarchy of the types involved in a
+ * conversion, so that the correct TypeConverter can be used. This is a helper class to CoreTypeConverterRegistry.
+ *
+ * In the CoreTypeConverterRegistry class, the registry of types if maintained in a ConcurrentMap that associates a type
+ * pair with the resolver for it (i.e.: it associates pair representing a conversion from String to Integer to a type
+ * converter - such as CamelBaseBulkConverterLoader).
+ *
+ * NOTE 1: a lot of this code is in the hot path of the core engine, so change with extreme caution to prevent
+ * performance issues on the core code.
+ *
+ * NOTE 2: also, a lot of this code runs rather slow operations, so calling these methods should be avoided as much as
+ * possible
+ *
+ */
+final class TypeResolverHelper {
+    private TypeResolverHelper() {
+
+    }
+
+    /**
+     * Lookup the type converter in the registry (given a type to convert to and a type to convert from, along with a
+     * mapping of all known converters)
+     *
+     * @param  toType     the type to convert to
+     * @param  fromType   the type to convert from
+     * @param  converters the map of all known converters
+     * @return            the type converter or null if unknown
+     */
+    static TypeConverter doLookup(
+            Class<?> toType, Class<?> fromType, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+        return doLookup(new TypeConvertible<>(fromType, toType), converters);
+    }
+
+    private static TypeConverter doLookup(
+            TypeConvertible<?, ?> typeConvertible, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+
+        // try with base converters first
+        final TypeConverter typeConverter = converters.get(typeConvertible);
+        if (typeConverter != null) {
+            return typeConverter;
+        }
+
+        final TypeConverter superConverterTc = tryMatch(typeConvertible, converters);
+        if (superConverterTc != null) {
+            return superConverterTc;
+        }
+
+        final TypeConverter arrayConverterTc = tryObjectArrayConverters(typeConvertible, converters);
+        if (arrayConverterTc != null) {
+            return arrayConverterTc;
+        }
+
+        final TypeConverter primitiveAwareConverter = tryPrimitive(typeConvertible, converters);
+        if (primitiveAwareConverter != null) {
+            return primitiveAwareConverter;
+        }
+
+        // only do these tests as fallback and only on the target type
+        if (!typeConvertible.getFrom().equals(Object.class)) {
+
+            final TypeConverter assignableConverter
+                    = tryAssignableFrom(typeConvertible, converters);
+            if (assignableConverter != null) {
+                return assignableConverter;
+            }
+
+            final TypeConverter objConverter = converters.get(new TypeConvertible<>(Object.class, typeConvertible.getTo()));
+            if (objConverter != null) {
+                return objConverter;
+            }
+        }
+
+        // none found
+        return null;
+    }
+
+    /**
+     * Try the base converters. That is, those matching a direct conversion (i.e.: when the from and to types requested
+     * do exist on the converters' map OR when the from and to types requested match for a _primitive type).
+     *
+     * For instance: From String.class, To: int.class (would match a method such as myConverter(String, Integer) or
+     * myConverter(String, int).
+     *
+     * @param  typeConvertible the type converter pair
+     * @param  converters      the map of all known converters
+     * @return                 the type converter or null if unknown
+     */
+    static TypeConverter tryAssignableFrom(
+            TypeConvertible<?, ?> typeConvertible, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+
+        /*
+         Let's try classes derived from this toType: basically it traverses the entries looking for assignable types
+         matching both the "from type" and the "to type" which are NOT Object (we usually try this later).
+         */
+        final Optional<Map.Entry<TypeConvertible<?, ?>, TypeConverter>> first = converters.entrySet().stream()
+                .filter(v -> v.getKey().isAssignableMatch(typeConvertible))
+                .findFirst();
+
+        return first.map(Map.Entry::getValue).orElse(null);
+    }
+
+    /**
+     * Try to resolve a TypeConverter by looking at Array matches for a given "from" type. This also includes evaluating
+     * candidates matching a receiver of Object[] type.
+     *
+     * @param  typeConvertible the type converter pair
+     * @param  converters      the map of all known converters
+     * @return                 the type converter or null if unknown
+     */
+    static TypeConverter tryObjectArrayConverters(
+            TypeConvertible<?, ?> typeConvertible, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+
+        return tryObjectArrayConverters(typeConvertible.getFrom(), typeConvertible.getTo(), converters);
+    }
+
+    private static TypeConverter tryObjectArrayConverters(
+            Class<?> fromType, Class<?> toType, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+        // Let the fallback converters handle the primitive arrays
+        if (fromType.isArray() && !fromType.getComponentType().isPrimitive()) {
+            /* We usually define our converters are receiving an object array (Object[]), however, this won't easily match:
+             * because an object array is not an interface or a super class of other array types (i.e.: not a super class
+             * of String[]). So, we take the direct road here and try check for an object array converter right away.
+             */
+            return converters.get(new TypeConvertible<>(Object[].class, toType));
+        }
+
+        return null;
+    }
+
+    /**
+     * Try to resolve the TypeConverter by forcing a costly and slow recursive check.
+     *
+     * @param  typeConvertible the type converter pair
+     * @param  converters      the map of all known converters
+     * @return                 the type converter or null if unknown
+     */
+    static TypeConverter tryMatch(
+            TypeConvertible<?, ?> typeConvertible, Map<TypeConvertible<?, ?>, TypeConverter> converters) {

Review Comment:
   There's indications that the plain loop is better as it creates less objects and all that. So, I changed to a loop as you suggested.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel] orpiske commented on a diff in pull request #11225: CAMEL-19398: refactor the core type-converter

Posted by "orpiske (via GitHub)" <gi...@apache.org>.
orpiske commented on code in PR #11225:
URL: https://github.com/apache/camel/pull/11225#discussion_r1308451384


##########
components/camel-vertx/camel-vertx-common/src/generated/java/org/apache/camel/component/vertx/common/CamelVertxCommonBulkConverterLoader.java:
##########
@@ -165,6 +167,41 @@ private Object doConvertTo(Class<?> from, Class<?> to, Exchange exchange, Object
         return null;
     }
 
+    private void doRegistration(TypeConverterRegistry registry) {
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.buffer.Buffer.class, byte[].class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonArray.class, byte[].class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonObject.class, byte[].class), this);
+        registry.addConverter(new TypeConvertible<>(byte[].class, io.vertx.core.buffer.Buffer.class), this);
+        registry.addConverter(new TypeConvertible<>(io.netty.buffer.ByteBuf.class, io.vertx.core.buffer.Buffer.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonArray.class, io.vertx.core.buffer.Buffer.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonObject.class, io.vertx.core.buffer.Buffer.class), this);
+        registry.addConverter(new TypeConvertible<>(java.io.InputStream.class, io.vertx.core.buffer.Buffer.class), this);
+        registry.addConverter(new TypeConvertible<>(java.lang.String.class, io.vertx.core.buffer.Buffer.class), this);
+        registry.addConverter(new TypeConvertible<>(byte[].class, io.vertx.core.json.JsonArray.class), this);
+        registry.addConverter(new TypeConvertible<>(io.netty.buffer.ByteBuf.class, io.vertx.core.json.JsonArray.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.buffer.Buffer.class, io.vertx.core.json.JsonArray.class), this);
+        registry.addConverter(new TypeConvertible<>(java.io.InputStream.class, io.vertx.core.json.JsonArray.class), this);
+        registry.addConverter(new TypeConvertible<>(java.lang.String.class, io.vertx.core.json.JsonArray.class), this);
+        registry.addConverter(new TypeConvertible<>(java.util.List.class, io.vertx.core.json.JsonArray.class), this);
+        registry.addConverter(new TypeConvertible<>(byte[].class, io.vertx.core.json.JsonObject.class), this);
+        registry.addConverter(new TypeConvertible<>(io.netty.buffer.ByteBuf.class, io.vertx.core.json.JsonObject.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.buffer.Buffer.class, io.vertx.core.json.JsonObject.class), this);
+        registry.addConverter(new TypeConvertible<>(java.io.InputStream.class, io.vertx.core.json.JsonObject.class), this);
+        registry.addConverter(new TypeConvertible<>(java.lang.String.class, io.vertx.core.json.JsonObject.class), this);
+        registry.addConverter(new TypeConvertible<>(java.util.Map.class, io.vertx.core.json.JsonObject.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.buffer.Buffer.class, java.io.InputStream.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonArray.class, java.io.InputStream.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonObject.class, java.io.InputStream.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.buffer.Buffer.class, java.lang.String.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonArray.class, java.lang.String.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonObject.class, java.lang.String.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonArray.class, java.util.List.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonObject.class, java.util.Map.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.buffer.Buffer.class, org.apache.camel.StreamCache.class), this);
+        
+        
+    }
+
     public TypeConverter lookup(Class<?> to, Class<?> from) {

Review Comment:
   There's still a lot of code relying on the lookup and it seems that we have some features that require the lookup behavior (i.e.: fallback promotion [ seems to be used for beans], CXF converter, etc). 
   
   For now, I am being conservative and I am aiming to be as compliant/compatible with the old code as possible to avoid introducing more changes than I can handle. 
   
   So, I am thinking about leaving this one for this first patch. Then slowly start to replacing usages of the lookup (or clean it up - at least). 
   
   What do you think?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel] orpiske commented on pull request #11225: CAMEL-19398: refactor the core type-converter

Posted by "orpiske (via GitHub)" <gi...@apache.org>.
orpiske commented on PR #11225:
URL: https://github.com/apache/camel/pull/11225#issuecomment-1696811872

   Using the tests from camel-performance-tests repo:
   | Test | 4.0.0 | This Patch | 
   |------|-----|------|
   | SlowTypeConverter | 35 ms/op | 7 ms/op |
   | FastTypeConverterTest | 35 ms/op | 7 ms/op | 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel] orpiske commented on a diff in pull request #11225: CAMEL-19398: refactor the core type-converter

Posted by "orpiske (via GitHub)" <gi...@apache.org>.
orpiske commented on code in PR #11225:
URL: https://github.com/apache/camel/pull/11225#discussion_r1309933015


##########
core/camel-base/src/main/java/org/apache/camel/impl/converter/TypeResolverHelper.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.impl.converter;
+
+import java.util.Map;
+import java.util.Optional;
+
+import org.apache.camel.TypeConverter;
+import org.apache.camel.converter.TypeConvertible;
+
+/**
+ * Helper methods for resolving the type conversions. This is an internal API and not meant for public usages.
+ *
+ * In a broader sense: the methods of this code help with traversing the class hierarchy of the types involved in a
+ * conversion, so that the correct TypeConverter can be used. This is a helper class to CoreTypeConverterRegistry.
+ *
+ * In the CoreTypeConverterRegistry class, the registry of types if maintained in a ConcurrentMap that associates a type
+ * pair with the resolver for it (i.e.: it associates pair representing a conversion from String to Integer to a type
+ * converter - such as CamelBaseBulkConverterLoader).
+ *
+ * NOTE 1: a lot of this code is in the hot path of the core engine, so change with extreme caution to prevent
+ * performance issues on the core code.
+ *
+ * NOTE 2: also, a lot of this code runs rather slow operations, so calling these methods should be avoided as much as
+ * possible
+ *
+ */
+final class TypeResolverHelper {
+    private TypeResolverHelper() {
+
+    }
+
+    /**
+     * Lookup the type converter in the registry (given a type to convert to and a type to convert from, along with a
+     * mapping of all known converters)
+     *
+     * @param  toType     the type to convert to
+     * @param  fromType   the type to convert from
+     * @param  converters the map of all known converters
+     * @return            the type converter or null if unknown
+     */
+    static TypeConverter doLookup(
+            Class<?> toType, Class<?> fromType, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+        return doLookup(new TypeConvertible<>(fromType, toType), converters);
+    }
+
+    private static TypeConverter doLookup(
+            TypeConvertible<?, ?> typeConvertible, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+
+        // try with base converters first
+        final TypeConverter typeConverter = converters.get(typeConvertible);
+        if (typeConverter != null) {
+            return typeConverter;
+        }
+
+        final TypeConverter superConverterTc = tryMatch(typeConvertible, converters);
+        if (superConverterTc != null) {
+            return superConverterTc;
+        }
+
+        final TypeConverter arrayConverterTc = tryObjectArrayConverters(typeConvertible, converters);

Review Comment:
   It seems we can remove this, so this code is not longer valid.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel] orpiske commented on a diff in pull request #11225: CAMEL-19398: refactor the core type-converter

Posted by "orpiske (via GitHub)" <gi...@apache.org>.
orpiske commented on code in PR #11225:
URL: https://github.com/apache/camel/pull/11225#discussion_r1308478121


##########
core/camel-base/src/main/java/org/apache/camel/impl/converter/TypeResolverHelper.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.impl.converter;
+
+import java.util.Map;
+import java.util.Optional;
+
+import org.apache.camel.TypeConverter;
+import org.apache.camel.converter.TypeConvertible;
+
+/**
+ * Helper methods for resolving the type conversions. This is an internal API and not meant for public usages.
+ *
+ * In a broader sense: the methods of this code help with traversing the class hierarchy of the types involved in a
+ * conversion, so that the correct TypeConverter can be used. This is a helper class to CoreTypeConverterRegistry.
+ *
+ * In the CoreTypeConverterRegistry class, the registry of types if maintained in a ConcurrentMap that associates a type
+ * pair with the resolver for it (i.e.: it associates pair representing a conversion from String to Integer to a type
+ * converter - such as CamelBaseBulkConverterLoader).
+ *
+ * NOTE 1: a lot of this code is in the hot path of the core engine, so change with extreme caution to prevent
+ * performance issues on the core code.
+ *
+ * NOTE 2: also, a lot of this code runs rather slow operations, so calling these methods should be avoided as much as
+ * possible
+ *
+ */
+final class TypeResolverHelper {
+    private TypeResolverHelper() {
+
+    }
+
+    /**
+     * Lookup the type converter in the registry (given a type to convert to and a type to convert from, along with a
+     * mapping of all known converters)
+     *
+     * @param  toType     the type to convert to
+     * @param  fromType   the type to convert from
+     * @param  converters the map of all known converters
+     * @return            the type converter or null if unknown
+     */
+    static TypeConverter doLookup(
+            Class<?> toType, Class<?> fromType, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+        return doLookup(new TypeConvertible<>(fromType, toType), converters);
+    }
+
+    private static TypeConverter doLookup(
+            TypeConvertible<?, ?> typeConvertible, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+
+        // try with base converters first
+        final TypeConverter typeConverter = converters.get(typeConvertible);
+        if (typeConverter != null) {
+            return typeConverter;
+        }
+
+        final TypeConverter superConverterTc = tryMatch(typeConvertible, converters);
+        if (superConverterTc != null) {
+            return superConverterTc;
+        }
+
+        final TypeConverter arrayConverterTc = tryObjectArrayConverters(typeConvertible, converters);

Review Comment:
   Maybe we can remove that one? 🤔 I'll give it a try



##########
core/camel-base/src/main/java/org/apache/camel/impl/converter/TypeResolverHelper.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.impl.converter;
+
+import java.util.Map;
+import java.util.Optional;
+
+import org.apache.camel.TypeConverter;
+import org.apache.camel.converter.TypeConvertible;
+
+/**
+ * Helper methods for resolving the type conversions. This is an internal API and not meant for public usages.
+ *
+ * In a broader sense: the methods of this code help with traversing the class hierarchy of the types involved in a
+ * conversion, so that the correct TypeConverter can be used. This is a helper class to CoreTypeConverterRegistry.
+ *
+ * In the CoreTypeConverterRegistry class, the registry of types if maintained in a ConcurrentMap that associates a type
+ * pair with the resolver for it (i.e.: it associates pair representing a conversion from String to Integer to a type
+ * converter - such as CamelBaseBulkConverterLoader).
+ *
+ * NOTE 1: a lot of this code is in the hot path of the core engine, so change with extreme caution to prevent
+ * performance issues on the core code.
+ *
+ * NOTE 2: also, a lot of this code runs rather slow operations, so calling these methods should be avoided as much as
+ * possible
+ *
+ */
+final class TypeResolverHelper {
+    private TypeResolverHelper() {
+
+    }
+
+    /**
+     * Lookup the type converter in the registry (given a type to convert to and a type to convert from, along with a
+     * mapping of all known converters)
+     *
+     * @param  toType     the type to convert to
+     * @param  fromType   the type to convert from
+     * @param  converters the map of all known converters
+     * @return            the type converter or null if unknown
+     */
+    static TypeConverter doLookup(
+            Class<?> toType, Class<?> fromType, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+        return doLookup(new TypeConvertible<>(fromType, toType), converters);
+    }
+
+    private static TypeConverter doLookup(
+            TypeConvertible<?, ?> typeConvertible, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+
+        // try with base converters first
+        final TypeConverter typeConverter = converters.get(typeConvertible);
+        if (typeConverter != null) {
+            return typeConverter;
+        }
+
+        final TypeConverter superConverterTc = tryMatch(typeConvertible, converters);
+        if (superConverterTc != null) {
+            return superConverterTc;
+        }
+
+        final TypeConverter arrayConverterTc = tryObjectArrayConverters(typeConvertible, converters);

Review Comment:
   Maybe we can remove that one? 🤔 I'll give it a try



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel] davsclaus commented on a diff in pull request #11225: CAMEL-19398: refactor the core type-converter

Posted by "davsclaus (via GitHub)" <gi...@apache.org>.
davsclaus commented on code in PR #11225:
URL: https://github.com/apache/camel/pull/11225#discussion_r1308425378


##########
components/camel-vertx/camel-vertx-common/src/generated/java/org/apache/camel/component/vertx/common/CamelVertxCommonBulkConverterLoader.java:
##########
@@ -165,6 +167,41 @@ private Object doConvertTo(Class<?> from, Class<?> to, Exchange exchange, Object
         return null;
     }
 
+    private void doRegistration(TypeConverterRegistry registry) {
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.buffer.Buffer.class, byte[].class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonArray.class, byte[].class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonObject.class, byte[].class), this);
+        registry.addConverter(new TypeConvertible<>(byte[].class, io.vertx.core.buffer.Buffer.class), this);
+        registry.addConverter(new TypeConvertible<>(io.netty.buffer.ByteBuf.class, io.vertx.core.buffer.Buffer.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonArray.class, io.vertx.core.buffer.Buffer.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonObject.class, io.vertx.core.buffer.Buffer.class), this);
+        registry.addConverter(new TypeConvertible<>(java.io.InputStream.class, io.vertx.core.buffer.Buffer.class), this);
+        registry.addConverter(new TypeConvertible<>(java.lang.String.class, io.vertx.core.buffer.Buffer.class), this);
+        registry.addConverter(new TypeConvertible<>(byte[].class, io.vertx.core.json.JsonArray.class), this);
+        registry.addConverter(new TypeConvertible<>(io.netty.buffer.ByteBuf.class, io.vertx.core.json.JsonArray.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.buffer.Buffer.class, io.vertx.core.json.JsonArray.class), this);
+        registry.addConverter(new TypeConvertible<>(java.io.InputStream.class, io.vertx.core.json.JsonArray.class), this);
+        registry.addConverter(new TypeConvertible<>(java.lang.String.class, io.vertx.core.json.JsonArray.class), this);
+        registry.addConverter(new TypeConvertible<>(java.util.List.class, io.vertx.core.json.JsonArray.class), this);
+        registry.addConverter(new TypeConvertible<>(byte[].class, io.vertx.core.json.JsonObject.class), this);
+        registry.addConverter(new TypeConvertible<>(io.netty.buffer.ByteBuf.class, io.vertx.core.json.JsonObject.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.buffer.Buffer.class, io.vertx.core.json.JsonObject.class), this);
+        registry.addConverter(new TypeConvertible<>(java.io.InputStream.class, io.vertx.core.json.JsonObject.class), this);
+        registry.addConverter(new TypeConvertible<>(java.lang.String.class, io.vertx.core.json.JsonObject.class), this);
+        registry.addConverter(new TypeConvertible<>(java.util.Map.class, io.vertx.core.json.JsonObject.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.buffer.Buffer.class, java.io.InputStream.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonArray.class, java.io.InputStream.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonObject.class, java.io.InputStream.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.buffer.Buffer.class, java.lang.String.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonArray.class, java.lang.String.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonObject.class, java.lang.String.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonArray.class, java.util.List.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonObject.class, java.util.Map.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.buffer.Buffer.class, org.apache.camel.StreamCache.class), this);
+        
+        
+    }
+
     public TypeConverter lookup(Class<?> to, Class<?> from) {

Review Comment:
   I assume the old code to lookup using switch statements then is no longer needed and will later be removed?



##########
core/camel-base/src/main/java/org/apache/camel/impl/converter/TypeResolverHelper.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.impl.converter;
+
+import java.util.Map;
+import java.util.Optional;
+
+import org.apache.camel.TypeConverter;
+import org.apache.camel.converter.TypeConvertible;
+
+/**
+ * Helper methods for resolving the type conversions. This is an internal API and not meant for public usages.
+ *
+ * In a broader sense: the methods of this code help with traversing the class hierarchy of the types involved in a
+ * conversion, so that the correct TypeConverter can be used. This is a helper class to CoreTypeConverterRegistry.
+ *
+ * In the CoreTypeConverterRegistry class, the registry of types if maintained in a ConcurrentMap that associates a type
+ * pair with the resolver for it (i.e.: it associates pair representing a conversion from String to Integer to a type
+ * converter - such as CamelBaseBulkConverterLoader).
+ *
+ * NOTE 1: a lot of this code is in the hot path of the core engine, so change with extreme caution to prevent
+ * performance issues on the core code.
+ *
+ * NOTE 2: also, a lot of this code runs rather slow operations, so calling these methods should be avoided as much as
+ * possible
+ *
+ */
+final class TypeResolverHelper {
+    private TypeResolverHelper() {
+
+    }
+
+    /**
+     * Lookup the type converter in the registry (given a type to convert to and a type to convert from, along with a
+     * mapping of all known converters)
+     *
+     * @param  toType     the type to convert to
+     * @param  fromType   the type to convert from
+     * @param  converters the map of all known converters
+     * @return            the type converter or null if unknown
+     */
+    static TypeConverter doLookup(
+            Class<?> toType, Class<?> fromType, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+        return doLookup(new TypeConvertible<>(fromType, toType), converters);
+    }
+
+    private static TypeConverter doLookup(
+            TypeConvertible<?, ?> typeConvertible, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+
+        // try with base converters first
+        final TypeConverter typeConverter = converters.get(typeConvertible);
+        if (typeConverter != null) {
+            return typeConverter;
+        }
+
+        final TypeConverter superConverterTc = tryMatch(typeConvertible, converters);
+        if (superConverterTc != null) {
+            return superConverterTc;
+        }
+
+        final TypeConverter arrayConverterTc = tryObjectArrayConverters(typeConvertible, converters);

Review Comment:
   Maybe you can eager check if from.isArray to avoid calling a few methods to then do the check that late.



##########
core/camel-base/src/main/java/org/apache/camel/impl/converter/TypeResolverHelper.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.impl.converter;
+
+import java.util.Map;
+import java.util.Optional;
+
+import org.apache.camel.TypeConverter;
+import org.apache.camel.converter.TypeConvertible;
+
+/**
+ * Helper methods for resolving the type conversions. This is an internal API and not meant for public usages.
+ *
+ * In a broader sense: the methods of this code help with traversing the class hierarchy of the types involved in a
+ * conversion, so that the correct TypeConverter can be used. This is a helper class to CoreTypeConverterRegistry.
+ *
+ * In the CoreTypeConverterRegistry class, the registry of types if maintained in a ConcurrentMap that associates a type
+ * pair with the resolver for it (i.e.: it associates pair representing a conversion from String to Integer to a type
+ * converter - such as CamelBaseBulkConverterLoader).
+ *
+ * NOTE 1: a lot of this code is in the hot path of the core engine, so change with extreme caution to prevent
+ * performance issues on the core code.
+ *
+ * NOTE 2: also, a lot of this code runs rather slow operations, so calling these methods should be avoided as much as
+ * possible
+ *
+ */
+final class TypeResolverHelper {
+    private TypeResolverHelper() {
+
+    }
+
+    /**
+     * Lookup the type converter in the registry (given a type to convert to and a type to convert from, along with a
+     * mapping of all known converters)
+     *
+     * @param  toType     the type to convert to
+     * @param  fromType   the type to convert from
+     * @param  converters the map of all known converters
+     * @return            the type converter or null if unknown
+     */
+    static TypeConverter doLookup(
+            Class<?> toType, Class<?> fromType, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+        return doLookup(new TypeConvertible<>(fromType, toType), converters);
+    }
+
+    private static TypeConverter doLookup(
+            TypeConvertible<?, ?> typeConvertible, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+
+        // try with base converters first
+        final TypeConverter typeConverter = converters.get(typeConvertible);
+        if (typeConverter != null) {
+            return typeConverter;
+        }
+
+        final TypeConverter superConverterTc = tryMatch(typeConvertible, converters);
+        if (superConverterTc != null) {
+            return superConverterTc;
+        }
+
+        final TypeConverter arrayConverterTc = tryObjectArrayConverters(typeConvertible, converters);
+        if (arrayConverterTc != null) {
+            return arrayConverterTc;
+        }
+
+        final TypeConverter primitiveAwareConverter = tryPrimitive(typeConvertible, converters);
+        if (primitiveAwareConverter != null) {
+            return primitiveAwareConverter;
+        }
+
+        // only do these tests as fallback and only on the target type
+        if (!typeConvertible.getFrom().equals(Object.class)) {
+
+            final TypeConverter assignableConverter
+                    = tryAssignableFrom(typeConvertible, converters);
+            if (assignableConverter != null) {
+                return assignableConverter;
+            }
+
+            final TypeConverter objConverter = converters.get(new TypeConvertible<>(Object.class, typeConvertible.getTo()));
+            if (objConverter != null) {
+                return objConverter;
+            }
+        }
+
+        // none found
+        return null;
+    }
+
+    /**
+     * Try the base converters. That is, those matching a direct conversion (i.e.: when the from and to types requested
+     * do exist on the converters' map OR when the from and to types requested match for a _primitive type).
+     *
+     * For instance: From String.class, To: int.class (would match a method such as myConverter(String, Integer) or
+     * myConverter(String, int).
+     *
+     * @param  typeConvertible the type converter pair
+     * @param  converters      the map of all known converters
+     * @return                 the type converter or null if unknown
+     */
+    static TypeConverter tryAssignableFrom(
+            TypeConvertible<?, ?> typeConvertible, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+
+        /*
+         Let's try classes derived from this toType: basically it traverses the entries looking for assignable types
+         matching both the "from type" and the "to type" which are NOT Object (we usually try this later).
+         */
+        final Optional<Map.Entry<TypeConvertible<?, ?>, TypeConverter>> first = converters.entrySet().stream()
+                .filter(v -> v.getKey().isAssignableMatch(typeConvertible))
+                .findFirst();
+
+        return first.map(Map.Entry::getValue).orElse(null);
+    }
+
+    /**
+     * Try to resolve a TypeConverter by looking at Array matches for a given "from" type. This also includes evaluating
+     * candidates matching a receiver of Object[] type.
+     *
+     * @param  typeConvertible the type converter pair
+     * @param  converters      the map of all known converters
+     * @return                 the type converter or null if unknown
+     */
+    static TypeConverter tryObjectArrayConverters(
+            TypeConvertible<?, ?> typeConvertible, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+
+        return tryObjectArrayConverters(typeConvertible.getFrom(), typeConvertible.getTo(), converters);
+    }
+
+    private static TypeConverter tryObjectArrayConverters(
+            Class<?> fromType, Class<?> toType, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+        // Let the fallback converters handle the primitive arrays
+        if (fromType.isArray() && !fromType.getComponentType().isPrimitive()) {
+            /* We usually define our converters are receiving an object array (Object[]), however, this won't easily match:
+             * because an object array is not an interface or a super class of other array types (i.e.: not a super class
+             * of String[]). So, we take the direct road here and try check for an object array converter right away.
+             */
+            return converters.get(new TypeConvertible<>(Object[].class, toType));
+        }
+
+        return null;
+    }
+
+    /**
+     * Try to resolve the TypeConverter by forcing a costly and slow recursive check.
+     *
+     * @param  typeConvertible the type converter pair
+     * @param  converters      the map of all known converters
+     * @return                 the type converter or null if unknown
+     */
+    static TypeConverter tryMatch(
+            TypeConvertible<?, ?> typeConvertible, Map<TypeConvertible<?, ?>, TypeConverter> converters) {

Review Comment:
   All this stream filter stuff does have a bit of overhead in the JDK - sometimes code with just a for loop is faster.
   
   



##########
core/camel-api/src/main/java/org/apache/camel/converter/ConverterFunction.java:
##########
@@ -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.
+ */
+
+package org.apache.camel.converter;
+
+import org.apache.camel.Exchange;
+
+@FunctionalInterface

Review Comment:
   Add javadoc



##########
core/camel-base/src/main/java/org/apache/camel/impl/converter/TypeResolverHelper.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.impl.converter;
+
+import java.util.Map;
+import java.util.Optional;
+
+import org.apache.camel.TypeConverter;
+import org.apache.camel.converter.TypeConvertible;
+
+/**
+ * Helper methods for resolving the type conversions. This is an internal API and not meant for public usages.
+ *
+ * In a broader sense: the methods of this code help with traversing the class hierarchy of the types involved in a
+ * conversion, so that the correct TypeConverter can be used. This is a helper class to CoreTypeConverterRegistry.
+ *
+ * In the CoreTypeConverterRegistry class, the registry of types if maintained in a ConcurrentMap that associates a type
+ * pair with the resolver for it (i.e.: it associates pair representing a conversion from String to Integer to a type
+ * converter - such as CamelBaseBulkConverterLoader).
+ *
+ * NOTE 1: a lot of this code is in the hot path of the core engine, so change with extreme caution to prevent
+ * performance issues on the core code.
+ *
+ * NOTE 2: also, a lot of this code runs rather slow operations, so calling these methods should be avoided as much as
+ * possible
+ *
+ */
+final class TypeResolverHelper {
+    private TypeResolverHelper() {
+
+    }
+
+    /**
+     * Lookup the type converter in the registry (given a type to convert to and a type to convert from, along with a
+     * mapping of all known converters)
+     *
+     * @param  toType     the type to convert to
+     * @param  fromType   the type to convert from
+     * @param  converters the map of all known converters
+     * @return            the type converter or null if unknown
+     */
+    static TypeConverter doLookup(
+            Class<?> toType, Class<?> fromType, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+        return doLookup(new TypeConvertible<>(fromType, toType), converters);

Review Comment:
   The downside of this is that a new object is created per lookup - the old code was not creating any new objects during convertion, which means no overhead on GC. But the new code is faster due to that JDK bug and other parts, so we take speed over GC



##########
core/camel-base/src/main/java/org/apache/camel/impl/converter/TypeResolverHelper.java:
##########
@@ -0,0 +1,192 @@
+/*
+ * 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.impl.converter;
+
+import java.util.Map;
+import java.util.Optional;
+
+import org.apache.camel.TypeConverter;
+import org.apache.camel.converter.TypeConvertible;
+
+/**
+ * Helper methods for resolving the type conversions. This is an internal API and not meant for public usages.
+ *
+ * In a broader sense: the methods of this code help with traversing the class hierarchy of the types involved in a
+ * conversion, so that the correct TypeConverter can be used. This is a helper class to CoreTypeConverterRegistry.
+ *
+ * In the CoreTypeConverterRegistry class, the registry of types if maintained in a ConcurrentMap that associates a type
+ * pair with the resolver for it (i.e.: it associates pair representing a conversion from String to Integer to a type
+ * converter - such as CamelBaseBulkConverterLoader).
+ *
+ * NOTE 1: a lot of this code is in the hot path of the core engine, so change with extreme caution to prevent
+ * performance issues on the core code.
+ *
+ * NOTE 2: also, a lot of this code runs rather slow operations, so calling these methods should be avoided as much as
+ * possible
+ *
+ */
+final class TypeResolverHelper {
+    private TypeResolverHelper() {
+
+    }
+
+    /**
+     * Lookup the type converter in the registry (given a type to convert to and a type to convert from, along with a
+     * mapping of all known converters)
+     *
+     * @param  toType     the type to convert to
+     * @param  fromType   the type to convert from
+     * @param  converters the map of all known converters
+     * @return            the type converter or null if unknown
+     */
+    static TypeConverter doLookup(
+            Class<?> toType, Class<?> fromType, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+        return doLookup(new TypeConvertible<>(fromType, toType), converters);
+    }
+
+    private static TypeConverter doLookup(
+            TypeConvertible<?, ?> typeConvertible, Map<TypeConvertible<?, ?>, TypeConverter> converters) {
+
+        // try with base converters first
+        final TypeConverter typeConverter = converters.get(typeConvertible);
+        if (typeConverter != null) {
+            return typeConverter;
+        }
+
+        final TypeConverter superConverterTc = tryMatch(typeConvertible, converters);
+        if (superConverterTc != null) {
+            return superConverterTc;
+        }
+
+        final TypeConverter arrayConverterTc = tryObjectArrayConverters(typeConvertible, converters);

Review Comment:
   Converting to array is seldom in use



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel] github-actions[bot] commented on pull request #11225: CAMEL-19398: refactor the core type-converter

Posted by "github-actions[bot] (via GitHub)" <gi...@apache.org>.
github-actions[bot] commented on PR #11225:
URL: https://github.com/apache/camel/pull/11225#issuecomment-1696799284

   :star2: Thank you for your contribution to the Apache Camel project! :star2: 
   
   :camel: Maintainers, please note that first-time contributors *require manual approval* for the GitHub Actions to run.
   
   :warning: Please note that the changes on this PR may be **tested automatically** if they change components.
   
   :robot: Use the command `/component-test (camel-)component-name1 (camel-)component-name2..` to request a test from the test bot.
   
   If necessary Apache Camel Committers may access logs and test results in the job summaries!


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel] orpiske commented on pull request #11225: CAMEL-19398: refactor the core type-converter

Posted by "orpiske (via GitHub)" <gi...@apache.org>.
orpiske commented on PR #11225:
URL: https://github.com/apache/camel/pull/11225#issuecomment-1699010495

   Marking as skip-tests because `camel-maven-package-plugin` is severely broken on GitHub.  


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel] orpiske commented on a diff in pull request #11225: CAMEL-19398: refactor the core type-converter

Posted by "orpiske (via GitHub)" <gi...@apache.org>.
orpiske commented on code in PR #11225:
URL: https://github.com/apache/camel/pull/11225#discussion_r1309932092


##########
components/camel-vertx/camel-vertx-common/src/generated/java/org/apache/camel/component/vertx/common/CamelVertxCommonBulkConverterLoader.java:
##########
@@ -165,6 +167,41 @@ private Object doConvertTo(Class<?> from, Class<?> to, Exchange exchange, Object
         return null;
     }
 
+    private void doRegistration(TypeConverterRegistry registry) {
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.buffer.Buffer.class, byte[].class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonArray.class, byte[].class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonObject.class, byte[].class), this);
+        registry.addConverter(new TypeConvertible<>(byte[].class, io.vertx.core.buffer.Buffer.class), this);
+        registry.addConverter(new TypeConvertible<>(io.netty.buffer.ByteBuf.class, io.vertx.core.buffer.Buffer.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonArray.class, io.vertx.core.buffer.Buffer.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonObject.class, io.vertx.core.buffer.Buffer.class), this);
+        registry.addConverter(new TypeConvertible<>(java.io.InputStream.class, io.vertx.core.buffer.Buffer.class), this);
+        registry.addConverter(new TypeConvertible<>(java.lang.String.class, io.vertx.core.buffer.Buffer.class), this);
+        registry.addConverter(new TypeConvertible<>(byte[].class, io.vertx.core.json.JsonArray.class), this);
+        registry.addConverter(new TypeConvertible<>(io.netty.buffer.ByteBuf.class, io.vertx.core.json.JsonArray.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.buffer.Buffer.class, io.vertx.core.json.JsonArray.class), this);
+        registry.addConverter(new TypeConvertible<>(java.io.InputStream.class, io.vertx.core.json.JsonArray.class), this);
+        registry.addConverter(new TypeConvertible<>(java.lang.String.class, io.vertx.core.json.JsonArray.class), this);
+        registry.addConverter(new TypeConvertible<>(java.util.List.class, io.vertx.core.json.JsonArray.class), this);
+        registry.addConverter(new TypeConvertible<>(byte[].class, io.vertx.core.json.JsonObject.class), this);
+        registry.addConverter(new TypeConvertible<>(io.netty.buffer.ByteBuf.class, io.vertx.core.json.JsonObject.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.buffer.Buffer.class, io.vertx.core.json.JsonObject.class), this);
+        registry.addConverter(new TypeConvertible<>(java.io.InputStream.class, io.vertx.core.json.JsonObject.class), this);
+        registry.addConverter(new TypeConvertible<>(java.lang.String.class, io.vertx.core.json.JsonObject.class), this);
+        registry.addConverter(new TypeConvertible<>(java.util.Map.class, io.vertx.core.json.JsonObject.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.buffer.Buffer.class, java.io.InputStream.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonArray.class, java.io.InputStream.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonObject.class, java.io.InputStream.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.buffer.Buffer.class, java.lang.String.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonArray.class, java.lang.String.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonObject.class, java.lang.String.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonArray.class, java.util.List.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.json.JsonObject.class, java.util.Map.class), this);
+        registry.addConverter(new TypeConvertible<>(io.vertx.core.buffer.Buffer.class, org.apache.camel.StreamCache.class), this);
+        
+        
+    }
+
     public TypeConverter lookup(Class<?> to, Class<?> from) {

Review Comment:
   Saving this for the subsequent patch. 



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [camel] orpiske merged pull request #11225: CAMEL-19398: refactor the core type-converter

Posted by "orpiske (via GitHub)" <gi...@apache.org>.
orpiske merged PR #11225:
URL: https://github.com/apache/camel/pull/11225


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@camel.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org