You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@hbase.apache.org by GitBox <gi...@apache.org> on 2020/02/27 17:33:54 UTC

[GitHub] [hbase] ramkrish86 commented on a change in pull request #1210: HBASE-23757. [OpenTracing] Migrate from HTrace to OpenTracing (Java code)

ramkrish86 commented on a change in pull request #1210: HBASE-23757. [OpenTracing] Migrate from HTrace to OpenTracing (Java code)
URL: https://github.com/apache/hbase/pull/1210#discussion_r385262996
 
 

 ##########
 File path: hbase-common/src/main/java/org/apache/hadoop/hbase/trace/TraceUtil.java
 ##########
 @@ -17,103 +17,169 @@
  */
 package org.apache.hadoop.hbase.trace;
 
+import io.opentracing.Scope;
+import io.opentracing.SpanContext;
+import io.opentracing.mock.MockTracer;
+import io.opentracing.propagation.Format;
+import io.opentracing.propagation.TextMapExtractAdapter;
+import io.opentracing.propagation.TextMapInjectAdapter;
+import io.opentracing.Span;
+import io.opentracing.SpanContext;
+import io.opentracing.Tracer;
+import io.opentracing.propagation.Format;
+import io.opentracing.propagation.TextMapExtractAdapter;
+import io.opentracing.propagation.TextMapInjectAdapter;
+import io.opentracing.util.GlobalTracer;
+import org.apache.commons.codec.binary.Hex;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.htrace.core.HTraceConfiguration;
 import org.apache.htrace.core.Sampler;
-import org.apache.htrace.core.Span;
 import org.apache.htrace.core.SpanReceiver;
 import org.apache.htrace.core.TraceScope;
-import org.apache.htrace.core.Tracer;
+import io.jaegertracing.Configuration.SamplerConfiguration;
+
+import org.apache.hadoop.tracing.TraceUtils;
+import org.apache.hbase.thirdparty.com.google.common.annotations.VisibleForTesting;
 import org.apache.yetus.audience.InterfaceAudience;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.io.ObjectOutputStream;
+import java.util.HashMap;
+import java.util.Map;
 
 /**
  * This wrapper class provides functions for accessing htrace 4+ functionality in a simplified way.
  */
 @InterfaceAudience.Private
 public final class TraceUtil {
-  private static HTraceConfiguration conf;
-  private static Tracer tracer;
+  static final Logger LOG = LoggerFactory.getLogger(TraceUtils.class);
+
+  private static io.jaegertracing.Configuration conf;
+  private static io.opentracing.Tracer tracer;
+
+  public static final String HBASE_OPENTRACING_TRACER = "hbase.opentracing.tracer";
+  public static final String HBASE_OPENTRACING_TRACER_DEFAULT = "jaeger";
+  public static final String HBASE_OPENTRACING_MOCKTRACER = "mock";
 
   private TraceUtil() {
   }
 
-  public static void initTracer(Configuration c) {
-    if (c != null) {
-      conf = new HBaseHTraceConfiguration(c);
+  public static void initTracer(Configuration c, String serviceName) {
+    if (GlobalTracer.isRegistered()) {
+      LOG.info("A tracer is already registered.");
+      return;
     }
 
-    if (tracer == null && conf != null) {
-      tracer = new Tracer.Builder("Tracer").conf(conf).build();
+    switch(c.get(HBASE_OPENTRACING_TRACER, HBASE_OPENTRACING_TRACER_DEFAULT)) {
+    case HBASE_OPENTRACING_TRACER_DEFAULT:
+      io.jaegertracing.Configuration conf = io.jaegertracing.Configuration.fromEnv(serviceName);
+      tracer = conf.getTracerBuilder().build();
+      break;
+    case HBASE_OPENTRACING_MOCKTRACER:
+      tracer = new MockTracer();
+      break;
+    default:
+      throw new RuntimeException("Unexpected tracer");
     }
+
+
+    GlobalTracer.register(tracer);
+
+  }
+
+  /*@VisibleForTesting
+  public static void registerTracerForTest(Tracer tracer) {
+    TraceUtil.tracer = tracer;
+    GlobalTracer.register(tracer);
+  }*/
+
+  public static Tracer getTracer() {
+    return tracer;
   }
 
   /**
-   * Wrapper method to create new TraceScope with the given description
-   * @return TraceScope or null when not tracing
+   * Wrapper method to create new Scope with the given description
+   * @return Scope or null when not tracing
    */
-  public static TraceScope createTrace(String description) {
-    return (tracer == null) ? null : tracer.newScope(description);
+  public static Scope createTrace(String description) {
+    return (tracer == null) ? null :
+        tracer.buildSpan(description).startActive(true);
   }
 
   /**
-   * Wrapper method to create new child TraceScope with the given description
+   * Wrapper method to create new child Scope with the given description
    * and parent scope's spanId
    * @param span parent span
-   * @return TraceScope or null when not tracing
+   * @return Scope or null when not tracing
    */
-  public static TraceScope createTrace(String description, Span span) {
-    if (span == null) {
-      return createTrace(description);
-    }
+  public static Scope createTrace(String description, Span span) {
+    if(span == null) return createTrace(description);
+
+    return (tracer == null) ? null : tracer.buildSpan(description).
+        asChildOf(span).startActive(true);
+  }
+
+  public static Scope createTrace(String description, SpanContext spanContext) {
+    if(spanContext == null) return createTrace(description);
 
-    return (tracer == null) ? null : tracer.newScope(description, span.getSpanId());
+    return (tracer == null) ? null : tracer.buildSpan(description).
+        asChildOf(spanContext).startActive(true);
   }
 
   /**
    * Wrapper method to add new sampler to the default tracer
    * @return true if added, false if it was already added
    */
-  public static boolean addSampler(Sampler sampler) {
+  public static boolean addSampler(SamplerConfiguration sampler) {
     if (sampler == null) {
       return false;
     }
 
-    return (tracer == null) ? false : tracer.addSampler(sampler);
+    conf = conf.withSampler(sampler);
 
 Review comment:
   I think this is getting called from clients which needs the tracing. This 'conf' may be null I believe when this is called. @jojochuang - thanks for pointing to me to the docs. I was trying do some similar stuff and I took your patch as a reference. Now there are some quesitons - Assume I have kept my RegionServer to work with No sampling (just assume I don trace the flushes/compactions). Now if the client side I have enabled some sampling(tracing) say for eg writes -  if the writes are propogated to the server and we do the tracing - in that case will the sampling applied as what the client has decided ? 

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


With regards,
Apache Git Services