You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@skywalking.apache.org by GitBox <gi...@apache.org> on 2022/07/15 15:17:07 UTC

[GitHub] [skywalking-rust] jmjoy opened a new pull request, #28: Auto finalize context and span when dropped.

jmjoy opened a new pull request, #28:
URL: https://github.com/apache/skywalking-rust/pull/28

   Major changes have been made to automatically finalize context and span (more Rust-style).
   
   1. In order to automatically finalize when `Drop` is implemented, the creator needs to be referenced on the struct, such as context reference tracer, span reference context, and reference counting needs to be used. Since async task usually needs to implement `Send` (such as `tokio::spawn`), it needs to use the type `Arc<Mutex<T>>`. The `Mutex` here only uses `try_lock`, so it will not cause blocking.
   2. Removed `TimeFetcher` because it is only used for integration testing, and conditional compilation is a better choice for mocking.
   3. Added global tracer and related methods.


-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-rust] wu-sheng commented on a diff in pull request #28: Auto finalize context and span when dropped.

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on code in PR #28:
URL: https://github.com/apache/skywalking-rust/pull/28#discussion_r922592533


##########
README.md:
##########
@@ -37,38 +37,38 @@ context after the span finished.
 use skywalking::context::tracer::Tracer;
 use skywalking::reporter::grpc::GrpcReporter;
 use std::error::Error;
-use std::sync::Arc;
 use tokio::signal;
 
-async fn handle_request(tracer: Arc<Tracer<GrpcReporter>>) {
+async fn handle_request(tracer: Tracer) {
     let mut ctx = tracer.create_trace_context();
 
     {
-        // Generate an Entry Span when a request
-        // is received. An Entry Span is generated only once per context.
-        let span = ctx.create_entry_span("op1").unwrap();
+        // Generate an Entry Span when a request is received.
+        // An Entry Span is generated only once per context.
+        // You should assign a variable name to guard the span not be dropped immediately.

Review Comment:
   ```suggestion
           // Assign a variable name to guard the span not to be dropped immediately.
   ```



-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-rust] wu-sheng commented on a diff in pull request #28: Auto finalize context and span when dropped.

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on code in PR #28:
URL: https://github.com/apache/skywalking-rust/pull/28#discussion_r922593209


##########
src/context/propagation/encoder.rs:
##########
@@ -23,11 +23,11 @@ pub fn encode_propagation(context: &TracingContext, endpoint: &str, address: &st
     let mut res = String::new();
 
     res += "1-";
-    res += format!("{}-", encode(context.trace_id.to_string())).as_str();
-    res += format!("{}-", encode(context.trace_segment_id.to_string())).as_str();
+    res += format!("{}-", encode(context.trace_id())).as_str();
+    res += format!("{}-", encode(context.trace_segment_id())).as_str();
     res += format!("{}-", context.peek_active_span_id().unwrap_or(0)).as_str();
-    res += format!("{}-", encode(context.service.as_str())).as_str();
-    res += format!("{}-", encode(context.service_instance.as_str())).as_str();
+    res += format!("{}-", encode(context.service())).as_str();
+    res += format!("{}-", encode(context.service_instance())).as_str();
     res += format!("{}-", encode(endpoint)).as_str();
     res += &encode(address);

Review Comment:
   Curiously, is multi `+` to link string and format multiple times good in performance? Or should we format once to use one format to link all parts?



-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-rust] jmjoy commented on a diff in pull request #28: Auto finalize context and span when dropped.

Posted by GitBox <gi...@apache.org>.
jmjoy commented on code in PR #28:
URL: https://github.com/apache/skywalking-rust/pull/28#discussion_r922267879


##########
Cargo.toml:
##########
@@ -32,10 +32,14 @@ license = "Apache-2.0"
 homepage = "https://skywalking.apache.org/"
 repository = "https://github.com/apache/skywalking-rust"
 
+[features]
+mock = []  # For internal integration testing only, do not use.
+
 [dependencies]
 async-stream = "0.3.3"
 base64 = "0.13.0"
 bytes = "1.1.0"
+cfg-if = "1.0.0"

Review Comment:
   Dependency `cfg-if`already exists, here re-import it.



##########
src/context/span.rs:
##########
@@ -0,0 +1,190 @@
+// 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.
+//
+
+use crate::skywalking_proto::v3::{KeyStringValuePair, Log, SpanLayer, SpanObject, SpanType};
+use std::fmt::Formatter;
+
+use super::{
+    system_time::{fetch_time, TimePeriod},
+    trace_context::{TracingContext, WeakTracingContext},
+};
+
+/// Span is a concept that represents trace information for a single RPC.
+/// The Rust SDK supports Entry Span to represent inbound to a service
+/// and Exit Span to represent outbound from a service.
+///
+/// # Example
+///
+/// ```
+/// use skywalking::context::tracer::Tracer;
+///
+/// async fn handle_request(tracer: Tracer) {
+///     let mut ctx = tracer.create_trace_context();
+///
+///     {
+///         // Generate an Entry Span when a request is received.
+///         // An Entry Span is generated only once per context.
+///         // You should assign a variable name to guard the span not be dropped immediately.
+///         let _span = ctx.create_entry_span("op1");
+///
+///         // Something...
+///
+///         {
+///             // Generates an Exit Span when executing an RPC.
+///             let _span2 = ctx.create_exit_span("op2", "remote_peer");
+///
+///             // Something...
+///
+///             // Auto close span2 when dropped.
+///         }
+///
+///         // Auto close span when dropped.
+///     }
+///
+///     // Auto report ctx when dropped.
+/// }
+/// ```
+#[derive(Clone)]
+pub struct Span {

Review Comment:
   Move `Span` to a single mod.



##########
src/context/tracer.rs:
##########
@@ -16,60 +16,123 @@
 
 use super::propagation::context::PropagationContext;
 use crate::{
-    context::trace_context::TracingContext, reporter::Reporter, skywalking_proto::v3::SegmentObject,
+    context::trace_context::TracingContext, reporter::DynReporter, reporter::Reporter,
+    skywalking_proto::v3::SegmentObject,
 };
-use futures_util::stream;
 use std::future::Future;
+use std::sync::Weak;
 use std::{collections::LinkedList, sync::Arc};
+use tokio::sync::OnceCell;
 use tokio::{
     sync::{
-        mpsc::{self, UnboundedReceiver},
+        mpsc::{self},
         Mutex,
     },
     task::JoinHandle,
 };
 
-/// Skywalking tracer.
-pub struct Tracer<R: Reporter + Send + Sync + 'static> {
+static GLOBAL_TRACER: OnceCell<Tracer> = OnceCell::const_new();

Review Comment:
   Added global tracer and related methods.



##########
src/context/system_time.rs:
##########
@@ -14,17 +14,30 @@
 // limitations under the License.
 //
 
-use crate::common::time::TimeFetcher;
-use std::time::{SystemTime, UNIX_EPOCH};
+use cfg_if::cfg_if;
 
-#[derive(Default)]
-pub struct UnixTimeStampFetcher {}
+pub(crate) enum TimePeriod {
+    Start,
+    Log,
+    End,
+}
 
-impl TimeFetcher for UnixTimeStampFetcher {
-    fn get(&self) -> i64 {
-        SystemTime::now()
-            .duration_since(UNIX_EPOCH)
-            .unwrap()
-            .as_millis() as i64
+cfg_if! {
+    if #[cfg(feature = "mock")] {

Review Comment:
   Use conditional compilation to mock fetching time.



##########
src/context/trace_context.rs:
##########
@@ -14,387 +14,392 @@
 // limitations under the License.
 //
 
-use crate::common::random_generator::RandomGenerator;
-use crate::common::time::TimeFetcher;
-use crate::context::propagation::context::PropagationContext;
-use crate::skywalking_proto::v3::{
-    KeyStringValuePair, Log, RefType, SegmentObject, SegmentReference, SpanLayer, SpanObject,
-    SpanType,
+use super::{
+    span::Span,
+    system_time::{fetch_time, TimePeriod},
+    tracer::{Tracer, WeakTracer},
 };
-use std::collections::LinkedList;
-use std::fmt::Formatter;
-use std::sync::Arc;
-
-use super::system_time::UnixTimeStampFetcher;
-
-/// Span is a concept that represents trace information for a single RPC.
-/// The Rust SDK supports Entry Span to represent inbound to a service
-/// and Exit Span to represent outbound from a service.
-///
-/// # Example
-///
-/// ```
-/// use skywalking::context::trace_context::TracingContext;
-///
-/// async fn handle_request() {
-///     let mut ctx = TracingContext::default("svc", "ins");
-///     {
-///         // Generate an Entry Span when a request
-///         // is received. An Entry Span is generated only once per context.
-///         let span = ctx.create_entry_span("operation1").unwrap();
-///         
-///         // Something...
-///         
-///         {
-///             // Generates an Exit Span when executing an RPC.
-///             let span2 = ctx.create_exit_span("operation2", "remote_peer").unwrap();
-///             
-///             // Something...
-///
-///             ctx.finalize_span(span2);
-///         }
-///
-///         ctx.finalize_span(span);
-///     }
-/// }
-/// ```
-pub struct Span {
-    span_internal: SpanObject,
-    time_fetcher: Arc<dyn TimeFetcher + Sync + Send>,
+use crate::{
+    common::random_generator::RandomGenerator, context::propagation::context::PropagationContext,
+};
+use crate::{
+    error::LOCK_MSG,
+    skywalking_proto::v3::{
+        RefType, SegmentObject, SegmentReference, SpanLayer, SpanObject, SpanType,
+    },
+};
+use std::{
+    fmt::Formatter,
+    mem::take,
+    sync::{
+        atomic::{AtomicI32, Ordering},
+        Arc, Mutex, Weak,
+    },
+};
+
+struct Inner {
+    trace_id: String,
+    trace_segment_id: String,
+    service: String,
+    service_instance: String,
+    next_span_id: AtomicI32,
+    spans: Mutex<Vec<SpanObject>>,
+    active_span_stack: Mutex<Vec<SpanObject>>,
+    segment_link: Option<PropagationContext>,
+    primary_endpoint_name: Mutex<String>,
 }
 
-impl std::fmt::Debug for Span {
+#[derive(Clone)]
+#[must_use = "You should call `create_entry_span` after `TracingContext` created."]
+pub struct TracingContext {
+    inner: Arc<Inner>,
+    tracer: WeakTracer,
+}
+
+impl std::fmt::Debug for TracingContext {
     fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
-        f.debug_struct("Span")
-            .field("span_internal", &self.span_internal)
+        f.debug_struct("TracingContext")
+            .field("trace_id", &self.inner.trace_id)
+            .field("trace_segment_id", &self.inner.trace_segment_id)
+            .field("service", &self.inner.service)
+            .field("service_instance", &self.inner.service_instance)
+            .field("next_span_id", &self.inner.next_span_id)
+            .field("spans", &self.inner.spans)
             .finish()
     }
 }
 
-const SKYWALKING_RUST_COMPONENT_ID: i32 = 11000;
-
-impl Span {
-    #[allow(clippy::too_many_arguments)]
-    pub fn new(
-        span_id: i32,
-        parent_span_id: i32,
-        operation_name: String,
-        remote_peer: String,
-        span_type: SpanType,
-        span_layer: SpanLayer,
-        skip_analysis: bool,
-        time_fetcher: Arc<dyn TimeFetcher + Sync + Send>,
+impl TracingContext {
+    /// Generate a new trace context. Typically called when no context has
+    /// been propagated and a new trace is to be started.
+    pub(crate) fn new(
+        service_name: impl ToString,
+        instance_name: impl ToString,
+        tracer: WeakTracer,
     ) -> Self {
-        let span_internal = SpanObject {
-            span_id,
-            parent_span_id,
-            start_time: time_fetcher.get(),
-            end_time: 0, // not set
-            refs: Vec::<SegmentReference>::new(),
-            operation_name,
-            peer: remote_peer,
-            span_type: span_type as i32,
-            span_layer: span_layer as i32,
-            component_id: SKYWALKING_RUST_COMPONENT_ID,
-            is_error: false,
-            tags: Vec::<KeyStringValuePair>::new(),
-            logs: Vec::<Log>::new(),
-            skip_analysis,
-        };
-
-        Span {
-            span_internal,
-            time_fetcher,
+        TracingContext {
+            inner: Arc::new(Inner {
+                trace_id: RandomGenerator::generate(),
+                trace_segment_id: RandomGenerator::generate(),
+                service: service_name.to_string(),
+                service_instance: instance_name.to_string(),
+                next_span_id: Default::default(),
+                spans: Default::default(),
+                segment_link: None,
+                active_span_stack: Default::default(),
+                primary_endpoint_name: Default::default(),
+            }),
+            tracer,
         }
     }
 
-    /// Close span. It only registers end time to the span.
-    pub fn close(&mut self) {
-        self.span_internal.end_time = self.time_fetcher.get();
+    /// Generate a new trace context using the propagated context.
+    /// They should be propagated on `sw8` header in HTTP request with encoded form.
+    /// You can retrieve decoded context with `skywalking::context::propagation::encoder::encode_propagation`
+    pub(crate) fn from_propagation_context(
+        service_name: impl ToString,
+        instance_name: impl ToString,
+        context: PropagationContext,
+        tracer: WeakTracer,
+    ) -> Self {
+        TracingContext {
+            inner: Arc::new(Inner {
+                trace_id: context.parent_trace_id.clone(),
+                trace_segment_id: RandomGenerator::generate(),
+                service: service_name.to_string(),
+                service_instance: instance_name.to_string(),
+                next_span_id: Default::default(),
+                spans: Default::default(),
+                segment_link: Some(context),
+                active_span_stack: Default::default(),
+                primary_endpoint_name: Default::default(),
+            }),
+            tracer,
+        }
     }
 
-    pub fn span_object(&self) -> &SpanObject {
-        &self.span_internal
+    #[inline]
+    pub fn trace_id(&self) -> &str {
+        &self.inner.trace_id
     }
 
-    pub fn span_object_mut(&mut self) -> &mut SpanObject {
-        &mut self.span_internal
+    #[inline]
+    pub fn trace_segment_id(&self) -> &str {
+        &self.inner.trace_segment_id
     }
 
-    /// Add logs to the span.
-    pub fn add_log(&mut self, message: Vec<(&str, &str)>) {
-        let log = Log {
-            time: self.time_fetcher.get(),
-            data: message
-                .into_iter()
-                .map(|v| {
-                    let (key, value) = v;
-                    KeyStringValuePair {
-                        key: key.to_string(),
-                        value: value.to_string(),
-                    }
-                })
-                .collect(),
-        };
-        self.span_internal.logs.push(log);
+    #[inline]
+    pub fn service(&self) -> &str {
+        &self.inner.service
     }
 
-    /// Add tag to the span.
-    pub fn add_tag(&mut self, tag: (&str, &str)) {
-        let (key, value) = tag;
-        self.span_internal.tags.push(KeyStringValuePair {
-            key: key.to_string(),
-            value: value.to_string(),
-        });
+    #[inline]
+    pub fn service_instance(&self) -> &str {
+        &self.inner.service_instance
     }
 
-    fn add_segment_reference(&mut self, segment_reference: SegmentReference) {
-        self.span_internal.refs.push(segment_reference);
+    fn next_span_id(&self) -> i32 {
+        self.inner.next_span_id.load(Ordering::Relaxed)
     }
-}
 
-pub struct TracingContext {
-    pub trace_id: String,
-    pub trace_segment_id: String,
-    pub service: String,
-    pub service_instance: String,
-    pub next_span_id: i32,
-    pub spans: Vec<Box<Span>>,
-    time_fetcher: Arc<dyn TimeFetcher + Sync + Send>,
-    segment_link: Option<PropagationContext>,
-    active_span_id_stack: LinkedList<i32>,
-}
+    #[inline]
+    fn inc_next_span_id(&self) -> i32 {
+        self.inner.next_span_id.fetch_add(1, Ordering::Relaxed)
+    }
 
-impl std::fmt::Debug for TracingContext {
-    fn fmt(&self, f: &mut Formatter<'_>) -> std::fmt::Result {
-        f.debug_struct("TracingContext")
-            .field("trace_id", &self.trace_id)
-            .field("trace_segment_id", &self.trace_segment_id)
-            .field("service", &self.service)
-            .field("service_instance", &self.service_instance)
-            .field("next_span_id", &self.next_span_id)
-            .field("spans", &self.spans)
-            .finish()
+    #[cfg(feature = "mock")]
+    pub fn with_spans<T>(&self, f: impl FnOnce(&Vec<SpanObject>) -> T) -> T {
+        f(&*self.inner.spans.try_lock().expect(LOCK_MSG))
     }
-}
 
-impl TracingContext {
-    /// Generate a new trace context. Typically called when no context has
-    /// been propagated and a new trace is to be started.
-    pub fn default(service_name: &str, instance_name: &str) -> Self {
-        let unix_time_fetcher = UnixTimeStampFetcher {};
-        TracingContext::default_internal(Arc::new(unix_time_fetcher), service_name, instance_name)
+    fn with_spans_mut<T>(&mut self, f: impl FnOnce(&mut Vec<SpanObject>) -> T) -> T {
+        f(&mut *self.inner.spans.try_lock().expect(LOCK_MSG))
     }
 
-    pub fn default_internal(
-        time_fetcher: Arc<dyn TimeFetcher + Sync + Send>,
-        service_name: &str,
-        instance_name: &str,
-    ) -> Self {
-        TracingContext {
-            trace_id: RandomGenerator::generate(),
-            trace_segment_id: RandomGenerator::generate(),
-            service: String::from(service_name),
-            service_instance: String::from(instance_name),
-            next_span_id: 0,
-            time_fetcher,
-            spans: Vec::new(),
-            segment_link: None,
-            active_span_id_stack: LinkedList::new(),
-        }
+    pub(crate) fn with_active_span_stack<T>(&self, f: impl FnOnce(&Vec<SpanObject>) -> T) -> T {
+        f(&*self.inner.active_span_stack.try_lock().expect(LOCK_MSG))
     }
 
-    /// Generate a new trace context using the propagated context.
-    /// They should be propagated on `sw8` header in HTTP request with encoded form.
-    /// You can retrieve decoded context with `skywalking::context::propagation::encoder::encode_propagation`
-    pub fn from_propagation_context(
-        service_name: &str,
-        instance_name: &str,
-        context: PropagationContext,
-    ) -> Self {
-        let unix_time_fetcher = UnixTimeStampFetcher {};
-        TracingContext::from_propagation_context_internal(
-            Arc::new(unix_time_fetcher),
-            service_name,
-            instance_name,
-            context,
-        )
+    pub(crate) fn with_active_span_stack_mut<T>(
+        &mut self,
+        f: impl FnOnce(&mut Vec<SpanObject>) -> T,
+    ) -> T {
+        f(&mut *self.inner.active_span_stack.try_lock().expect(LOCK_MSG))
     }
 
-    pub fn from_propagation_context_internal(
-        time_fetcher: Arc<dyn TimeFetcher + Sync + Send>,
-        service_name: &str,
-        instance_name: &str,
-        context: PropagationContext,
-    ) -> Self {
-        TracingContext {
-            trace_id: context.parent_trace_id.clone(),
-            trace_segment_id: RandomGenerator::generate(),
-            service: service_name.to_string(),
-            service_instance: instance_name.to_string(),
-            next_span_id: 0,
-            time_fetcher,
-            spans: Vec::new(),
-            segment_link: Some(context),
-            active_span_id_stack: LinkedList::new(),
-        }
+    pub(crate) fn try_with_active_span_stack<T>(
+        &self,
+        f: impl FnOnce(&Vec<SpanObject>) -> T,
+    ) -> Option<T> {
+        self.inner
+            .active_span_stack
+            .try_lock()
+            .ok()
+            .map(|stack| f(&*stack))
     }
 
-    /// A wrapper of create entry span, which close generated span automatically.
-    /// Note that, we may use async operation in closure. But that is not unstable feature in 2021/12.
-    /// <https://github.com/rust-lang/rust/issues/62290>
-    /// So we should create and close spans manually in general.
-    pub fn entry<F: FnMut(&Span)>(
-        &mut self,
-        operation_name: &str,
-        mut process_fn: F,
-    ) -> crate::Result<()> {
-        match self.create_entry_span(operation_name) {
-            Ok(mut span) => {
-                process_fn(span.as_ref());
-                span.close();
-                Ok(())
-            }
-            Err(message) => Err(message),
-        }
+    pub(crate) fn with_active_span<T>(&self, f: impl FnOnce(&SpanObject) -> T) -> Option<T> {
+        self.with_active_span_stack(|stack| stack.last().map(f))
+    }
+
+    // TODO Using for capture and continued.
+    #[allow(dead_code)]
+    fn with_primary_endpoint_name<T>(&self, f: impl FnOnce(&String) -> T) -> T {
+        f(&*self.inner.primary_endpoint_name.try_lock().expect(LOCK_MSG))
+    }
+
+    fn with_primary_endpoint_name_mut<T>(&mut self, f: impl FnOnce(&mut String) -> T) -> T {
+        f(&mut *self.inner.primary_endpoint_name.try_lock().expect(LOCK_MSG))
     }
 
     /// Create a new entry span, which is an initiator of collection of spans.
     /// This should be called by invocation of the function which is triggered by
     /// external service.
-    pub fn create_entry_span(&mut self, operation_name: &str) -> crate::Result<Box<Span>> {
-        if self.next_span_id >= 1 {
-            return Err(crate::Error::CreateSpan("entry span have already exist."));
+    ///
+    /// # Panics
+    ///
+    /// Panic if entry span have already exist.
+    pub fn create_entry_span(&mut self, operation_name: &str) -> Span {
+        if self.next_span_id() >= 1 {
+            panic!("entry span have already exist.");

Review Comment:
   Use panic instead of returning error, because it is caused by the programmer's incorrect usage, not external input. If there is a need to return an error in the future, can add a `try_` prefix method.



-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-rust] codecov-commenter commented on pull request #28: Auto finalize context and span when dropped.

Posted by GitBox <gi...@apache.org>.
codecov-commenter commented on PR #28:
URL: https://github.com/apache/skywalking-rust/pull/28#issuecomment-1186062047

   # [Codecov](https://codecov.io/gh/apache/skywalking-rust/pull/28?src=pr&el=h1&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) Report
   > Merging [#28](https://codecov.io/gh/apache/skywalking-rust/pull/28?src=pr&el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (40d71bf) into [master](https://codecov.io/gh/apache/skywalking-rust/commit/0c43b32b9e0a6fa172c2e0129e55df9ca6659597?el=desc&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) (0c43b32) will **decrease** coverage by `33.74%`.
   > The diff coverage is `23.80%`.
   
   ```diff
   @@             Coverage Diff             @@
   ##           master      #28       +/-   ##
   ===========================================
   - Coverage   75.29%   41.54%   -33.75%     
   ===========================================
     Files          10       12        +2     
     Lines         344      284       -60     
   ===========================================
   - Hits          259      118      -141     
   - Misses         85      166       +81     
   ```
   
   
   | [Impacted Files](https://codecov.io/gh/apache/skywalking-rust/pull/28?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | Coverage Δ | |
   |---|---|---|
   | [src/context/span.rs](https://codecov.io/gh/apache/skywalking-rust/pull/28/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3JjL2NvbnRleHQvc3Bhbi5ycw==) | `0.00% <0.00%> (ø)` | |
   | [src/context/system\_time.rs](https://codecov.io/gh/apache/skywalking-rust/pull/28/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3JjL2NvbnRleHQvc3lzdGVtX3RpbWUucnM=) | `0.00% <0.00%> (ø)` | |
   | [src/reporter/grpc.rs](https://codecov.io/gh/apache/skywalking-rust/pull/28/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3JjL3JlcG9ydGVyL2dycGMucnM=) | `0.00% <0.00%> (ø)` | |
   | [src/reporter/log.rs](https://codecov.io/gh/apache/skywalking-rust/pull/28/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3JjL3JlcG9ydGVyL2xvZy5ycw==) | `0.00% <0.00%> (ø)` | |
   | [tests/trace\_context.rs](https://codecov.io/gh/apache/skywalking-rust/pull/28/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-dGVzdHMvdHJhY2VfY29udGV4dC5ycw==) | `0.00% <ø> (-100.00%)` | :arrow_down: |
   | [src/context/trace\_context.rs](https://codecov.io/gh/apache/skywalking-rust/pull/28/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3JjL2NvbnRleHQvdHJhY2VfY29udGV4dC5ycw==) | `23.20% <24.34%> (-51.59%)` | :arrow_down: |
   | [src/context/tracer.rs](https://codecov.io/gh/apache/skywalking-rust/pull/28/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3JjL2NvbnRleHQvdHJhY2VyLnJz) | `41.17% <36.66%> (+41.17%)` | :arrow_up: |
   | [src/context/propagation/encoder.rs](https://codecov.io/gh/apache/skywalking-rust/pull/28/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-c3JjL2NvbnRleHQvcHJvcGFnYXRpb24vZW5jb2Rlci5ycw==) | `100.00% <100.00%> (ø)` | |
   | [tests/propagation.rs](https://codecov.io/gh/apache/skywalking-rust/pull/28/diff?src=pr&el=tree&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation#diff-dGVzdHMvcHJvcGFnYXRpb24ucnM=) | `100.00% <100.00%> (+3.22%)` | :arrow_up: |
   | ... and [3 more](https://codecov.io/gh/apache/skywalking-rust/pull/28/diff?src=pr&el=tree-more&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation) | |
   
   ------
   
   [Continue to review full report at Codecov](https://codecov.io/gh/apache/skywalking-rust/pull/28?src=pr&el=continue&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   > **Legend** - [Click here to learn more](https://docs.codecov.io/docs/codecov-delta?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation)
   > `Δ = absolute <relative> (impact)`, `ø = not affected`, `? = missing data`
   > Powered by [Codecov](https://codecov.io/gh/apache/skywalking-rust/pull/28?src=pr&el=footer&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Last update [0c43b32...40d71bf](https://codecov.io/gh/apache/skywalking-rust/pull/28?src=pr&el=lastupdated&utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation). Read the [comment docs](https://docs.codecov.io/docs/pull-request-comments?utm_medium=referral&utm_source=github&utm_content=comment&utm_campaign=pr+comments&utm_term=The+Apache+Software+Foundation).
   


-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-rust] wu-sheng commented on a diff in pull request #28: Auto finalize context and span when dropped.

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on code in PR #28:
URL: https://github.com/apache/skywalking-rust/pull/28#discussion_r922592340


##########
README.md:
##########
@@ -37,38 +37,38 @@ context after the span finished.
 use skywalking::context::tracer::Tracer;
 use skywalking::reporter::grpc::GrpcReporter;
 use std::error::Error;
-use std::sync::Arc;
 use tokio::signal;
 
-async fn handle_request(tracer: Arc<Tracer<GrpcReporter>>) {
+async fn handle_request(tracer: Tracer) {
     let mut ctx = tracer.create_trace_context();
 
     {
-        // Generate an Entry Span when a request
-        // is received. An Entry Span is generated only once per context.
-        let span = ctx.create_entry_span("op1").unwrap();
+        // Generate an Entry Span when a request is received.
+        // An Entry Span is generated only once per context.

Review Comment:
   Is this limited through codes? In some rare cases(such as queue bulk consuming), we may face multi-entry spans in one context.



-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-rust] jmjoy commented on a diff in pull request #28: Auto finalize context and span when dropped.

Posted by GitBox <gi...@apache.org>.
jmjoy commented on code in PR #28:
URL: https://github.com/apache/skywalking-rust/pull/28#discussion_r922611928


##########
README.md:
##########
@@ -37,38 +37,38 @@ context after the span finished.
 use skywalking::context::tracer::Tracer;
 use skywalking::reporter::grpc::GrpcReporter;
 use std::error::Error;
-use std::sync::Arc;
 use tokio::signal;
 
-async fn handle_request(tracer: Arc<Tracer<GrpcReporter>>) {
+async fn handle_request(tracer: Tracer) {
     let mut ctx = tracer.create_trace_context();
 
     {
-        // Generate an Entry Span when a request
-        // is received. An Entry Span is generated only once per context.
-        let span = ctx.create_entry_span("op1").unwrap();
+        // Generate an Entry Span when a request is received.
+        // An Entry Span is generated only once per context.

Review Comment:
   Limitation of codes have removed.



-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-rust] wu-sheng commented on a diff in pull request #28: Auto finalize context and span when dropped.

Posted by GitBox <gi...@apache.org>.
wu-sheng commented on code in PR #28:
URL: https://github.com/apache/skywalking-rust/pull/28#discussion_r922592690


##########
README.md:
##########
@@ -37,38 +37,38 @@ context after the span finished.
 use skywalking::context::tracer::Tracer;
 use skywalking::reporter::grpc::GrpcReporter;
 use std::error::Error;
-use std::sync::Arc;
 use tokio::signal;
 
-async fn handle_request(tracer: Arc<Tracer<GrpcReporter>>) {
+async fn handle_request(tracer: Tracer) {
     let mut ctx = tracer.create_trace_context();
 
     {
-        // Generate an Entry Span when a request
-        // is received. An Entry Span is generated only once per context.
-        let span = ctx.create_entry_span("op1").unwrap();
+        // Generate an Entry Span when a request is received.
+        // An Entry Span is generated only once per context.
+        // You should assign a variable name to guard the span not be dropped immediately.

Review Comment:
   Usually, we don't use `subject` in doc. 



-- 
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: notifications-unsubscribe@skywalking.apache.org

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


[GitHub] [skywalking-rust] wu-sheng merged pull request #28: Auto finalize context and span when dropped.

Posted by GitBox <gi...@apache.org>.
wu-sheng merged PR #28:
URL: https://github.com/apache/skywalking-rust/pull/28


-- 
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: notifications-unsubscribe@skywalking.apache.org

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