You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@camel.apache.org by GitBox <gi...@apache.org> on 2020/07/18 21:22:20 UTC

[GitHub] [camel] rubenvp8510 opened a new pull request #4016: [WIP] opentelemetry support

rubenvp8510 opened a new pull request #4016:
URL: https://github.com/apache/camel/pull/4016


   [ ] Make sure there is a [JIRA issue](https://issues.apache.org/jira/browse/CAMEL) filed for the change (usually before you start working on it).  Trivial changes like typos do not require a JIRA issue.  Your pull request should address just this issue, without pulling in other changes.
   [ ] Each commit in the pull request should have a meaningful subject line and body.
   [ ] If you're unsure, you can format the pull request title like `[CAMEL-XXX] Fixes bug in camel-file component`, where you replace `CAMEL-XXX` with the appropriate JIRA issue.
   [ ] Write a pull request description that is detailed enough to understand what the pull request does, how, and why.
   [ ] Run `mvn clean install -Psourcecheck` in your module with source check enabled to make sure basic checks pass and there are no checkstyle violations. A more thorough check will be performed on your pull request automatically.
   Below are the contribution guidelines:
   https://github.com/apache/camel/blob/master/CONTRIBUTING.md


----------------------------------------------------------------
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.

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



[GitHub] [camel] oscerd commented on pull request #4016: CAMEL-15339 Opentelemetry component

Posted by GitBox <gi...@apache.org>.
oscerd commented on pull request #4016:
URL: https://github.com/apache/camel/pull/4016#issuecomment-665441636






----------------------------------------------------------------
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.

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



[GitHub] [camel] davsclaus commented on a change in pull request #4016: CAMEL-15339 Opentelemetry component

Posted by GitBox <gi...@apache.org>.
davsclaus commented on a change in pull request #4016:
URL: https://github.com/apache/camel/pull/4016#discussion_r461367222



##########
File path: components/camel-opentelemetry/src/main/java/org/apache/camel/opentelemetry/OpenTelemetrySpanAdapter.java
##########
@@ -0,0 +1,121 @@
+/*
+ * 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.opentelemetry;
+
+import java.util.EnumMap;
+import java.util.Map;
+
+import io.opentelemetry.common.AttributeValue;
+import io.opentelemetry.common.Attributes;
+import io.opentelemetry.trace.attributes.SemanticAttributes;
+import org.apache.camel.tracing.SpanAdapter;
+import org.apache.camel.tracing.Tag;
+
+public class OpenTelemetrySpanAdapter implements SpanAdapter {
+    private static final String DEFAULT_EVENT_NAME = "log";
+
+    private static EnumMap<Tag, String> tagMap = new EnumMap<>(Tag.class);
+
+    static {
+        tagMap.put(Tag.COMPONENT, "component");
+        tagMap.put(Tag.DB_TYPE, SemanticAttributes.DB_TYPE.key());
+        tagMap.put(Tag.DB_STATEMENT, SemanticAttributes.DB_STATEMENT.key());
+        tagMap.put(Tag.DB_INSTANCE, SemanticAttributes.DB_INSTANCE.key());
+        tagMap.put(Tag.HTTP_METHOD, SemanticAttributes.HTTP_METHOD.key());
+        tagMap.put(Tag.HTTP_STATUS, SemanticAttributes.HTTP_STATUS_CODE.key());
+        tagMap.put(Tag.HTTP_URL, SemanticAttributes.HTTP_URL.key());
+        tagMap.put(Tag.MESSAGE_BUS_DESTINATION, "message_bus.destination");
+    }
+
+
+    io.opentelemetry.trace.Span span;
+
+    OpenTelemetrySpanAdapter(io.opentelemetry.trace.Span span) {
+        this.span = span;
+    }
+
+    io.opentelemetry.trace.Span getOpenTelemetrySpan() {
+        return this.span;
+    }
+
+    @Override public void setComponent(String component) {
+        this.span.setAttribute("component", component);
+    }
+
+    @Override public void setError(boolean error) {
+        this.span.setAttribute("error", error);
+    }
+
+    @Override public void setTag(Tag key, String value) {
+        this.span.setAttribute(tagMap.get(key), value);
+    }
+
+    @Override public void setTag(Tag key, Number value) {
+        this.span.setAttribute(tagMap.get(key), value.intValue());
+    }
+
+    @Override public void setTag(String key, String value) {
+        this.span.setAttribute(key, value);
+    }
+
+    @Override public void setTag(String key, Number value) {
+        this.span.setAttribute(key, value.intValue());
+    }
+
+    @Override public void setTag(String key, Boolean value) {
+        this.span.setAttribute(key, value);
+    }
+
+    @Override public void log(Map<String, String> fields) {
+        span.addEvent(getEventNameFromFields(fields), convertToAttributes(fields));
+

Review comment:
       Remove empty line

##########
File path: components/camel-opentelemetry/src/main/java/org/apache/camel/opentelemetry/propagators/OpenTelemetryGetter.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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.opentelemetry.propagators;
+
+import javax.annotation.Nullable;
+
+import io.opentelemetry.context.propagation.HttpTextFormat;
+import org.apache.camel.tracing.ExtractAdapter;
+
+public class OpenTelemetryGetter implements HttpTextFormat.Getter<ExtractAdapter> {
+
+    @Nullable @Override public String get(ExtractAdapter adapter, String key) {

Review comment:
       We do not use Nullable annotations in camel (yet)

##########
File path: components/camel-opentelemetry/pom.xml
##########
@@ -0,0 +1,89 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+
+         http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.camel</groupId>
+    <artifactId>components</artifactId>
+    <version>3.5.0-SNAPSHOT</version>
+  </parent>
+
+
+  <artifactId>camel-opentelemetry</artifactId>
+  <packaging>jar</packaging>
+  <name>Camel :: OpenTelemetry</name>
+  <description>Distributed tracing using OpenTelemetry</description>
+
+  <properties>
+    <firstVersion>3.5.0</firstVersion>
+    <label>monitoring,microservice</label>
+    <title>OpenTelemetry</title>
+    <opentracing-agent.lib>${project.build.directory}/lib</opentracing-agent.lib>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-support</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-tracing</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-netty-shaded</artifactId>
+      <version>${grpc-version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-sdk</artifactId>
+      <version>${opentelemetry-version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-api</artifactId>
+      <version>${opentelemetry-version}</version>
+    </dependency>
+    <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-exporters-otlp</artifactId>
+      <version>${opentelemetry-version}</version>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.logging.log4j</groupId>
+      <artifactId>log4j-slf4j-impl</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-test-spring-junit5</artifactId>
+      <scope>test</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-exporters-inmemory</artifactId>
+      <version>0.6.0</version>

Review comment:
       Use version placeholder

##########
File path: components/camel-tracing/src/main/java/org/apache/camel/tracing/Tracer.java
##########
@@ -0,0 +1,345 @@
+/*
+ * 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.tracing;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.ServiceLoader;
+import java.util.Set;
+
+import org.apache.camel.CamelContext;
+import org.apache.camel.CamelContextAware;
+import org.apache.camel.Component;
+import org.apache.camel.Endpoint;
+import org.apache.camel.Exchange;
+import org.apache.camel.ExtendedCamelContext;
+import org.apache.camel.NamedNode;
+import org.apache.camel.Route;
+import org.apache.camel.RuntimeCamelException;
+import org.apache.camel.StaticService;
+import org.apache.camel.spi.CamelEvent;
+import org.apache.camel.spi.CamelLogger;
+import org.apache.camel.spi.InterceptStrategy;
+import org.apache.camel.spi.LogListener;
+import org.apache.camel.spi.RoutePolicy;
+import org.apache.camel.spi.RoutePolicyFactory;
+import org.apache.camel.support.DefaultEndpoint;
+import org.apache.camel.support.EndpointHelper;
+import org.apache.camel.support.EventNotifierSupport;
+import org.apache.camel.support.RoutePolicySupport;
+import org.apache.camel.support.service.ServiceHelper;
+import org.apache.camel.support.service.ServiceSupport;
+import org.apache.camel.tracing.decorators.AbstractInternalSpanDecorator;
+import org.apache.camel.util.ObjectHelper;
+import org.apache.camel.util.StringHelper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public abstract class Tracer extends ServiceSupport implements RoutePolicyFactory, StaticService, CamelContextAware {
+    protected static final Map<String, SpanDecorator> DECORATORS = new HashMap<>();
+    private static final Logger LOG = LoggerFactory.getLogger(Tracer.class);
+
+    static {
+        ServiceLoader.load(SpanDecorator.class).forEach(d -> {
+            SpanDecorator existing = DECORATORS.get(d.getComponent());
+            // Add span decorator if no existing decorator for the component,
+            // or if derived from the existing decorator's class, allowing
+            // custom decorators to be added if they extend the standard
+            // decorators
+            if (existing == null || existing.getClass().isInstance(d)) {
+                DECORATORS.put(d.getComponent(), d);
+            }
+        });
+    }
+
+    private final TracingLogListener logListener = new TracingLogListener();
+    private final TracingEventNotifier eventNotifier = new TracingEventNotifier();
+    private Set<String> excludePatterns = new HashSet<>(0);
+    private InterceptStrategy tracingStrategy;
+    private boolean encoding;
+    private CamelContext camelContext;
+
+    protected abstract void initTracer();
+
+    protected abstract SpanAdapter startSendingEventSpan(String operationName, SpanKind kind, SpanAdapter parent);
+
+    protected abstract SpanAdapter startExchangeBeginSpan(String operationName, SpanKind kind, SpanAdapter parent);
+
+    protected abstract void finishSpan(SpanAdapter span);
+
+    protected abstract void inject(SpanAdapter span, InjectAdapter adapter);
+
+    /**
+     * Returns the currently used tracing strategy which is responsible for tracking invoked EIP or
+     * beans.
+     *
+     * @return The currently used tracing strategy
+     */
+    public InterceptStrategy getTracingStrategy() {
+        return tracingStrategy;
+    }
+
+    /**
+     * Specifies the instance responsible for tracking invoked EIP and beans with OpenTracing.
+     *
+     * @param tracingStrategy The instance which tracks invoked EIP and beans
+     */
+    public void setTracingStrategy(InterceptStrategy tracingStrategy) {
+        this.tracingStrategy = tracingStrategy;
+    }
+
+    public void addDecorator(SpanDecorator decorator) {
+        DECORATORS.put(decorator.getComponent(), decorator);
+    }
+
+    @Override
+    public CamelContext getCamelContext() {
+        return camelContext;
+    }
+
+    @Override
+    public void setCamelContext(CamelContext camelContext) {
+        this.camelContext = camelContext;
+    }
+
+    public Set<String> getExcludePatterns() {
+        return excludePatterns;
+    }
+
+    public void setExcludePatterns(Set<String> excludePatterns) {
+        this.excludePatterns = excludePatterns;
+    }
+
+    public boolean isEncoding() {
+        return encoding;
+    }
+
+    public void setEncoding(boolean encoding) {
+        this.encoding = encoding;
+    }
+
+    /**
+     * Adds an exclude pattern that will disable tracing for Camel messages that
+     * matches the pattern.
+     *
+     * @param pattern the pattern such as route id, endpoint url
+     */
+    public void addExcludePattern(String pattern) {
+        excludePatterns.add(pattern);
+    }
+
+    @Override public RoutePolicy createRoutePolicy(CamelContext camelContext, String routeId, NamedNode route) {
+        init(camelContext);
+        return new TracingRoutePolicy();
+
+    }
+
+    /**
+     * Registers this {@link Tracer} on the {@link CamelContext} if
+     * not already registered.
+     */
+    public void init(CamelContext camelContext) {
+        if (!camelContext.hasService(this)) {
+            try {
+                // start this service eager so we init before Camel is starting up
+                camelContext.addService(this, true, true);
+            } catch (Exception e) {
+                throw RuntimeCamelException.wrapRuntimeCamelException(e);
+            }
+        }
+    }
+
+    @Override protected void doInit() throws Exception {
+        ObjectHelper.notNull(camelContext, "CamelContext", this);
+
+        camelContext.getManagementStrategy().addEventNotifier(eventNotifier);
+        if (!camelContext.getRoutePolicyFactories().contains(this)) {
+            camelContext.addRoutePolicyFactory(this);
+        }
+        camelContext.adapt(ExtendedCamelContext.class).addLogListener(logListener);
+
+        if (tracingStrategy != null) {
+            camelContext.adapt(ExtendedCamelContext.class).addInterceptStrategy(tracingStrategy);
+        }
+        initTracer();
+        ServiceHelper.startService(eventNotifier);
+    }
+
+    @Override
+    protected void doShutdown() throws Exception {
+        // stop event notifier
+        camelContext.getManagementStrategy().removeEventNotifier(eventNotifier);
+        ServiceHelper.stopService(eventNotifier);
+
+        // remove route policy
+        camelContext.getRoutePolicyFactories().remove(this);
+    }
+
+    protected SpanDecorator getSpanDecorator(Endpoint endpoint) {
+        SpanDecorator sd = null;
+
+        String uri = endpoint.getEndpointUri();
+        String splitURI[] = StringHelper.splitOnCharacter(uri, ":", 2);
+        if (splitURI[1] != null) {
+            String scheme = splitURI[0];
+            sd = DECORATORS.get(scheme);
+        }
+        if (sd == null) {
+            // okay there was no decorator found via component name (scheme), then try FQN
+            if (endpoint instanceof DefaultEndpoint) {
+                Component comp = ((DefaultEndpoint) endpoint).getComponent();
+                String fqn = comp.getClass().getName();
+                // lookup via FQN
+                sd = DECORATORS.values().stream().filter(d -> fqn.equals(d.getComponentClassName())).findFirst()
+                .orElse(null);
+            }
+        }
+        if (sd == null) {
+            sd = SpanDecorator.DEFAULT;
+        }
+
+        return sd;
+    }
+
+    private boolean isExcluded(Exchange exchange, Endpoint endpoint) {
+        String url = endpoint.getEndpointUri();
+        if (url != null && !excludePatterns.isEmpty()) {
+            for (String pattern : excludePatterns) {
+                if (EndpointHelper.matchEndpoint(exchange.getContext(), url, pattern)) {
+                    return true;
+                }
+            }
+        }
+        return false;
+    }
+
+    private final class TracingEventNotifier extends EventNotifierSupport {

Review comment:
       In the logging below change OpenTracing to Tracing




----------------------------------------------------------------
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.

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



[GitHub] [camel] oscerd merged pull request #4016: CAMEL-15339 Opentelemetry component

Posted by GitBox <gi...@apache.org>.
oscerd merged pull request #4016:
URL: https://github.com/apache/camel/pull/4016


   


----------------------------------------------------------------
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.

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



[GitHub] [camel] oscerd commented on a change in pull request #4016: [WIP] opentelemetry support

Posted by GitBox <gi...@apache.org>.
oscerd commented on a change in pull request #4016:
URL: https://github.com/apache/camel/pull/4016#discussion_r457096530



##########
File path: components/camel-opentelemetry/pom.xml
##########
@@ -0,0 +1,73 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+
+         http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.camel</groupId>
+    <artifactId>components</artifactId>
+    <version>3.5.0-SNAPSHOT</version>
+  </parent>
+
+
+  <artifactId>camel-opentelemetry</artifactId>
+  <packaging>jar</packaging>
+  <name>Camel :: OpenTelemetry</name>
+  <description>Distributed tracing using OpenTelemetry</description>
+
+  <properties>
+    <firstVersion>3.5.0</firstVersion>
+    <label>monitoring,microservice</label>
+    <title>OpenTelemetry</title>
+    <opentracing-agent.lib>${project.build.directory}/lib</opentracing-agent.lib>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-support</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-tracing</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-netty-shaded</artifactId>
+      <version>1.28.0</version>

Review comment:
       you should use the version from parent POM

##########
File path: components/camel-opentelemetry/pom.xml
##########
@@ -0,0 +1,73 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+
+    Licensed to the Apache Software Foundation (ASF) under one or more
+    contributor license agreements.  See the NOTICE file distributed with
+    this work for additional information regarding copyright ownership.
+    The ASF licenses this file to You under the Apache License, Version 2.0
+    (the "License"); you may not use this file except in compliance with
+    the License.  You may obtain a copy of the License at
+
+         http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing, software
+    distributed under the License is distributed on an "AS IS" BASIS,
+    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+    See the License for the specific language governing permissions and
+    limitations under the License.
+
+-->
+<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance" xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
+  <modelVersion>4.0.0</modelVersion>
+
+  <parent>
+    <groupId>org.apache.camel</groupId>
+    <artifactId>components</artifactId>
+    <version>3.5.0-SNAPSHOT</version>
+  </parent>
+
+
+  <artifactId>camel-opentelemetry</artifactId>
+  <packaging>jar</packaging>
+  <name>Camel :: OpenTelemetry</name>
+  <description>Distributed tracing using OpenTelemetry</description>
+
+  <properties>
+    <firstVersion>3.5.0</firstVersion>
+    <label>monitoring,microservice</label>
+    <title>OpenTelemetry</title>
+    <opentracing-agent.lib>${project.build.directory}/lib</opentracing-agent.lib>
+  </properties>
+
+  <dependencies>
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-support</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>org.apache.camel</groupId>
+      <artifactId>camel-tracing</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-netty-shaded</artifactId>
+      <version>1.28.0</version>
+    </dependency>
+    <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-sdk</artifactId>
+      <version>0.6.0</version>

Review comment:
       It should be declared in the parent POM

##########
File path: components/camel-opentelemetry/src/main/java/org/apache/camel/opentelemetry/OpenTelemetrySpanWrapper.java
##########
@@ -0,0 +1,126 @@
+/*
+ * 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.opentelemetry;
+
+import java.util.*;

Review comment:
       Please use the explicit imports, no * operator

##########
File path: components/camel-opentracing/src/test/java/org/apache/camel/opentracing/CamelOpenTracingTestSupport.java
##########
@@ -14,16 +14,11 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+
 package org.apache.camel.opentracing;
 
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
+import java.net.*;

Review comment:
       Please use the explicit imports.

##########
File path: components/camel-opentracing/src/main/java/org/apache/camel/opentracing/GetBaggageProcessor.java
##########
@@ -24,6 +24,7 @@
 import org.apache.camel.spi.IdAware;

Review comment:
       This will probably create conflicts with this #4001 once we merge 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.

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



[GitHub] [camel] rubenvp8510 commented on a change in pull request #4016: [WIP] opentelemetry support

Posted by GitBox <gi...@apache.org>.
rubenvp8510 commented on a change in pull request #4016:
URL: https://github.com/apache/camel/pull/4016#discussion_r457546647



##########
File path: components/camel-opentracing/src/main/java/org/apache/camel/opentracing/GetBaggageProcessor.java
##########
@@ -24,6 +24,7 @@
 import org.apache.camel.spi.IdAware;

Review comment:
       I see :( I'll fix those conflicts.




----------------------------------------------------------------
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.

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



[GitHub] [camel] davsclaus commented on pull request #4016: CAMEL-15339 Opentelemetry component

Posted by GitBox <gi...@apache.org>.
davsclaus commented on pull request #4016:
URL: https://github.com/apache/camel/pull/4016#issuecomment-665438331


   After this PR we should likely get camel-zipkin to use camel-tracing as well.
   And then we need to update the camel karaf features also


----------------------------------------------------------------
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.

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



[GitHub] [camel] jam01 commented on pull request #4016: CAMEL-15339 Opentelemetry component

Posted by GitBox <gi...@apache.org>.
jam01 commented on pull request #4016:
URL: https://github.com/apache/camel/pull/4016#issuecomment-665449216


   There are some changes from #4044 that'll have to reworked into the new tracing component, I believe.


----------------------------------------------------------------
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.

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



[GitHub] [camel] rubenvp8510 commented on pull request #4016: [WIP] opentelemetry support

Posted by GitBox <gi...@apache.org>.
rubenvp8510 commented on pull request #4016:
URL: https://github.com/apache/camel/pull/4016#issuecomment-661043003


   > Please run with the sourcecheck profile enabled because the indentation in parent pom seems to be wrong and in the components you'll get a bunch of errors to fix in terms of codestyle.
   > 
   > Also, what is the purpose of this PR? I see a camel-tracing component introduced. This seems to be a big change and refactoring so it probably deserve a JIRA issue with some explanation of what is the plan and what will be in the common module and what not.
   > 
   > Thanks.
   
   Hi, Thanks for the review, the purpose is to support opentelemetry, doing the same as opentracing does but using opentelemetry instead.
   
   The purpose of the refactorization is for reuse the span decorators. I'll check the code style, but I ran the `mvn clean install -Psourcecheck `locally and it passed. :/


----------------------------------------------------------------
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.

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