You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@rocketmq.apache.org by li...@apache.org on 2021/10/22 06:13:30 UTC

[rocketmq-client-cpp] branch main updated: Implmenet trace for message sending (#377)

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

lizhanhui pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/rocketmq-client-cpp.git


The following commit(s) were added to refs/heads/main by this push:
     new bffad4f  Implmenet trace for message sending (#377)
bffad4f is described below

commit bffad4f08a67a876441bdc7d68b6b1d74f42ddf2
Author: aaron ai <ya...@gmail.com>
AuthorDate: Fri Oct 22 14:13:24 2021 +0800

    Implmenet trace for message sending (#377)
---
 src/main/cpp/base/MixAll.cpp                    |  23 ++++
 src/main/cpp/base/include/MixAll.h              |  23 ++++
 src/main/cpp/client/TracingUtility.cpp          | 154 ------------------------
 src/main/cpp/rocketmq/ProducerImpl.cpp          |  20 ++-
 src/main/cpp/tracing/TracingUtility.cpp         |   1 +
 src/main/cpp/tracing/exporters/OtlpExporter.cpp |  26 +++-
 6 files changed, 80 insertions(+), 167 deletions(-)

diff --git a/src/main/cpp/base/MixAll.cpp b/src/main/cpp/base/MixAll.cpp
index 7006410..93d2d38 100644
--- a/src/main/cpp/base/MixAll.cpp
+++ b/src/main/cpp/base/MixAll.cpp
@@ -66,6 +66,13 @@ const uint32_t MixAll::DEFAULT_COMPRESS_BODY_THRESHOLD_ = 1024 * 1024 * 4;
 const char* MixAll::HOME_PROFILE_ENV_ = "HOME";
 const char* MixAll::MESSAGE_KEY_SEPARATOR = " ";
 
+const char* MixAll::TRACE_RESOURCE_ATTRIBUTE_KEY_TELEMETRY_SDK_LANGUAGE = "telemetry.sdk.language";
+const char* MixAll::TRACE_RESOURCE_ATTRIBUTE_VALUE_TELEMETRY_SDK_LANGUAGE = "cpp";
+
+const char* MixAll::TRACE_RESOURCE_ATTRIBUTE_KEY_HOST_NAME = "host.name";
+const char* MixAll::TRACE_RESOURCE_ATTRIBUTE_KEY_SERVICE_NAME = "service.name";
+const char* MixAll::TRACE_RESOURCE_ATTRIBUTE_VALUE_SERVICE_NAME = "rocketmq-client";
+
 // Span name list
 const char* MixAll::SPAN_NAME_SEND_MESSAGE = "SendMessage";
 const char* MixAll::SPAN_NAME_END_TRANSACTION = "EndTransaction";
@@ -77,6 +84,7 @@ const char* MixAll::SPAN_NAME_PULL_MESSAGE = "PullMessage";
 // https://github.com/open-telemetry/opentelemetry-specification
 
 // RocketMQ span attribute name list
+const char* MixAll::SPAN_ATTRIBUTE_KEY_ROCKETMQ_OPERATION = "messaging.rocketmq.operation";
 const char* MixAll::SPAN_ATTRIBUTE_KEY_ROCKETMQ_NAMESPACE = "messaging.rocketmq.namespace";
 const char* MixAll::SPAN_ATTRIBUTE_KEY_ROCKETMQ_TAG = "messaging.rocketmq.tag";
 const char* MixAll::SPAN_ATTRIBUTE_KEY_ROCKETMQ_KEYS = "messaging.rocketmq.keys";
@@ -99,6 +107,17 @@ const char* MixAll::SPAN_ATTRIBUTE_VALUE_ROCKETMQ_FIFO_MESSAGE = "fifo";
 const char* MixAll::SPAN_ATTRIBUTE_VALUE_ROCKETMQ_DELAY_MESSAGE = "delay";
 const char* MixAll::SPAN_ATTRIBUTE_VALUE_ROCKETMQ_TRANSACTION_MESSAGE = "transaction";
 
+const char* MixAll::SPAN_ATTRIBUTE_VALUE_ROCKETMQ_SEND_OPERATION = "send";
+const char* MixAll::SPAN_ATTRIBUTE_VALUE_ROCKETMQ_RECEIVE_OPERATION = "receive";
+const char* MixAll::SPAN_ATTRIBUTE_VALUE_ROCKETMQ_PULL_OPERATION = "pull";
+const char* MixAll::SPAN_ATTRIBUTE_VALUE_ROCKETMQ_AWAIT_OPERATION = "await";
+const char* MixAll::SPAN_ATTRIBUTE_VALUE_ROCKETMQ_PROCESS_OPERATION = "process";
+const char* MixAll::SPAN_ATTRIBUTE_VALUE_ROCKETMQ_ACK_OPERATION = "ack";
+const char* MixAll::SPAN_ATTRIBUTE_VALUE_ROCKETMQ_NACK_OPERATION = "nack";
+const char* MixAll::SPAN_ATTRIBUTE_VALUE_ROCKETMQ_COMMIT_OPERATION = "commit";
+const char* MixAll::SPAN_ATTRIBUTE_VALUE_ROCKETMQ_ROLLBACK_OPERATION = "rollback";
+const char* MixAll::SPAN_ATTRIBUTE_VALUE_ROCKETMQ_DLQ_OPERATION = "dlq";
+
 // Messaging span attribute name list
 const char* MixAll::SPAN_ATTRIBUTE_KEY_MESSAGING_SYSTEM = "messaging.system";
 const char* MixAll::SPAN_ATTRIBUTE_KEY_MESSAGING_DESTINATION = "messaging.destination";
@@ -110,6 +129,10 @@ const char* MixAll::SPAN_ATTRIBUTE_KEY_MESSAGING_ID = "messaging.message_id";
 const char* MixAll::SPAN_ATTRIBUTE_KEY_MESSAGING_PAYLOAD_SIZE_BYTES = "messaging.message_payload_size_bytes";
 const char* MixAll::SPAN_ATTRIBUTE_KEY_MESSAGING_OPERATION = "messaging.operation";
 
+const char* MixAll::SPAN_ATTRIBUTE_VALUE_MESSAGING_SEND_OPERATION = "send";
+const char* MixAll::SPAN_ATTRIBUTE_VALUE_MESSAGING_RECEIVE_OPERATION = "receive";
+const char* MixAll::SPAN_ATTRIBUTE_VALUE_MESSAGING_PROCESS_OPERATION = "process";
+
 const char* MixAll::SPAN_ATTRIBUTE_KEY_HOST_NAME = "host.name";
 const char* MixAll::SPAN_ATTRIBUTE_KEY_TRANSACTION_RESOLUTION = "commitAction";
 
diff --git a/src/main/cpp/base/include/MixAll.h b/src/main/cpp/base/include/MixAll.h
index 9a82b30..8a47ecf 100644
--- a/src/main/cpp/base/include/MixAll.h
+++ b/src/main/cpp/base/include/MixAll.h
@@ -61,6 +61,13 @@ public:
 
   static const char* MESSAGE_KEY_SEPARATOR;
 
+  static const char* TRACE_RESOURCE_ATTRIBUTE_KEY_TELEMETRY_SDK_LANGUAGE;
+  static const char* TRACE_RESOURCE_ATTRIBUTE_VALUE_TELEMETRY_SDK_LANGUAGE;
+
+  static const char* TRACE_RESOURCE_ATTRIBUTE_KEY_HOST_NAME;
+  static const char* TRACE_RESOURCE_ATTRIBUTE_KEY_SERVICE_NAME;
+  static const char* TRACE_RESOURCE_ATTRIBUTE_VALUE_SERVICE_NAME;
+
   // Tracing span name list
   static const char* SPAN_NAME_SEND_MESSAGE;
   static const char* SPAN_NAME_END_TRANSACTION;
@@ -69,6 +76,7 @@ public:
   static const char* SPAN_NAME_PULL_MESSAGE;
 
   // RocketMQ span attribute name list
+  static const char* SPAN_ATTRIBUTE_KEY_ROCKETMQ_OPERATION;
   static const char* SPAN_ATTRIBUTE_KEY_ROCKETMQ_NAMESPACE;
   static const char* SPAN_ATTRIBUTE_KEY_ROCKETMQ_TAG;
   static const char* SPAN_ATTRIBUTE_KEY_ROCKETMQ_KEYS;
@@ -91,6 +99,17 @@ public:
   static const char* SPAN_ATTRIBUTE_VALUE_ROCKETMQ_DELAY_MESSAGE;
   static const char* SPAN_ATTRIBUTE_VALUE_ROCKETMQ_TRANSACTION_MESSAGE;
 
+  static const char* SPAN_ATTRIBUTE_VALUE_ROCKETMQ_SEND_OPERATION;
+  static const char* SPAN_ATTRIBUTE_VALUE_ROCKETMQ_RECEIVE_OPERATION;
+  static const char* SPAN_ATTRIBUTE_VALUE_ROCKETMQ_PULL_OPERATION;
+  static const char* SPAN_ATTRIBUTE_VALUE_ROCKETMQ_AWAIT_OPERATION;
+  static const char* SPAN_ATTRIBUTE_VALUE_ROCKETMQ_PROCESS_OPERATION;
+  static const char* SPAN_ATTRIBUTE_VALUE_ROCKETMQ_ACK_OPERATION;
+  static const char* SPAN_ATTRIBUTE_VALUE_ROCKETMQ_NACK_OPERATION;
+  static const char* SPAN_ATTRIBUTE_VALUE_ROCKETMQ_COMMIT_OPERATION;
+  static const char* SPAN_ATTRIBUTE_VALUE_ROCKETMQ_ROLLBACK_OPERATION;
+  static const char* SPAN_ATTRIBUTE_VALUE_ROCKETMQ_DLQ_OPERATION;
+
   // Messaging attribute name list
   static const char* SPAN_ATTRIBUTE_KEY_MESSAGING_SYSTEM;
   static const char* SPAN_ATTRIBUTE_KEY_MESSAGING_DESTINATION;
@@ -102,6 +121,10 @@ public:
   static const char* SPAN_ATTRIBUTE_KEY_MESSAGING_PAYLOAD_SIZE_BYTES;
   static const char* SPAN_ATTRIBUTE_KEY_MESSAGING_OPERATION;
 
+  static const char* SPAN_ATTRIBUTE_VALUE_MESSAGING_SEND_OPERATION;
+  static const char* SPAN_ATTRIBUTE_VALUE_MESSAGING_RECEIVE_OPERATION;
+  static const char* SPAN_ATTRIBUTE_VALUE_MESSAGING_PROCESS_OPERATION;
+
   static const char* SPAN_ATTRIBUTE_KEY_HOST_NAME;
   static const char* SPAN_ATTRIBUTE_KEY_TRANSACTION_RESOLUTION;
 
diff --git a/src/main/cpp/client/TracingUtility.cpp b/src/main/cpp/client/TracingUtility.cpp
deleted file mode 100644
index b2f2161..0000000
--- a/src/main/cpp/client/TracingUtility.cpp
+++ /dev/null
@@ -1,154 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *     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.
- */
-#ifdef ENABLE_TRACING
-#include "TracingUtility.h"
-#include <utility>
-
-ROCKETMQ_NAMESPACE_BEGIN
-
-TracingUtility& TracingUtility::get() {
-  static TracingUtility tracing_utility;
-  return tracing_utility;
-}
-
-const std::string TracingUtility::INVALID_TRACE_ID = "00000000000000000000000000000000";
-const std::string TracingUtility::INVALID_SPAN_ID = "0000000000000000";
-
-uint8_t TracingUtility::hexToInt(char c) {
-  if (c >= '0' && c <= '9') {
-    return (int)(c - '0');
-  } else if (c >= 'a' && c <= 'f') {
-    return (int)(c - 'a' + 10);
-  } else if (c >= 'A' && c <= 'F') {
-    return (int)(c - 'A' + 10);
-  } else {
-    return -1;
-  }
-}
-
-void TracingUtility::generateHexFromString(const std::string& string, int bytes, uint8_t* buf) {
-  const char* str_id = string.data();
-  for (int i = 0; i < bytes; i++) {
-    int tmp = hexToInt(str_id[i]);
-    if (tmp < 0) {
-      for (int j = 0; j < bytes / 2; j++) {
-        buf[j] = 0;
-      }
-      return;
-    }
-    if (i % 2 == 0) {
-      buf[i / 2] = tmp * 16;
-    } else {
-      buf[i / 2] += tmp;
-    }
-  }
-}
-
-trace::TraceId TracingUtility::generateTraceIdFromString(const std::string& trace_id) {
-  int trace_id_len = kHeaderElementLengths[1];
-  uint8_t buf[kTraceIdBytes / 2];
-  uint8_t* b_ptr = buf;
-  TracingUtility::generateHexFromString(trace_id, trace_id_len, b_ptr);
-  return trace::TraceId(buf);
-}
-
-bool TracingUtility::isValidHex(const std::string& str) {
-  for (char i : str) {
-    if (!(i >= '0' && i <= '9') && !(i >= 'a' && i <= 'f')) {
-      return false;
-    }
-  }
-  return true;
-}
-
-trace::SpanId TracingUtility::generateSpanIdFromString(const std::string& span_id) {
-  int span_id_len = kHeaderElementLengths[2];
-  uint8_t buf[kSpanIdBytes / 2];
-  uint8_t* b_ptr = buf;
-  generateHexFromString(span_id, span_id_len, b_ptr);
-  return trace::SpanId(buf);
-}
-
-trace::TraceFlags TracingUtility::generateTraceFlagsFromString(std::string trace_flags) {
-  if (trace_flags.length() > 2) {
-    return trace::TraceFlags(0); // check for invalid length of flags
-  }
-  int tmp1 = hexToInt(trace_flags[0]);
-  int tmp2 = hexToInt(trace_flags[1]);
-  if (tmp1 < 0 || tmp2 < 0) {
-    return trace::TraceFlags(0); // check for invalid char
-  }
-  uint8_t buf = tmp1 * 16 + tmp2;
-  return trace::TraceFlags(buf);
-}
-
-std::string TracingUtility::injectSpanContextToTraceParent(const trace::SpanContext& span_context) {
-  char trace_id[32];
-  trace::TraceId(span_context.trace_id()).ToLowerBase16(trace_id);
-  char span_id[16];
-  trace::SpanId(span_context.span_id()).ToLowerBase16(span_id);
-  char trace_flags[2];
-  trace::TraceFlags(span_context.trace_flags()).ToLowerBase16(trace_flags);
-  // Note: This is only temporary replacement for appendable string
-  std::string hex_string = "00-";
-  for (char i : trace_id) {
-    hex_string.push_back(i);
-  }
-  hex_string.push_back('-');
-  for (char i : span_id) {
-    hex_string.push_back(i);
-  }
-  hex_string.push_back('-');
-  for (char trace_flag : trace_flags) {
-    hex_string.push_back(trace_flag);
-  }
-  return hex_string;
-}
-
-// Assumed that all span context is remote.
-trace::SpanContext TracingUtility::extractContextFromTraceParent(const std::string& trace_parent) {
-  if (trace_parent.length() != kHeaderSize || trace_parent[kHeaderElementLengths[0]] != '-' ||
-      trace_parent[kHeaderElementLengths[0] + kHeaderElementLengths[1] + 1] != '-' ||
-      trace_parent[kHeaderElementLengths[0] + kHeaderElementLengths[1] + kHeaderElementLengths[2] + 2] != '-') {
-    // Unresolvable trace_parent header. Returning INVALID span context.
-    return trace::SpanContext(false, false);
-  }
-  std::string version = trace_parent.substr(0, kHeaderElementLengths[0]);
-  std::string trace_id = trace_parent.substr(kHeaderElementLengths[0] + 1, kHeaderElementLengths[1]);
-  std::string span_id =
-      trace_parent.substr(kHeaderElementLengths[0] + kHeaderElementLengths[1] + 2, kHeaderElementLengths[2]);
-  std::string trace_flags =
-      trace_parent.substr(kHeaderElementLengths[0] + kHeaderElementLengths[1] + kHeaderElementLengths[2] + 3);
-
-  if (version == "ff" || trace_id == INVALID_TRACE_ID || span_id == INVALID_SPAN_ID) {
-    return trace::SpanContext(false, false);
-  }
-
-  // validate ids
-  if (!TracingUtility::isValidHex(version) || !TracingUtility::isValidHex(trace_id) ||
-      !TracingUtility::isValidHex(span_id) || !TracingUtility::isValidHex(trace_flags)) {
-    return trace::SpanContext(false, false);
-  }
-
-  trace::TraceId trace_id_obj = TracingUtility::generateTraceIdFromString(trace_id);
-  trace::SpanId span_id_obj = TracingUtility::generateSpanIdFromString(span_id);
-  trace::TraceFlags trace_flags_obj = generateTraceFlagsFromString(trace_flags);
-  return trace::SpanContext(trace_id_obj, span_id_obj, trace_flags_obj, true);
-}
-
-ROCKETMQ_NAMESPACE_END
-#endif
\ No newline at end of file
diff --git a/src/main/cpp/rocketmq/ProducerImpl.cpp b/src/main/cpp/rocketmq/ProducerImpl.cpp
index 5b51fcd..870bf48 100644
--- a/src/main/cpp/rocketmq/ProducerImpl.cpp
+++ b/src/main/cpp/rocketmq/ProducerImpl.cpp
@@ -321,21 +321,18 @@ void ProducerImpl::sendImpl(RetrySendCallback* callback) {
     auto span_context = opencensus::trace::propagation::FromTraceParentHeader(message.traceContext());
 
     auto span = opencensus::trace::Span::BlankSpan();
+    std::string span_name =
+        resourceNamespace() + "/" + message.getTopic() + " " + MixAll::SPAN_ATTRIBUTE_VALUE_ROCKETMQ_SEND_OPERATION;
     if (span_context.IsValid()) {
-      span = opencensus::trace::Span::StartSpanWithRemoteParent(MixAll::SPAN_NAME_SEND_MESSAGE, span_context,
-                                                                {&Samplers::always()});
+      span = opencensus::trace::Span::StartSpanWithRemoteParent(span_name, span_context, {&Samplers::always()});
     } else {
-      span = opencensus::trace::Span::StartSpan(MixAll::SPAN_NAME_SEND_MESSAGE, nullptr, {&Samplers::always()});
+      span = opencensus::trace::Span::StartSpan(span_name, nullptr, {&Samplers::always()});
     }
+    span.AddAttribute(MixAll::SPAN_ATTRIBUTE_KEY_MESSAGING_OPERATION,
+                      MixAll::SPAN_ATTRIBUTE_VALUE_ROCKETMQ_SEND_OPERATION);
+    span.AddAttribute(MixAll::SPAN_ATTRIBUTE_KEY_ROCKETMQ_OPERATION,
+                      MixAll::SPAN_ATTRIBUTE_VALUE_MESSAGING_SEND_OPERATION);
     TracingUtility::addUniversalSpanAttributes(message, *this, span);
-
-    span.AddAttribute(MixAll::SPAN_ATTRIBUTE_KEY_ROCKETMQ_ACCESS_KEY,
-                      credentialsProvider()->getCredentials().accessKey());
-    span.AddAttribute(MixAll::SPAN_ATTRIBUTE_KEY_ROCKETMQ_NAMESPACE, resourceNamespace());
-    span.AddAttribute(MixAll::SPAN_ATTRIBUTE_KEY_MESSAGING_DESTINATION, message.getTopic());
-    span.AddAttribute(MixAll::SPAN_ATTRIBUTE_KEY_MESSAGING_ID, message.getMsgId());
-    span.AddAttribute(MixAll::SPAN_ATTRIBUTE_KEY_ROCKETMQ_CLIENT_GROUP, getGroupName());
-    span.AddAttribute(MixAll::SPAN_ATTRIBUTE_KEY_ROCKETMQ_TAG, message.getTags());
     const auto& keys = callback->message().getKeys();
     if (!keys.empty()) {
       span.AddAttribute(MixAll::SPAN_ATTRIBUTE_KEY_ROCKETMQ_KEYS,
@@ -343,7 +340,6 @@ void ProducerImpl::sendImpl(RetrySendCallback* callback) {
     }
     // Note: attempt-time is 0-based
     span.AddAttribute(MixAll::SPAN_ATTRIBUTE_KEY_ROCKETMQ_ATTEMPT, 1 + callback->attemptTime());
-    span.AddAttribute(MixAll::SPAN_ATTRIBUTE_KEY_HOST_NAME, UtilAll::hostname());
 
     if (message.deliveryTimestamp() != absl::ToChronoTime(absl::UnixEpoch())) {
       span.AddAttribute(MixAll::SPAN_ATTRIBUTE_KEY_ROCKETMQ_DELIVERY_TIMESTAMP,
diff --git a/src/main/cpp/tracing/TracingUtility.cpp b/src/main/cpp/tracing/TracingUtility.cpp
index 25805db..6446e3f 100644
--- a/src/main/cpp/tracing/TracingUtility.cpp
+++ b/src/main/cpp/tracing/TracingUtility.cpp
@@ -55,6 +55,7 @@ void TracingUtility::addUniversalSpanAttributes(const MQMessage& message, Client
   span.AddAttribute(MixAll::SPAN_ATTRIBUTE_KEY_MESSAGING_DESTINATION, message.getTopic());
   span.AddAttribute(MixAll::SPAN_ATTRIBUTE_KEY_MESSAGING_DESTINATION_KIND,
                     MixAll::SPAN_ATTRIBUTE_VALUE_DESTINATION_KIND);
+  span.AddAttribute(MixAll::SPAN_ATTRIBUTE_KEY_MESSAGING_PROTOCOL, MixAll::SPAN_ATTRIBUTE_VALUE_MESSAGING_PROTOCOL);
   span.AddAttribute(MixAll::SPAN_ATTRIBUTE_KEY_MESSAGING_PROTOCOL_VERSION,
                     MixAll::SPAN_ATTRIBUTE_VALUE_MESSAGING_PROTOCOL_VERSION);
   //   span.AddAttribute(MixAll::SPAN_ATTRIBUTE_KEY_MESSAGING_URL, "abc")
diff --git a/src/main/cpp/tracing/exporters/OtlpExporter.cpp b/src/main/cpp/tracing/exporters/OtlpExporter.cpp
index 78aaa26..1ebbc22 100644
--- a/src/main/cpp/tracing/exporters/OtlpExporter.cpp
+++ b/src/main/cpp/tracing/exporters/OtlpExporter.cpp
@@ -18,6 +18,7 @@
 #include "InvocationContext.h"
 #include "MixAll.h"
 #include "Signature.h"
+#include "UtilAll.h"
 #include "fmt/format.h"
 #include "opentelemetry/proto/collector/trace/v1/trace_service.pb.h"
 #include "opentelemetry/proto/common/v1/common.pb.h"
@@ -333,6 +334,7 @@ void OtlpExporterHandler::Export(const std::vector<::opencensus::trace::exporter
           value->set_int_value(attribute.second.int_value());
           break;
       }
+      kv->set_allocated_value(value);
       item->mutable_attributes()->AddAllocated(kv);
     }
 
@@ -348,6 +350,28 @@ void OtlpExporterHandler::Export(const std::vector<::opencensus::trace::exporter
     instrument_library_span->mutable_spans()->AddAllocated(item);
   }
   resource->mutable_instrumentation_library_spans()->AddAllocated(instrument_library_span);
+
+  auto telemetry_sdk_language_kv = new common::KeyValue();
+  telemetry_sdk_language_kv->set_key(MixAll::TRACE_RESOURCE_ATTRIBUTE_KEY_TELEMETRY_SDK_LANGUAGE);
+  auto telemetry_sdk_language_value = new common::AnyValue();
+  telemetry_sdk_language_value->set_string_value(MixAll::TRACE_RESOURCE_ATTRIBUTE_VALUE_TELEMETRY_SDK_LANGUAGE);
+  telemetry_sdk_language_kv->set_allocated_value(telemetry_sdk_language_value);
+  resource->mutable_resource()->mutable_attributes()->AddAllocated(telemetry_sdk_language_kv);
+
+  auto host_name_kv = new common::KeyValue();
+  host_name_kv->set_key(MixAll::TRACE_RESOURCE_ATTRIBUTE_KEY_HOST_NAME);
+  auto host_name_value = new common::AnyValue();
+  host_name_value->set_string_value(UtilAll::hostname());
+  host_name_kv->set_allocated_value(host_name_value);
+  resource->mutable_resource()->mutable_attributes()->AddAllocated(host_name_kv);
+
+  auto service_name_kv = new common::KeyValue();
+  service_name_kv->set_key(MixAll::TRACE_RESOURCE_ATTRIBUTE_KEY_SERVICE_NAME);
+  auto service_name_value = new common::AnyValue();
+  service_name_value->set_string_value(MixAll::TRACE_RESOURCE_ATTRIBUTE_VALUE_SERVICE_NAME);
+  service_name_kv->set_allocated_value(service_name_value);
+  resource->mutable_resource()->mutable_attributes()->AddAllocated(service_name_kv);
+
   request.mutable_resource_spans()->AddAllocated(resource);
 
   auto invocation_context = new InvocationContext<collector_trace::ExportTraceServiceResponse>();
@@ -362,7 +386,7 @@ void OtlpExporterHandler::Export(const std::vector<::opencensus::trace::exporter
   }
   auto callback = [](const InvocationContext<collector_trace::ExportTraceServiceResponse>* invocation_context) {
     if (invocation_context->status.ok()) {
-      SPDLOG_DEBUG("Export tracing spans OK");
+      SPDLOG_DEBUG("Export tracing spans OK, target={}", invocation_context->remote_address);
     } else {
       SPDLOG_WARN("Failed to export tracing spans to {}, gRPC code:{}, gRPC error message: {}",
                   invocation_context->remote_address, invocation_context->status.error_code(),