ctubbsii commented on a change in pull request #2259:
URL: https://github.com/apache/accumulo/pull/2259#discussion_r717030901



##########
File path: assemble/bin/accumulo-cluster
##########
@@ -73,15 +73,6 @@ function verify_config {
       exit 1
     fi
   fi
-  if [[ ! -f "${conf}/tracers" ]]; then

Review comment:
       If a tracers file is detected, we could ease the transition away from 
the old tracers service by providing some warning message here.

##########
File path: 
core/src/main/java/org/apache/accumulo/core/util/threads/ThreadPools.java
##########
@@ -101,88 +106,105 @@ public static ExecutorService 
createExecutorService(final AccumuloConfiguration
 
     switch (p) {
       case GENERAL_SIMPLETIMER_THREADPOOL_SIZE:
-        return createScheduledExecutorService(conf.getCount(p), "SimpleTimer", 
false);
+        return createScheduledExecutorService(conf.getCount(p), "SimpleTimer");
       case MANAGER_BULK_THREADPOOL_SIZE:
         return createFixedThreadPool(conf.getCount(p),
             conf.getTimeInMillis(Property.MANAGER_BULK_THREADPOOL_TIMEOUT), 
TimeUnit.MILLISECONDS,
-            "bulk import", true);
+            "bulk import");
       case MANAGER_RENAME_THREADS:
-        return createFixedThreadPool(conf.getCount(p), "bulk move", false);
+        return createFixedThreadPool(conf.getCount(p), "bulk move");
       case MANAGER_FATE_THREADPOOL_SIZE:
-        return createFixedThreadPool(conf.getCount(p), "Repo Runner", false);
+        return createFixedThreadPool(conf.getCount(p), "Repo Runner");
       case MANAGER_STATUS_THREAD_POOL_SIZE:
         int threads = conf.getCount(p);
         if (threads == 0) {
           return createThreadPool(0, Integer.MAX_VALUE, 60L, TimeUnit.SECONDS,
-              "GatherTableInformation", new SynchronousQueue<Runnable>(), 
OptionalInt.empty(),
-              false);
+              "GatherTableInformation", new SynchronousQueue<Runnable>(), 
OptionalInt.empty());
         } else {
-          return createFixedThreadPool(threads, "GatherTableInformation", 
false);
+          return createFixedThreadPool(threads, "GatherTableInformation");
         }
       case TSERV_WORKQ_THREADS:
-        return createFixedThreadPool(conf.getCount(p), "distributed work 
queue", false);
+        return createFixedThreadPool(conf.getCount(p), "distributed work 
queue");
       case TSERV_MINC_MAXCONCURRENT:
-        return createFixedThreadPool(conf.getCount(p), 0L, 
TimeUnit.MILLISECONDS, "minor compactor",
-            true);
+        return createFixedThreadPool(conf.getCount(p), 0L, 
TimeUnit.MILLISECONDS,
+            "minor compactor");
       case TSERV_MIGRATE_MAXCONCURRENT:
         return createFixedThreadPool(conf.getCount(p), 0L, 
TimeUnit.MILLISECONDS,
-            "tablet migration", true);
+            "tablet migration");
       case TSERV_ASSIGNMENT_MAXCONCURRENT:
         return createFixedThreadPool(conf.getCount(p), 0L, 
TimeUnit.MILLISECONDS,
-            "tablet assignment", true);
+            "tablet assignment");
       case TSERV_SUMMARY_RETRIEVAL_THREADS:
         return createThreadPool(conf.getCount(p), conf.getCount(p), 60, 
TimeUnit.SECONDS,
-            "summary file retriever", true);
+            "summary file retriever");
       case TSERV_SUMMARY_REMOTE_THREADS:
         return createThreadPool(conf.getCount(p), conf.getCount(p), 60, 
TimeUnit.SECONDS,
-            "summary remote", true);
+            "summary remote");
       case TSERV_SUMMARY_PARTITION_THREADS:
         return createThreadPool(conf.getCount(p), conf.getCount(p), 60, 
TimeUnit.SECONDS,
-            "summary partition", true);
+            "summary partition");
       case GC_DELETE_THREADS:
-        return createFixedThreadPool(conf.getCount(p), "deleting", false);
+        return createFixedThreadPool(conf.getCount(p), "deleting");
       case REPLICATION_WORKER_THREADS:
-        return createFixedThreadPool(conf.getCount(p), "replication task", 
false);
+        return createFixedThreadPool(conf.getCount(p), "replication task");
       default:
         throw new RuntimeException("Unhandled thread pool property: " + p);
     }
   }
 
-  public static ThreadPoolExecutor createFixedThreadPool(int numThreads, final 
String name,
-      boolean enableTracing) {
-    return createFixedThreadPool(numThreads, DEFAULT_TIMEOUT_MILLISECS, 
TimeUnit.MILLISECONDS, name,
-        enableTracing);
+  public static ThreadPoolExecutor createFixedThreadPool(int numThreads, final 
String name) {
+    return createFixedThreadPool(numThreads, DEFAULT_TIMEOUT_MILLISECS, 
TimeUnit.MILLISECONDS,
+        name);
   }
 
   public static ThreadPoolExecutor createFixedThreadPool(int numThreads, final 
String name,
-      BlockingQueue<Runnable> queue, boolean enableTracing) {
+      BlockingQueue<Runnable> queue) {
     return createThreadPool(numThreads, numThreads, DEFAULT_TIMEOUT_MILLISECS,
-        TimeUnit.MILLISECONDS, name, queue, OptionalInt.empty(), 
enableTracing);
+        TimeUnit.MILLISECONDS, name, queue, OptionalInt.empty());
   }
 
   public static ThreadPoolExecutor createFixedThreadPool(int numThreads, long 
timeOut,
-      TimeUnit units, final String name, boolean enableTracing) {
+      TimeUnit units, final String name) {
     return createThreadPool(numThreads, numThreads, timeOut, units, name,
-        new LinkedBlockingQueue<Runnable>(), OptionalInt.empty(), 
enableTracing);
+        new LinkedBlockingQueue<Runnable>(), OptionalInt.empty());
   }
 
   public static ThreadPoolExecutor createThreadPool(int coreThreads, int 
maxThreads, long timeOut,
-      TimeUnit units, final String name, boolean enableTracing) {
+      TimeUnit units, final String name) {
     return createThreadPool(coreThreads, maxThreads, timeOut, units, name,
-        new LinkedBlockingQueue<Runnable>(), OptionalInt.empty(), 
enableTracing);
+        new LinkedBlockingQueue<Runnable>(), OptionalInt.empty());
   }
 
   public static ThreadPoolExecutor createThreadPool(int coreThreads, int 
maxThreads, long timeOut,
-      TimeUnit units, final String name, BlockingQueue<Runnable> queue, 
OptionalInt priority,
-      boolean enableTracing) {
-    ThreadPoolExecutor result = null;
-    if (enableTracing) {
-      result = new TracingThreadPoolExecutor(coreThreads, maxThreads, timeOut, 
units, queue,
-          new NamedThreadFactory(name, priority));
-    } else {
-      result = new ThreadPoolExecutor(coreThreads, maxThreads, timeOut, units, 
queue,
-          new NamedThreadFactory(name, priority));
-    }
+      TimeUnit units, final String name, BlockingQueue<Runnable> queue, 
OptionalInt priority) {
+    ThreadPoolExecutor result = new ThreadPoolExecutor(coreThreads, 
maxThreads, timeOut, units,
+        queue, new NamedThreadFactory(name, priority)) {
+
+      @Override
+      public void execute(Runnable arg0) {
+        super.execute(Context.current().wrap(arg0));
+      }

Review comment:
       This is nice, but I'm a little bit surprised OpenTelemetry upstream 
didn't already have an Executor that did these.

##########
File path: 
server/tracer/src/main/java/org/apache/accumulo/tracer/DefaultTracerProvider.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.accumulo.tracer;
+
+import org.apache.accumulo.core.trace.TracerProvider;
+import org.apache.commons.lang3.StringUtils;
+
+import com.google.auto.service.AutoService;
+
+import io.opentelemetry.api.OpenTelemetry;
+import io.opentelemetry.api.trace.Tracer;
+import io.opentelemetry.sdk.autoconfigure.OpenTelemetrySdkAutoConfiguration;
+
+@AutoService(TracerProvider.class)
+public class DefaultTracerProvider implements TracerProvider {
+
+  @Override
+  public Tracer getTracer() {
+    // Set the service name if not set
+    String svcNameEnvVar = System.getenv("OTEL_SERVICE_NAME");
+    String svcNameProp = System.getenv("otel.service.name");
+    String appName = System.getProperty("accumulo.application"); // set in 
accumulo-env.sh
+    if (StringUtils.isEmpty(svcNameEnvVar) && StringUtils.isEmpty(svcNameProp)
+        && !StringUtils.isEmpty(appName)) {
+      System.setProperty("otel.service.name", appName);
+    }

Review comment:
       This can all be replaced by setting `-Dotel.service.name=...` in the 
`accumulo-env.sh` file to the same value as `accumulo.application`. The user 
can change it from that default if they wish.

##########
File path: 
core/src/main/java/org/apache/accumulo/core/clientImpl/ThriftScanner.java
##########
@@ -396,10 +428,17 @@ else if (log.isTraceEnabled())
           // because a thread on the server side may still be processing the 
timed out continue scan
           scanState.scanID = null;
 
-          if (scanState.isolated)
+          if (scanState.isolated) {
+            child2.recordException(e, 
Attributes.builder().put("exception.message", e.getMessage())

Review comment:
       Do we need to explicitly add the `exception.message` every time we call 
`recordException`? It seems like it should do that much on its own at least.

##########
File path: 
core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
##########
@@ -302,7 +304,8 @@ public synchronized void flush() throws 
MutationsRejectedException {
     if (closed)
       throw new IllegalStateException("Closed");
 
-    try (TraceScope span = Trace.startSpan("flush")) {
+    Span span = 
TraceUtil.getTracer().spanBuilder("TabletServerBatchWriter::flush").startSpan();

Review comment:
       Some of these SpanBuilder options could be set in future to indicate 
that the span is client side or server side of an RPC call, using 
`setSpanKind()`. This could be very useful. Other useful things might be to set 
the parent or to set that it has no parent.

##########
File path: core/src/main/java/org/apache/accumulo/core/trace/TracerProvider.java
##########
@@ -16,10 +16,19 @@
  * specific language governing permissions and limitations
  * under the License.
  */
-package org.apache.accumulo.tracer;
+package org.apache.accumulo.core.trace;
 
-import org.apache.accumulo.tracer.thrift.RemoteSpan;
+import io.opentelemetry.api.trace.Tracer;
+
+public interface TracerProvider {

Review comment:
       OpenTelemetry already has a type called `TracerProvider`, and it is one 
of the main types in the OpenTelemetry interface, which is the entry point into 
the API. If we do decide to keep the ServiceLoader and not merely instantiate 
an `OpenTelemetry` class, it should be a `ServiceLoader<OpenTelemetryFactory>`, 
with this interface called `OpenTelemetryFactory` and a `getOpenTelemetry()` 
method instead of a `getTracer()` method, so as to not overlap API concepts.
   
   I *think* what I'm calling `OpenTelemetryFactory` here is what upstream is 
calling an `OpenTelemetry SDK`, but it's not quite clear from their 
documentation what they intend.

##########
File path: 
core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
##########
@@ -334,7 +343,8 @@ public synchronized void close() throws 
MutationsRejectedException {
     if (closed)
       return;
 
-    try (TraceScope span = Trace.startSpan("close")) {
+    Span span = 
TraceUtil.getTracer().spanBuilder("TabletServerBatchWriter::close").startSpan();

Review comment:
       Since we have `TraceUtil` anyway, it might be useful to just make a 
method in `TraceUtil.java` that does:
   
   ```java
     Span newSpan(String name) {
       return getTracer().spanBuilder(name).startSpan();
     }
   ```
   
   This would help clean up some boilerplate verbosity, and keep the `Tracer` 
management code in `TraceUtil`.

##########
File path: core/src/test/java/org/apache/accumulo/core/conf/PropertyTest.java
##########
@@ -96,6 +96,7 @@ public void testPorts() {
   }
 
   // This test verifies all "sensitive" properties are properly marked as 
sensitive
+  @SuppressWarnings("deprecation")

Review comment:
       tip: in order to avoid suppressing too aggressively, which can hide 
things we want to change or haven't yet triaged, you can assign the deprecated 
enum to a local variable, suppressing only that assignment.

##########
File path: core/src/main/java/org/apache/accumulo/core/conf/Property.java
##########
@@ -600,8 +600,10 @@
       "1.8.0"),
   TSERV_MINC_MAXCONCURRENT("tserver.compaction.minor.concurrent.max", "4", 
PropertyType.COUNT,
       "The maximum number of concurrent minor compactions for a tablet 
server", "1.3.5"),
+  @Deprecated(since = "2.1.0", forRemoval = true)
   TSERV_MAJC_TRACE_PERCENT("tserver.compaction.major.trace.percent", "0.1", 
PropertyType.FRACTION,
       "The percent of major compactions to trace", "1.7.0"),
+  @Deprecated(since = "2.1.0", forRemoval = true)
   TSERV_MINC_TRACE_PERCENT("tserver.compaction.minor.trace.percent", "0.1", 
PropertyType.FRACTION,
       "The percent of minor compactions to trace", "1.7.0"),

Review comment:
       One of the problems I ran into trying to go to HTrace 4.1 is that I 
couldn't figure out how to manage multiple different tracers, with different 
sampling options, without getting things too complicated. It might be nice to 
provide some ability to configure different different tracers with different 
samplers, but I don't think we need to do it right away. We can add something 
like that later, assuming the built-in OpenTelemetry collectors doesn't already 
have configuration for that.

##########
File path: 
core/src/main/java/org/apache/accumulo/core/clientImpl/TabletServerBatchWriter.java
##########
@@ -707,16 +721,24 @@ void queueMutations(final MutationSet mutationsToSend) {
             updateUnknownErrors("Error processing mutation set", e);
           }
         }
-      }));
+      });
     }
 
     private void addMutations(MutationSet mutationsToSend) {
       Map<String,TabletServerMutations<Mutation>> binnedMutations = new 
HashMap<>();
-      try (TraceScope span = Trace.startSpan("binMutations")) {
+      Span span =
+          
TraceUtil.getTracer().spanBuilder("TabletServerBatchWriter::binMutations").startSpan();
+      try (Scope scope = span.makeCurrent()) {

Review comment:
       It's a little annoying that constructing a new Span and adding it to the 
current context are two separate steps. It's probably not worth creating a 
wrapper around this, but if we were to do it, we could return an AutoCloseable 
Span type that calls `span.end()` and `scope.close()` in its own `.close()` 
method.

##########
File path: core/src/main/java/org/apache/accumulo/core/trace/TraceUtil.java
##########
@@ -18,176 +18,82 @@
  */
 package org.apache.accumulo.core.trace;
 
-import java.io.IOException;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Proxy;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Properties;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
+import java.util.Optional;
+import java.util.ServiceLoader;
 
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ClientProperty;
-import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
-import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.trace.thrift.TInfo;
-import org.apache.hadoop.util.ShutdownHookManager;
-import org.apache.htrace.HTraceConfiguration;
-import org.apache.htrace.NullScope;
-import org.apache.htrace.Span;
-import org.apache.htrace.SpanReceiver;
-import org.apache.htrace.SpanReceiverBuilder;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceInfo;
-import org.apache.htrace.TraceScope;
-import org.apache.htrace.impl.CountSampler;
-import org.apache.htrace.impl.ProbabilitySampler;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-/**
- * Utility class for tracing within Accumulo. Not intended for client use!
- */
-public class TraceUtil {
-
-  private static final Logger log = LoggerFactory.getLogger(TraceUtil.class);
-  public static final String TRACE_HOST_PROPERTY = "trace.host";
-  public static final String TRACE_SERVICE_PROPERTY = "trace.service";
-  public static final String TRACER_ZK_HOST = "tracer.zookeeper.host";
-  public static final String TRACER_ZK_TIMEOUT = "tracer.zookeeper.timeout";
-  public static final String TRACER_ZK_PATH = "tracer.zookeeper.path";
-  private static final HashSet<SpanReceiver> receivers = new HashSet<>();
-
-  /**
-   * Enable tracing by setting up SpanReceivers for the current process. If 
host name is null, it
-   * will be determined. If service name is null, the simple name of the class 
will be used.
-   * Properties required in the client configuration include
-   * {@link org.apache.accumulo.core.conf.ClientProperty#TRACE_SPAN_RECEIVERS} 
and any properties
-   * specific to the span receiver.
-   */
-  public static void enableClientTraces(String hostname, String service, 
Properties properties) {
-    // @formatter:off
-      enableTracing(hostname, service,
-          ClientProperty.TRACE_SPAN_RECEIVERS.getValue(properties),
-          ClientProperty.INSTANCE_ZOOKEEPERS.getValue(properties),
-          ConfigurationTypeHelper
-              
.getTimeInMillis(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT.getValue(properties)),
-          ClientProperty.TRACE_ZOOKEEPER_PATH.getValue(properties),
-          ClientProperty.toMap(
-              ClientProperty.getPrefix(properties, 
ClientProperty.TRACE_SPAN_RECEIVER_PREFIX)));
-      // @formatter:on
-  }
-
-  /**
-   * Enable tracing by setting up SpanReceivers for the current process. If 
host name is null, it
-   * will be determined. If service name is null, the simple name of the class 
will be used.
-   */
-  public static void enableServerTraces(String hostname, String service,
-      AccumuloConfiguration conf) {
-    // @formatter:off
-      enableTracing(hostname, service,
-          conf.get(Property.TRACE_SPAN_RECEIVERS),
-          conf.get(Property.INSTANCE_ZK_HOST),
-          conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT),
-          conf.get(Property.TRACE_ZK_PATH),
-          
conf.getAllPropertiesWithPrefix(Property.TRACE_SPAN_RECEIVER_PREFIX));
-      // @formatter:on
-  }
+import io.opentelemetry.api.OpenTelemetry;
+import io.opentelemetry.api.common.Attributes;
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.api.trace.Tracer;
+import io.opentelemetry.api.trace.propagation.W3CTraceContextPropagator;
+import io.opentelemetry.context.Context;
+import io.opentelemetry.context.Scope;
+import io.opentelemetry.context.propagation.TextMapGetter;
 
-  private static void enableTracing(String hostname, String service, String 
spanReceivers,
-      String zookeepers, long timeout, String zkPath, Map<String,String> 
spanReceiverProps) {
+public class TraceUtil {
 
-    Map<String,
-        String> htraceConfigProps = 
spanReceiverProps.entrySet().stream().collect(Collectors.toMap(
-            k -> 
String.valueOf(k).substring(Property.TRACE_SPAN_RECEIVER_PREFIX.getKey().length()),
-            String::valueOf, (a, b) -> {
-              throw new AssertionError("duplicate can't happen");
-            }, HashMap::new));
-    htraceConfigProps.put(TRACER_ZK_HOST, zookeepers);
-    htraceConfigProps.put(TRACER_ZK_TIMEOUT, Long.toString(timeout));
-    htraceConfigProps.put(TRACER_ZK_PATH, zkPath);
-    if (hostname != null) {
-      htraceConfigProps.put(TRACE_HOST_PROPERTY, hostname);
-    }
-    if (service != null) {
-      htraceConfigProps.put(TRACE_SERVICE_PROPERTY, service);
-    }
-    ShutdownHookManager.get().addShutdownHook(TraceUtil::disable, 0);
-    synchronized (receivers) {
-      if (!receivers.isEmpty()) {
-        log.info("Already loaded span receivers, enable tracing does not need 
to be called again");
-        return;
+  public static final String INSTRUMENTATION_NAME = 
"io.opentelemetry.contrib.accumulo";
+  private static Tracer INSTANCE = null;
+
+  public static synchronized Tracer getTracer() {
+    if (INSTANCE == null) {
+      ServiceLoader<TracerProvider> loader = 
ServiceLoader.load(TracerProvider.class);
+      Optional<TracerProvider> first = loader.findFirst();
+      if (first.isEmpty()) {
+        // If no OpenTelemetry implementation on the ClassPath, then use the 
NOOP implementation
+        INSTANCE = OpenTelemetry.noop().getTracer(INSTRUMENTATION_NAME);
+      } else {
+        INSTANCE = first.get().getTracer();

Review comment:
       I've spent most of my time reviewing this PR thinking about how we 
should do the bootstrapping of an implementation.
   
   I don't think we want to use the ServiceLoader to load the first found on 
the class path, though this is a useful first pass implementation. Instead, I 
think we should specify the class name directly that implements the 
`OpenTelemetry` interface. Then, we just load that class, and call 
`getTracer(name)` on it, where `name` is also configurable (since we won't know 
what implementations are available from the `OpenTelemetry` object we've 
instantiated). This would align with how we've elected to inject other 
user-pluggable components, and would be analogous to how we previously 
configured HTrace span receivers (except, we'd be plugging in the whole Tracer 
instrumentation implementation, and not just the receivers).
   
   The Java ServiceLoader is great, but I think it adds some extra layers in 
our code that I wouldn't want to maintain. I would instead see if we can figure 
out how to contribute to the OpenTelemetry project with some kind of 
`ServiceLoader<OpenTelemetry>` baked in to it directly. That way, we'd benefit 
from being able to just use it out of the box. In the meantime, it's pretty 
straightforward to just explicitly specify the class name for an 
`OpenTelemetry` class.

##########
File path: core/src/main/java/org/apache/accumulo/core/trace/TraceUtil.java
##########
@@ -18,176 +18,82 @@
  */
 package org.apache.accumulo.core.trace;
 
-import java.io.IOException;
 import java.lang.reflect.InvocationHandler;
 import java.lang.reflect.InvocationTargetException;
 import java.lang.reflect.Proxy;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Properties;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
+import java.util.Optional;
+import java.util.ServiceLoader;
 
-import org.apache.accumulo.core.conf.AccumuloConfiguration;
-import org.apache.accumulo.core.conf.ClientProperty;
-import org.apache.accumulo.core.conf.ConfigurationTypeHelper;
-import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.trace.thrift.TInfo;
-import org.apache.hadoop.util.ShutdownHookManager;
-import org.apache.htrace.HTraceConfiguration;
-import org.apache.htrace.NullScope;
-import org.apache.htrace.Span;
-import org.apache.htrace.SpanReceiver;
-import org.apache.htrace.SpanReceiverBuilder;
-import org.apache.htrace.Trace;
-import org.apache.htrace.TraceInfo;
-import org.apache.htrace.TraceScope;
-import org.apache.htrace.impl.CountSampler;
-import org.apache.htrace.impl.ProbabilitySampler;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
 
-/**
- * Utility class for tracing within Accumulo. Not intended for client use!
- */
-public class TraceUtil {
-
-  private static final Logger log = LoggerFactory.getLogger(TraceUtil.class);
-  public static final String TRACE_HOST_PROPERTY = "trace.host";
-  public static final String TRACE_SERVICE_PROPERTY = "trace.service";
-  public static final String TRACER_ZK_HOST = "tracer.zookeeper.host";
-  public static final String TRACER_ZK_TIMEOUT = "tracer.zookeeper.timeout";
-  public static final String TRACER_ZK_PATH = "tracer.zookeeper.path";
-  private static final HashSet<SpanReceiver> receivers = new HashSet<>();
-
-  /**
-   * Enable tracing by setting up SpanReceivers for the current process. If 
host name is null, it
-   * will be determined. If service name is null, the simple name of the class 
will be used.
-   * Properties required in the client configuration include
-   * {@link org.apache.accumulo.core.conf.ClientProperty#TRACE_SPAN_RECEIVERS} 
and any properties
-   * specific to the span receiver.
-   */
-  public static void enableClientTraces(String hostname, String service, 
Properties properties) {
-    // @formatter:off
-      enableTracing(hostname, service,
-          ClientProperty.TRACE_SPAN_RECEIVERS.getValue(properties),
-          ClientProperty.INSTANCE_ZOOKEEPERS.getValue(properties),
-          ConfigurationTypeHelper
-              
.getTimeInMillis(ClientProperty.INSTANCE_ZOOKEEPERS_TIMEOUT.getValue(properties)),
-          ClientProperty.TRACE_ZOOKEEPER_PATH.getValue(properties),
-          ClientProperty.toMap(
-              ClientProperty.getPrefix(properties, 
ClientProperty.TRACE_SPAN_RECEIVER_PREFIX)));
-      // @formatter:on
-  }
-
-  /**
-   * Enable tracing by setting up SpanReceivers for the current process. If 
host name is null, it
-   * will be determined. If service name is null, the simple name of the class 
will be used.
-   */
-  public static void enableServerTraces(String hostname, String service,
-      AccumuloConfiguration conf) {
-    // @formatter:off
-      enableTracing(hostname, service,
-          conf.get(Property.TRACE_SPAN_RECEIVERS),
-          conf.get(Property.INSTANCE_ZK_HOST),
-          conf.getTimeInMillis(Property.INSTANCE_ZK_TIMEOUT),
-          conf.get(Property.TRACE_ZK_PATH),
-          
conf.getAllPropertiesWithPrefix(Property.TRACE_SPAN_RECEIVER_PREFIX));
-      // @formatter:on
-  }
+import io.opentelemetry.api.OpenTelemetry;
+import io.opentelemetry.api.common.Attributes;
+import io.opentelemetry.api.trace.Span;
+import io.opentelemetry.api.trace.Tracer;
+import io.opentelemetry.api.trace.propagation.W3CTraceContextPropagator;
+import io.opentelemetry.context.Context;
+import io.opentelemetry.context.Scope;
+import io.opentelemetry.context.propagation.TextMapGetter;
 
-  private static void enableTracing(String hostname, String service, String 
spanReceivers,
-      String zookeepers, long timeout, String zkPath, Map<String,String> 
spanReceiverProps) {
+public class TraceUtil {
 
-    Map<String,
-        String> htraceConfigProps = 
spanReceiverProps.entrySet().stream().collect(Collectors.toMap(
-            k -> 
String.valueOf(k).substring(Property.TRACE_SPAN_RECEIVER_PREFIX.getKey().length()),
-            String::valueOf, (a, b) -> {
-              throw new AssertionError("duplicate can't happen");
-            }, HashMap::new));
-    htraceConfigProps.put(TRACER_ZK_HOST, zookeepers);
-    htraceConfigProps.put(TRACER_ZK_TIMEOUT, Long.toString(timeout));
-    htraceConfigProps.put(TRACER_ZK_PATH, zkPath);
-    if (hostname != null) {
-      htraceConfigProps.put(TRACE_HOST_PROPERTY, hostname);
-    }
-    if (service != null) {
-      htraceConfigProps.put(TRACE_SERVICE_PROPERTY, service);
-    }
-    ShutdownHookManager.get().addShutdownHook(TraceUtil::disable, 0);
-    synchronized (receivers) {
-      if (!receivers.isEmpty()) {
-        log.info("Already loaded span receivers, enable tracing does not need 
to be called again");
-        return;
+  public static final String INSTRUMENTATION_NAME = 
"io.opentelemetry.contrib.accumulo";
+  private static Tracer INSTANCE = null;
+
+  public static synchronized Tracer getTracer() {
+    if (INSTANCE == null) {
+      ServiceLoader<TracerProvider> loader = 
ServiceLoader.load(TracerProvider.class);
+      Optional<TracerProvider> first = loader.findFirst();
+      if (first.isEmpty()) {
+        // If no OpenTelemetry implementation on the ClassPath, then use the 
NOOP implementation
+        INSTANCE = OpenTelemetry.noop().getTracer(INSTRUMENTATION_NAME);

Review comment:
       The NOOP implementation completely ignores the parameter to this. 
Looking at the code, it makes it hard to figure out what values are supposed to 
be used for other implementations. It seems that the `instrumentationName` is 
dependent on the specific OpenTelementry implementation being used. I found a 
more useful explanation at 
https://github.com/open-telemetry/oteps/blob/main/text/0016-named-tracers.md 
instead of the built-in Javadoc. In essence, it should refer to the library 
being plugged in, and not our own code, though I'm not sure that helps clarify 
it any more.

##########
File path: 
core/src/main/java/org/apache/accumulo/core/rpc/TraceProtocolFactory.java
##########
@@ -36,18 +37,19 @@
   @Override
   public TProtocol getProtocol(TTransport trans) {
     return new TCompactProtocol(trans) {
-      private TraceScope span = null;
+      private Span span = null;
 
       @Override
       public void writeMessageBegin(TMessage message) throws TException {
-        span = Trace.startSpan("client:" + message.name);
+        span = TraceUtil.getTracer().spanBuilder("client:" + 
message.name).startSpan();

Review comment:
       I think this is where we'd probably want to set the `CLIENT` span kind.

##########
File path: core/src/main/thrift/trace.thrift
##########
@@ -19,7 +19,8 @@
 namespace java org.apache.accumulo.core.trace.thrift
 namespace cpp org.apache.accumulo.core.trace.thrift
 
+# OpenTelemetry uses the standards at https://www.w3.org/TR/trace-context/
+# to propagate information across process boundaries.
 struct TInfo {
-  1:i64 traceId
-  2:i64 parentId
+  1:map<string,string> headers

Review comment:
       This breaks Thrift in unfriendly ways. Can break it more gracefully by 
incrementing the parameter number to 3:
   
   ```suggestion
     // 1:i64 traceId - removed in 2.1.0
     // 2:i64 parentId - removed in 2.1.0
     3:map<string,string> headers
   ```
   
   This may have an impact in serialized FaTE operations in ZK. I think I 
remember TInfo leaking into the Java serialization we put there. It should be 
fine, as one shouldn't have FaTE outstanding during upgrade, but it may have 
bitten us before.
   
   Since this is the only struct in this file, it might be best to use a new 
`TraceInfo` type in `client.thrift` and delete this file. However, that would 
require updating a lot of RPC methods to use the new type. It's probably best 
to do that, but maybe as a follow-on ticket, because it'd be a lot larger.

##########
File path: 
server/base/src/main/java/org/apache/accumulo/server/util/ChangeSecret.java
##########
@@ -63,18 +65,21 @@
   public static void main(String[] args) throws Exception {
     var siteConfig = SiteConfiguration.auto();
     var hadoopConf = new Configuration();
+
     Opts opts = new Opts();
-    List<String> argsList = new ArrayList<>(args.length + 2);
-    argsList.add("--old");
-    argsList.add("--new");
-    argsList.addAll(Arrays.asList(args));
-    try (TraceScope clientSpan =
-        opts.parseArgsAndTrace(ChangeSecret.class.getName(), 
argsList.toArray(new String[0]))) {
-
-      ServerContext context = opts.getServerContext();
-      try (var fs = context.getVolumeManager()) {
-        ServerDirs serverDirs = new ServerDirs(siteConfig, hadoopConf);
-        verifyHdfsWritePermission(serverDirs, fs);
+    ServerContext context = opts.getServerContext();
+    try (var fs = context.getVolumeManager()) {
+      ServerDirs serverDirs = new ServerDirs(siteConfig, hadoopConf);
+      verifyHdfsWritePermission(serverDirs, fs);
+
+      List<String> argsList = new ArrayList<>(args.length + 2);
+      argsList.add("--old");
+      argsList.add("--new");
+      argsList.addAll(Arrays.asList(args));
+
+      Span span = opts.parseArgsAndTrace(ChangeSecret.class.getName(), 
args).spanBuilder("main")
+          .startSpan();
+      try (Scope scope = span.makeCurrent()) {

Review comment:
       Honestly, I'm not sure why we're bothering to trace this utility class. 
Can probably just rip some of this out.

##########
File path: 
minicluster/src/main/java/org/apache/accumulo/miniclusterImpl/MiniAccumuloConfigImpl.java
##########
@@ -111,6 +111,7 @@ public MiniAccumuloConfigImpl(File dir, String 
rootPassword) {
   /**
    * Set directories and fully populate site config
    */
+  @SuppressWarnings("deprecation")

Review comment:
       Please avoid sweeping warnings suppressions, if possible. Assigning a 
deprecated enum to a local variable would work here, as is done for the other 
suppressions in this method below.

##########
File path: pom.xml
##########
@@ -954,6 +962,40 @@
                 
<unusedDeclaredDependency>org.apache.logging.log4j:log4j-1.2-api:jar:*</unusedDeclaredDependency>
                 
<unusedDeclaredDependency>org.apache.logging.log4j:log4j-slf4j-impl:jar:*</unusedDeclaredDependency>
                 
<unusedDeclaredDependency>org.apache.logging.log4j:log4j-web:jar:*</unusedDeclaredDependency>
+                <!-- ignore opentelemetry runtime dependencies -->
+                
<unusedDeclaredDependency>io.opentelemetry:opentelemetry-sdk:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>com.google.protobuf:protobuf-java-util:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.grpc:grpc-api:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.grpc:grpc-context:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.grpc:grpc-core:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.grpc:grpc-netty-shaded:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.grpc:grpc-protobuf:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.grpc:grpc-protobuf-lite:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.grpc:grpc-stub:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.opentelemetry:opentelemetry-api-metrics:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.opentelemetry:opentelemetry-exporter-jaeger:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.opentelemetry:opentelemetry-exporter-logging:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.opentelemetry:opentelemetry-exporter-prometheus:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.opentelemetry:opentelemetry-exporter-zipkin:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.opentelemetry:opentelemetry-exporter-otlp:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.opentelemetry:opentelemetry-exporter-otlp-common:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.opentelemetry:opentelemetry-exporter-otlp-trace:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.opentelemetry:opentelemetry-proto:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.opentelemetry:opentelemetry-sdk-common:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.opentelemetry:opentelemetry-sdk-metrics:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.opentelemetry:opentelemetry-sdk-trace:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.opentelemetry:opentelemetry-semconv:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.perfmark:perfmark-api:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.perfmark:perfmark-impl:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>com.squareup.okhttp3:okhttp:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>com.squareup.okio:okio:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.prometheus:simpleclient:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.prometheus:simpleclient_tracer_common:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.prometheus:simpleclient_tracer_otel:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.prometheus:simpleclient_tracer_otel_agent:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.zipkin.reporter2:zipkin-reporter:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.zipkin.reporter2:zipkin-sender-okhttp3:jar:*</unusedDeclaredDependency>
+                
<unusedDeclaredDependency>io.zipkin.zipkin2:zipkin:jar:*</unusedDeclaredDependency>

Review comment:
       I'm not sure most of these, if any, are needed. Many aren't even 
declared in this PR.

##########
File path: server/base/src/main/java/org/apache/accumulo/server/util/ZooZap.java
##########
@@ -122,14 +120,6 @@ public static void main(String[] args) {
         }
       }
 
-      if (opts.zapTracers) {
-        String path = siteConf.get(Property.TRACE_ZK_PATH);
-        try {
-          zapDirectory(zoo, path, opts);
-        } catch (Exception e) {
-          // do nothing if the /tracers node does not exist.
-        }
-      }

Review comment:
       It might be useful to keep the option, so it doesn't break scripts, even 
if it is now a noop.

##########
File path: pom.xml
##########
@@ -140,6 +140,7 @@
     <maven.compiler.target>11</maven.compiler.target>
     <!-- surefire/failsafe plugin option -->
     
<maven.test.redirectTestOutputToFile>true</maven.test.redirectTestOutputToFile>
+    <opentelemetry.version>1.5.0</opentelemetry.version>

Review comment:
       I'd inline this, since it's only used once, for the bom. Unless you 
think we really need it.

##########
File path: 
server/tracer/src/main/java/org/apache/accumulo/tracer/DefaultTracerProvider.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.accumulo.tracer;
+
+import org.apache.accumulo.core.trace.TracerProvider;
+import org.apache.commons.lang3.StringUtils;
+
+import com.google.auto.service.AutoService;
+
+import io.opentelemetry.api.OpenTelemetry;
+import io.opentelemetry.api.trace.Tracer;
+import io.opentelemetry.sdk.autoconfigure.OpenTelemetrySdkAutoConfiguration;
+
+@AutoService(TracerProvider.class)
+public class DefaultTracerProvider implements TracerProvider {
+
+  @Override
+  public Tracer getTracer() {
+    // Set the service name if not set
+    String svcNameEnvVar = System.getenv("OTEL_SERVICE_NAME");
+    String svcNameProp = System.getenv("otel.service.name");
+    String appName = System.getProperty("accumulo.application"); // set in 
accumulo-env.sh
+    if (StringUtils.isEmpty(svcNameEnvVar) && StringUtils.isEmpty(svcNameProp)
+        && !StringUtils.isEmpty(appName)) {
+      System.setProperty("otel.service.name", appName);
+    }
+    // Configures a global OpenTelemetry object that can be configured using
+    // the instructions at
+    // 
https://github.com/open-telemetry/opentelemetry-java/tree/main/sdk-extensions/autoconfigure
+    OpenTelemetry otel = OpenTelemetrySdkAutoConfiguration.initialize();
+    return otel.getTracer(INSTRUMENTATION_NAME);

Review comment:
       This can be replaced with:
   
   ```suggestion
       return GlobalOpenTelemetry.get();
   ```
   
   See other review comment where I suggested this interface should return an 
`OpenTelemetry` object, rather than a `Tracer`.

##########
File path: 
server/tserver/src/main/java/org/apache/accumulo/tserver/ThriftClientHandler.java
##########
@@ -976,17 +1008,31 @@ public void update(TInfo tinfo, TCredentials 
credentials, TKeyExtent tkeyExtent,
         // Instead of always looping on true, skip completely when durability 
is NONE.
         while (durability != Durability.NONE) {
           try {
-            try (TraceScope wal = Trace.startSpan("wal")) {
+            Span span2 = 
tracer.spanBuilder("ThriftClientHandler::update::wal").startSpan();
+            try (Scope scope = span2.makeCurrent()) {
               server.logger.log(session, mutation, durability);
+            } catch (Exception e) {
+              span2.recordException(e, 
Attributes.builder().put("exception.message", e.getMessage())
+                  .put("exception.escaped", true).build());
+              throw e;
+            } finally {
+              span2.end();
             }
             break;
           } catch (IOException ex) {
             log.warn("Error writing mutations to log", ex);
           }
         }
 
-        try (TraceScope commit = Trace.startSpan("commit")) {
+        Span span3 = 
tracer.spanBuilder("ThriftClientHandler::update::commit").startSpan();

Review comment:
       These span builders on the server side could specify the span kind as 
SERVER. Also, I'm not sure where this happens, but shouldn't these set the 
parent to the span information incoming from the client side of the RPC?

##########
File path: 
minicluster/src/main/java/org/apache/accumulo/minicluster/ServerType.java
##########
@@ -31,7 +31,6 @@
   ZOOKEEPER("ZooKeeper"),
   TABLET_SERVER("TServer"),
   GARBAGE_COLLECTOR("GC"),
-  TRACER("Tracer"),

Review comment:
       This is public API. The deprecated enum entry should be left in place 
for now.

##########
File path: 
server/tracer/src/main/java/org/apache/accumulo/tracer/DefaultTracerProvider.java
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.accumulo.tracer;
+
+import org.apache.accumulo.core.trace.TracerProvider;
+import org.apache.commons.lang3.StringUtils;
+
+import com.google.auto.service.AutoService;
+
+import io.opentelemetry.api.OpenTelemetry;
+import io.opentelemetry.api.trace.Tracer;
+import io.opentelemetry.sdk.autoconfigure.OpenTelemetrySdkAutoConfiguration;
+
+@AutoService(TracerProvider.class)
+public class DefaultTracerProvider implements TracerProvider {

Review comment:
       This could benefit from a better name other than having the word 
"Default" in it. It can still serve as the default, but a more descriptive name 
would help. Maybe `AutoconfigureOpenTelemetryFactory`?
   
   Also, this file can live in core.spi, along with the interface. We don't 
need a separate module for this. This entire module and its pom.xml can be 
removed.

##########
File path: 
server/monitor/src/main/java/org/apache/accumulo/monitor/rest/trace/TracesResource.java
##########
@@ -18,19 +18,9 @@
  */

Review comment:
       Could do more to delete these files entirely from the monitor, rather 
than simply make them blank slates.

##########
File path: 
server/base/src/main/java/org/apache/accumulo/server/compaction/FileCompactor.java
##########
@@ -336,7 +340,9 @@ private void compactLocalityGroup(String lgName, 
Set<ByteSequence> columnFamilie
       boolean inclusive, FileSKVWriter mfw, CompactionStats majCStats)
       throws IOException, CompactionCanceledException {
     ArrayList<FileSKVIterator> readers = new 
ArrayList<>(filesToCompact.size());
-    try (TraceScope span = Trace.startSpan("compact")) {
+    Tracer tracer = TraceUtil.getTracer();
+    Span compactSpan = 
tracer.spanBuilder("FileCompactor::compact").startSpan();

Review comment:
       Our naming conventions are terribly inconsistent. I like them sometime 
like what you did here, but the rest of our code seems to follow very random 
naming. If we did try to reduce this boiler-plate, it might be easier to 
enforce a better naming convention whatever TraceUtil method is used to 
encapsulate that boiler-plate.

##########
File path: server/tracer/pom.xml
##########
@@ -31,95 +31,294 @@
   <name>Apache Accumulo Tracer Server</name>
   <description>The tracer server for Apache Accumulo to listen for, and store, 
distributed tracing messages.</description>
   <dependencies>
-    <dependency>
-      <groupId>com.beust</groupId>
-      <artifactId>jcommander</artifactId>
-    </dependency>
     <dependency>
       <groupId>com.google.auto.service</groupId>
       <artifactId>auto-service</artifactId>
       <optional>true</optional>
     </dependency>
     <dependency>
-      <groupId>com.google.guava</groupId>
-      <artifactId>guava</artifactId>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-api</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-sdk</artifactId>
+    </dependency>
+    <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-sdk-extension-autoconfigure</artifactId>
+      <version>1.5.0-alpha</version>
     </dependency>
     <dependency>
       <groupId>org.apache.accumulo</groupId>
       <artifactId>accumulo-core</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.apache.accumulo</groupId>
-      <artifactId>accumulo-server-base</artifactId>
+      <groupId>org.apache.commons</groupId>
+      <artifactId>commons-lang3</artifactId>
     </dependency>
     <dependency>
-      <groupId>org.apache.accumulo</groupId>
-      <artifactId>accumulo-start</artifactId>
+      <groupId>com.google.protobuf</groupId>
+      <artifactId>protobuf-java-util</artifactId>
+      <version>3.17.2</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.squareup.okhttp3</groupId>
+      <artifactId>okhttp</artifactId>
+      <version>3.14.9</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>com.squareup.okio</groupId>
+      <artifactId>okio</artifactId>
+      <version>1.17.2</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-api</artifactId>
+      <version>1.38.0</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-context</artifactId>
+      <version>1.38.0</version>
+      <scope>runtime</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-client-api</artifactId>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-core</artifactId>
+      <version>1.38.0</version>
+      <scope>runtime</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.htrace</groupId>
-      <artifactId>htrace-core</artifactId>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-netty-shaded</artifactId>
+      <version>1.38.0</version>
+      <scope>runtime</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.thrift</groupId>
-      <artifactId>libthrift</artifactId>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-protobuf</artifactId>
+      <version>1.38.0</version>
+      <scope>runtime</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.zookeeper</groupId>
-      <artifactId>zookeeper</artifactId>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-protobuf-lite</artifactId>
+      <version>1.38.0</version>
+      <scope>runtime</scope>
     </dependency>
     <dependency>
-      <groupId>org.slf4j</groupId>
-      <artifactId>slf4j-api</artifactId>
+      <groupId>io.grpc</groupId>
+      <artifactId>grpc-stub</artifactId>
+      <version>1.38.0</version>
+      <scope>runtime</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.hadoop</groupId>
-      <artifactId>hadoop-client-runtime</artifactId>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-api-metrics</artifactId>
+      <version>1.5.0-alpha</version>
       <scope>runtime</scope>
     </dependency>
     <dependency>
-      <groupId>junit</groupId>
-      <artifactId>junit</artifactId>
-      <scope>test</scope>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-exporter-jaeger</artifactId>
+      <scope>runtime</scope>
     </dependency>
     <dependency>
-      <groupId>org.apache.logging.log4j</groupId>
-      <artifactId>log4j-slf4j-impl</artifactId>
-      <scope>test</scope>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-exporter-logging</artifactId>
+      <scope>runtime</scope>
     </dependency>
     <dependency>
-      <groupId>org.easymock</groupId>
-      <artifactId>easymock</artifactId>
-      <scope>test</scope>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-exporter-otlp</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-exporter-otlp-common</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-exporter-otlp-trace</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-exporter-prometheus</artifactId>
+      <version>1.5.0-alpha</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-exporter-zipkin</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-proto</artifactId>
+      <version>1.5.0-alpha</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-sdk-common</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-sdk-metrics</artifactId>
+      <version>1.5.0-alpha</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-sdk-trace</artifactId>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.opentelemetry</groupId>
+      <artifactId>opentelemetry-semconv</artifactId>
+      <version>1.5.0-alpha</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.perfmark</groupId>
+      <artifactId>perfmark-api</artifactId>
+      <version>0.23.0</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.perfmark</groupId>
+      <artifactId>perfmark-impl</artifactId>
+      <version>0.23.0</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.prometheus</groupId>
+      <artifactId>simpleclient</artifactId>
+      <version>0.11.0</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.prometheus</groupId>
+      <artifactId>simpleclient_tracer_common</artifactId>
+      <version>0.11.0</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.prometheus</groupId>
+      <artifactId>simpleclient_tracer_otel</artifactId>
+      <version>0.11.0</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.prometheus</groupId>
+      <artifactId>simpleclient_tracer_otel_agent</artifactId>
+      <version>0.11.0</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.zipkin.reporter2</groupId>
+      <artifactId>zipkin-reporter</artifactId>
+      <version>2.16.3</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.zipkin.reporter2</groupId>
+      <artifactId>zipkin-sender-okhttp3</artifactId>
+      <version>2.16.3</version>
+      <scope>runtime</scope>
+    </dependency>
+    <dependency>
+      <groupId>io.zipkin.zipkin2</groupId>
+      <artifactId>zipkin</artifactId>
+      <version>2.23.2</version>
+      <scope>runtime</scope>
     </dependency>
   </dependencies>
-  <profiles>
-    <profile>
-      <id>thrift</id>
-      <build>
-        <plugins>
-          <plugin>
-            <groupId>org.codehaus.mojo</groupId>
-            <artifactId>exec-maven-plugin</artifactId>
-            <executions>
-              <execution>
-                <id>generate-thrift</id>
-                <goals>
-                  <goal>exec</goal>
-                </goals>
-                <phase>generate-sources</phase>
-                <configuration>
-                  
<executable>${basedir}/src/main/scripts/generate-thrift.sh</executable>
-                </configuration>
-              </execution>
-            </executions>
-          </plugin>
-        </plugins>
-      </build>
-    </profile>
-  </profiles>
+  <build>
+    <plugins>
+      <plugin>
+        <groupId>org.apache.maven.plugins</groupId>
+        <artifactId>maven-shade-plugin</artifactId>
+        <!--  override version to prevent IllegalArgumentException -->
+        <version>3.2.4</version>

Review comment:
       Could you explain this override a bit more? And, if necessary, override 
it at the top-level POM, so we're consistent at least.
   
   Also, why are we shading anything? Is this just for testing/development 
while you were working on this? I don't think we should keep this.

##########
File path: 
server/base/src/main/java/org/apache/accumulo/server/problems/ProblemReports.java
##########
@@ -68,7 +68,7 @@
    * is reporting lots of problems, but problem reports can not be processed
    */
   private ExecutorService reportExecutor = ThreadPools.createThreadPool(0, 1, 
60, TimeUnit.SECONDS,
-      "acu-problem-reporter", new LinkedBlockingQueue<>(500), 
OptionalInt.empty(), false);

Review comment:
       The removal of this boolean makes the changeset a lot harder to review. 
I'd suggest avoiding that and adding it in a quick second commit after this PR 
is merged, and just ignoring the parameter in the method for now, so the 
changeset is smaller and easier to review, but it's probably easier to just 
leave it alone, since it's already done.




-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


Reply via email to