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

domgarguilo pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/accumulo.git

commit 95ed5e5ea601db90ba556237089356795aec52ae
Merge: 0c741b0d93 02193118d4
Author: Dom Garguilo <domgargu...@apache.org>
AuthorDate: Wed Sep 25 16:23:15 2024 -0400

    Merge remote-tracking branch 'upstream/3.1'

 .../manager/metrics/fate/FateMetricValues.java     | 28 ++++---
 .../accumulo/manager/metrics/fate/FateMetrics.java | 96 ++++++----------------
 .../metrics/fate/meta/MetaFateMetricValues.java    |  4 +-
 .../metrics/fate/user/UserFateMetricValues.java    |  3 +-
 4 files changed, 44 insertions(+), 87 deletions(-)

diff --cc 
server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java
index fb847afd17,702eaa7e56..ea494bd44d
--- 
a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java
+++ 
b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetricValues.java
@@@ -24,7 -25,11 +25,8 @@@ import java.util.Map
  import java.util.TreeMap;
  
  import org.apache.accumulo.core.fate.AdminUtil;
 -import org.apache.accumulo.core.fate.ReadOnlyTStore;
 -import org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus;
 -import org.apache.accumulo.server.ServerContext;
 -import org.apache.zookeeper.KeeperException;
 -import org.apache.zookeeper.data.Stat;
 +import org.apache.accumulo.core.fate.ReadOnlyFateStore;
++import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus;
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
@@@ -36,16 -41,21 +38,16 @@@ public abstract class FateMetricValues 
  
    private static final Logger log = 
LoggerFactory.getLogger(FateMetricValues.class);
  
 -  private final long updateTime;
 -  private final long currentFateOps;
 -  private final long zkFateChildOpsTotal;
 -  private final long zkConnectionErrors;
 +  protected final long updateTime;
 +  protected final long currentFateOps;
  
-   protected final Map<String,Long> txStateCounters;
-   protected final Map<String,Long> opTypeCounters;
+   private final EnumMap<TStatus,Long> txStateCounters;
+   private final Map<String,Long> opTypeCounters;
  
 -  private FateMetricValues(final long updateTime, final long currentFateOps,
 -      final long zkFateChildOpsTotal, final long zkConnectionErrors,
 +  protected FateMetricValues(final long updateTime, final long currentFateOps,
-       final Map<String,Long> txStateCounters, final Map<String,Long> 
opTypeCounters) {
+       final EnumMap<TStatus,Long> txStateCounters, final Map<String,Long> 
opTypeCounters) {
      this.updateTime = updateTime;
      this.currentFateOps = currentFateOps;
 -    this.zkFateChildOpsTotal = zkFateChildOpsTotal;
 -    this.zkConnectionErrors = zkConnectionErrors;
      this.txStateCounters = txStateCounters;
      this.opTypeCounters = opTypeCounters;
    }
@@@ -74,82 -92,145 +76,82 @@@
      return opTypeCounters;
    }
  
 -  /**
 -   * The FATE transaction stores the transaction type as a debug string in 
the transaction zknode.
 -   * This method returns a map of counters of the current occurrences of each 
operation type that is
 -   * IN_PROGRESS
 -   *
 -   * @param context Accumulo context
 -   * @param fateRootPath the zookeeper path to fate info
 -   * @param zooStore a readonly zoostore
 -   * @return the current FATE metric values.
 -   */
 -  public static FateMetricValues getFromZooKeeper(final ServerContext context,
 -      final String fateRootPath, final ReadOnlyTStore<FateMetrics> zooStore) {
 -
 -    FateMetricValues.Builder builder = FateMetricValues.builder();
 +  protected static <T extends AbstractBuilder<T,U>,U extends 
FateMetricValues> T
 +      getFateMetrics(final ReadOnlyFateStore<FateMetrics<U>> fateStore, T 
builder) {
  
 -    AdminUtil<FateMetrics> admin = new AdminUtil<>(false);
 +    AdminUtil<FateMetrics<U>> admin = new AdminUtil<>(false);
  
 -    try {
 +    List<AdminUtil.TransactionStatus> currFates =
 +        admin.getTransactionStatus(Map.of(fateStore.type(), fateStore), null, 
null, null);
  
 -      List<AdminUtil.TransactionStatus> currFates =
 -          admin.getTransactionStatus(zooStore, null, null);
 +    builder.withCurrentFateOps(currFates.size());
  
 -      builder.withCurrentFateOps(currFates.size());
 -
 -      // states are enumerated - create new map with counts initialized to 0.
 -      EnumMap<TStatus,Long> states = new EnumMap<>(TStatus.class);
 -      for (TStatus t : TStatus.values()) {
 -        states.put(t, 0L);
 -      }
 +    // states are enumerated - create new map with counts initialized to 0.
-     Map<String,Long> states = new TreeMap<>();
-     for (ReadOnlyFateStore.TStatus t : ReadOnlyFateStore.TStatus.values()) {
-       states.put(t.name(), 0L);
++    EnumMap<TStatus,Long> states = new EnumMap<>(TStatus.class);
++    for (TStatus t : TStatus.values()) {
++      states.put(t, 0L);
 +    }
  
 -      // op types are dynamic, no count initialization needed - clearing prev 
values will
 -      // need to be handled by the caller - this is just the counts for 
current op types.
 -      Map<String,Long> opTypeCounters = new TreeMap<>();
 +    // op types are dynamic, no count initialization needed - clearing prev 
values will
 +    // need to be handled by the caller - this is just the counts for current 
op types.
 +    Map<String,Long> opTypeCounters = new TreeMap<>();
  
 -      for (AdminUtil.TransactionStatus tx : currFates) {
 +    for (AdminUtil.TransactionStatus tx : currFates) {
  
-       String stateName = tx.getStatus().name();
 -        TStatus stateName = tx.getStatus();
++      TStatus stateName = tx.getStatus();
  
 -        // incr count for state
 -        states.merge(stateName, 1L, Long::sum);
 +      // incr count for state
 +      states.merge(stateName, 1L, Long::sum);
  
 -        // incr count for op type for for in_progress transactions.
 -        if (ReadOnlyTStore.TStatus.IN_PROGRESS.equals(tx.getStatus())) {
 -          String opType = tx.getTxName();
 -          if (opType == null || opType.isEmpty()) {
 -            opType = "UNKNOWN";
 -          }
 -          opTypeCounters.merge(opType, 1L, Long::sum);
 +      // incr count for op type for for in_progress transactions.
 +      if (ReadOnlyFateStore.TStatus.IN_PROGRESS.equals(tx.getStatus())) {
 +        String opType = tx.getTxName();
 +        if (opType == null || opType.isEmpty()) {
 +          opType = "UNKNOWN";
          }
 +        opTypeCounters.merge(opType, 1L, Long::sum);
        }
 -
 -      builder.withTxStateCounters(states);
 -      builder.withOpTypeCounters(opTypeCounters);
 -
 -      Stat node = 
context.getZooReaderWriter().getZooKeeper().exists(fateRootPath, false);
 -      builder.withZkFateChildOpsTotal(node.getCversion());
 -
 -      if (log.isTraceEnabled()) {
 -        log.trace(
 -            "ZkNodeStat: {czxid: {}, mzxid: {}, pzxid: {}, ctime: {}, mtime: 
{}, "
 -                + "version: {}, cversion: {}, num children: {}",
 -            node.getCzxid(), node.getMzxid(), node.getPzxid(), 
node.getCtime(), node.getMtime(),
 -            node.getVersion(), node.getCversion(), node.getNumChildren());
 -      }
 -
 -    } catch (KeeperException ex) {
 -      log.debug("Error connecting to ZooKeeper", ex);
 -      builder.incrZkConnectionErrors();
 -    } catch (InterruptedException ex) {
 -      Thread.currentThread().interrupt();
      }
  
 -    return builder.build();
 -  }
 -
 -  @Override
 -  public String toString() {
 -    return "FateMetricValues{updateTime=" + updateTime + ", currentFateOps=" 
+ currentFateOps
 -        + ", zkFateChildOpsTotal=" + zkFateChildOpsTotal + ", 
zkConnectionErrors="
 -        + zkConnectionErrors + '}';
 -  }
 +    builder.withTxStateCounters(states);
 +    builder.withOpTypeCounters(opTypeCounters);
  
 -  public static Builder builder() {
 -    return new Builder();
 +    return builder;
    }
  
 -  static class Builder {
 -
 -    private long currentFateOps = 0;
 -    private long zkFateChildOpsTotal = 0;
 -    private long zkConnectionErrors = 0;
 +  @SuppressWarnings("unchecked")
 +  protected static abstract class AbstractBuilder<T extends 
AbstractBuilder<T,U>,
 +      U extends FateMetricValues> {
  
 -    private final EnumMap<TStatus,Long> txStateCounters;
 -    private Map<String,Long> opTypeCounters;
 -
 -    Builder() {
 +    protected long currentFateOps = 0;
-     protected final Map<String,Long> txStateCounters;
++    protected final EnumMap<TStatus,Long> txStateCounters;
 +    protected Map<String,Long> opTypeCounters;
  
 +    protected AbstractBuilder() {
        // states are enumerated - create new map with counts initialized to 0.
-       txStateCounters = new TreeMap<>();
-       for (ReadOnlyFateStore.TStatus t : ReadOnlyFateStore.TStatus.values()) {
-         txStateCounters.put(t.name(), 0L);
+       txStateCounters = new EnumMap<>(TStatus.class);
+       for (TStatus t : TStatus.values()) {
+         txStateCounters.put(t, 0L);
        }
  
        opTypeCounters = Collections.emptyMap();
      }
  
 -    Builder withCurrentFateOps(final long value) {
 +    public T withCurrentFateOps(final long value) {
        this.currentFateOps = value;
 -      return this;
 -    }
 -
 -    Builder withZkFateChildOpsTotal(final long value) {
 -      this.zkFateChildOpsTotal = value;
 -      return this;
 -    }
 -
 -    Builder incrZkConnectionErrors() {
 -      this.zkConnectionErrors += 1L;
 -      return this;
 -    }
 -
 -    Builder withZkConnectionErrors(final long value) {
 -      this.zkConnectionErrors = value;
 -      return this;
 +      return (T) this;
      }
  
-     public T withTxStateCounters(final Map<String,Long> txStateCounters) {
 -    Builder withTxStateCounters(final EnumMap<TStatus,Long> txStateCounters) {
++    public T withTxStateCounters(final EnumMap<TStatus,Long> txStateCounters) 
{
        this.txStateCounters.putAll(txStateCounters);
 -      return this;
 +      return (T) this;
      }
  
 -    Builder withOpTypeCounters(final Map<String,Long> opTypeCounters) {
 +    public T withOpTypeCounters(final Map<String,Long> opTypeCounters) {
        this.opTypeCounters = new TreeMap<>(opTypeCounters);
 -      return this;
 +      return (T) this;
      }
  
 -    FateMetricValues build() {
 -      return new FateMetricValues(System.currentTimeMillis(), currentFateOps, 
zkFateChildOpsTotal,
 -          zkConnectionErrors, txStateCounters, opTypeCounters);
 -    }
 +    protected abstract U build();
    }
  }
diff --cc 
server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java
index be11b10584,18e376e48c..c04513f9fe
--- 
a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java
+++ 
b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/FateMetrics.java
@@@ -22,27 -23,31 +22,28 @@@ import static org.apache.accumulo.core.
  import static org.apache.accumulo.core.metrics.Metric.FATE_TX;
  import static org.apache.accumulo.core.metrics.Metric.FATE_TYPE_IN_PROGRESS;
  
- import java.util.List;
+ import java.util.EnumMap;
  import java.util.Map.Entry;
 +import java.util.Objects;
  import java.util.concurrent.ScheduledExecutorService;
  import java.util.concurrent.ScheduledFuture;
  import java.util.concurrent.TimeUnit;
  import java.util.concurrent.atomic.AtomicLong;
  
 -import org.apache.accumulo.core.Constants;
 -import org.apache.accumulo.core.fate.ReadOnlyTStore;
 -import org.apache.accumulo.core.fate.ReadOnlyTStore.TStatus;
 -import org.apache.accumulo.core.fate.ZooStore;
 +import org.apache.accumulo.core.fate.ReadOnlyFateStore;
++import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus;
  import org.apache.accumulo.core.metrics.MetricsProducer;
  import org.apache.accumulo.core.util.threads.ThreadPools;
  import org.apache.accumulo.server.ServerContext;
  import org.slf4j.Logger;
  import org.slf4j.LoggerFactory;
  
+ import io.micrometer.core.instrument.Gauge;
  import io.micrometer.core.instrument.MeterRegistry;
  import io.micrometer.core.instrument.Metrics;
- import io.micrometer.core.instrument.Tag;
  import io.micrometer.core.instrument.Tags;
  
 -public class FateMetrics implements MetricsProducer {
 +public abstract class FateMetrics<T extends FateMetricValues> implements 
MetricsProducer {
  
    private static final Logger log = 
LoggerFactory.getLogger(FateMetrics.class);
  
@@@ -51,113 -56,81 +52,64 @@@
  
    private static final String OP_TYPE_TAG = "op.type";
  
 -  private final ServerContext context;
 -  private final ReadOnlyTStore<FateMetrics> zooStore;
 -  private final String fateRootPath;
 -  private final long refreshDelay;
 +  protected final ServerContext context;
 +  protected final ReadOnlyFateStore<FateMetrics<T>> fateStore;
 +  protected final long refreshDelay;
  
-   protected final AtomicLong totalCurrentOpsGauge = new AtomicLong(0);
-   protected final AtomicLong newTxGauge = new AtomicLong(0);
-   protected final AtomicLong submittedTxGauge = new AtomicLong(0);
-   protected final AtomicLong inProgressTxGauge = new AtomicLong(0);
-   protected final AtomicLong failedInProgressTxGauge = new AtomicLong(0);
-   protected final AtomicLong failedTxGauge = new AtomicLong(0);
-   protected final AtomicLong successfulTxGauge = new AtomicLong(0);
-   protected final AtomicLong unknownTxGauge = new AtomicLong(0);
 -  private final AtomicLong totalCurrentOpsCount = new AtomicLong(0);
 -  private final AtomicLong totalOpsCount = new AtomicLong(0);
 -  private final AtomicLong fateErrorsCount = new AtomicLong(0);
++  protected final AtomicLong totalCurrentOpsCount = new AtomicLong(0);
+   private final EnumMap<TStatus,AtomicLong> txStatusCounters = new 
EnumMap<>(TStatus.class);
  
    public FateMetrics(final ServerContext context, final long 
minimumRefreshDelay) {
 -
      this.context = context;
 -    this.fateRootPath = context.getZooKeeperRoot() + Constants.ZFATE;
      this.refreshDelay = Math.max(DEFAULT_MIN_REFRESH_DELAY, 
minimumRefreshDelay);
 +    this.fateStore = Objects.requireNonNull(buildStore(context));
 +  }
  
 -    try {
 -      this.zooStore = new ZooStore<>(fateRootPath, 
context.getZooReaderWriter());
 -    } catch (KeeperException ex) {
 -      throw new IllegalStateException(
 -          "FATE Metrics - Failed to create zoo store - metrics unavailable", 
ex);
 -    } catch (InterruptedException ex) {
 -      Thread.currentThread().interrupt();
 -      throw new IllegalStateException(
 -          "FATE Metrics - Interrupt received while initializing zoo store");
 -    }
 +  protected abstract ReadOnlyFateStore<FateMetrics<T>> 
buildStore(ServerContext context);
  
 -    for (TStatus status : TStatus.values()) {
 -      txStatusCounters.put(status, new AtomicLong(0));
 -    }
 +  protected abstract T getMetricValues();
  
 +  protected void update() {
 +    update(getMetricValues());
    }
  
 -  private void update() {
 -
 -    FateMetricValues metricValues =
 -        FateMetricValues.getFromZooKeeper(context, fateRootPath, zooStore);
 -
 +  protected void update(T metricValues) {
-     totalCurrentOpsGauge.set(metricValues.getCurrentFateOps());
- 
-     for (Entry<String,Long> vals : 
metricValues.getTxStateCounters().entrySet()) {
-       switch (ReadOnlyFateStore.TStatus.valueOf(vals.getKey())) {
-         case NEW:
-           newTxGauge.set(vals.getValue());
-           break;
-         case SUBMITTED:
-           submittedTxGauge.set(vals.getValue());
-           break;
-         case IN_PROGRESS:
-           inProgressTxGauge.set(vals.getValue());
-           break;
-         case FAILED_IN_PROGRESS:
-           failedInProgressTxGauge.set(vals.getValue());
-           break;
-         case FAILED:
-           failedTxGauge.set(vals.getValue());
-           break;
-         case SUCCESSFUL:
-           successfulTxGauge.set(vals.getValue());
-           break;
-         case UNKNOWN:
-           unknownTxGauge.set(vals.getValue());
-           break;
-         default:
-           log.warn("Unhandled status type: {}", vals.getKey());
+     totalCurrentOpsCount.set(metricValues.getCurrentFateOps());
 -    totalOpsCount.set(metricValues.getZkFateChildOpsTotal());
 -    fateErrorsCount.set(metricValues.getZkConnectionErrors());
+ 
+     for (Entry<TStatus,Long> entry : 
metricValues.getTxStateCounters().entrySet()) {
+       AtomicLong counter = txStatusCounters.get(entry.getKey());
+       if (counter != null) {
+         counter.set(entry.getValue());
+       } else {
+         log.warn("Unhandled TStatus: {}", entry.getKey());
        }
      }
-     metricValues.getOpTypeCounters().forEach((name, count) -> {
-       Metrics.gauge(FATE_TYPE_IN_PROGRESS.getName(), Tags.of(OP_TYPE_TAG, 
name), count);
-     });
+ 
+     metricValues.getOpTypeCounters().forEach((name, count) -> Metrics
+         .gauge(FATE_TYPE_IN_PROGRESS.getName(), Tags.of(OP_TYPE_TAG, name), 
count));
    }
  
    @Override
    public void registerMetrics(final MeterRegistry registry) {
-     var type = fateStore.type().name().toLowerCase();
-     var instanceTypeTag = Tag.of("instanceType", type);
- 
-     registry.gauge(FATE_OPS.getName(), totalCurrentOpsGauge);
- 
-     registry.gauge(FATE_TX.getName(), List
-         .of(Tag.of("state", 
ReadOnlyFateStore.TStatus.NEW.name().toLowerCase()), instanceTypeTag),
-         newTxGauge);
-     registry.gauge(FATE_TX.getName(),
-         List.of(Tag.of("state", 
ReadOnlyFateStore.TStatus.SUBMITTED.name().toLowerCase()),
-             instanceTypeTag),
-         submittedTxGauge);
-     registry.gauge(FATE_TX.getName(),
-         List.of(Tag.of("state", 
ReadOnlyFateStore.TStatus.IN_PROGRESS.name().toLowerCase()),
-             instanceTypeTag),
-         inProgressTxGauge);
-     registry.gauge(FATE_TX.getName(),
-         List.of(Tag.of("state", 
ReadOnlyFateStore.TStatus.FAILED_IN_PROGRESS.name().toLowerCase()),
-             instanceTypeTag),
-         failedInProgressTxGauge);
-     registry.gauge(FATE_TX.getName(),
-         List.of(Tag.of("state", 
ReadOnlyFateStore.TStatus.FAILED.name().toLowerCase()),
-             instanceTypeTag),
-         failedTxGauge);
-     registry.gauge(FATE_TX.getName(),
-         List.of(Tag.of("state", 
ReadOnlyFateStore.TStatus.SUCCESSFUL.name().toLowerCase()),
-             instanceTypeTag),
-         successfulTxGauge);
-     registry.gauge(FATE_TX.getName(),
-         List.of(Tag.of("state", 
ReadOnlyFateStore.TStatus.UNKNOWN.name().toLowerCase()),
-             instanceTypeTag),
-         unknownTxGauge);
++    String type = fateStore.type().name().toLowerCase();
++
+     Gauge.builder(FATE_OPS.getName(), totalCurrentOpsCount, AtomicLong::get)
+         .description(FATE_OPS.getDescription()).register(registry);
 -    Gauge.builder(FATE_OPS_ACTIVITY.getName(), totalOpsCount, AtomicLong::get)
 -        .description(FATE_OPS_ACTIVITY.getDescription()).register(registry);
 -    Gauge.builder(FATE_ERRORS.getName(), fateErrorsCount, AtomicLong::get)
 -        .description(FATE_ERRORS.getDescription()).tags("type", 
"zk.connection").register(registry);
+ 
+     txStatusCounters.forEach((status, counter) -> Gauge
+         .builder(FATE_TX.getName(), counter, 
AtomicLong::get).description(FATE_TX.getDescription())
 -        .tags("state", status.name().toLowerCase()).register(registry));
 -
 -    update();
++        .tags("state", status.name().toLowerCase(), "instanceType", 
type).register(registry));
  
      // get fate status is read only operation - no reason to be nice on 
shutdown.
 -    ScheduledExecutorService scheduler =
 -        ThreadPools.getServerThreadPools().createScheduledExecutorService(1, 
"fateMetricsPoller");
 +    ScheduledExecutorService scheduler = ThreadPools.getServerThreadPools()
 +        .createScheduledExecutorService(1, type + "FateMetricsPoller");
      Runtime.getRuntime().addShutdownHook(new Thread(scheduler::shutdownNow));
  
 +    // Only update as part of the scheduler thread.
 +    // We have to call update() in a new thread because this method to
 +    // register metrics is called on start up in the Manager before it's 
finished
 +    // initializing, so we can't scan the User fate store until after startup 
is done.
 +    // If we called update() here in this method directly we would get stuck 
forever.
      ScheduledFuture<?> future = scheduler.scheduleAtFixedRate(() -> {
        try {
          update();
diff --cc 
server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/meta/MetaFateMetricValues.java
index f7ca146c4b,0000000000..a0923d1864
mode 100644,000000..100644
--- 
a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/meta/MetaFateMetricValues.java
+++ 
b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/meta/MetaFateMetricValues.java
@@@ -1,144 -1,0 +1,146 @@@
 +/*
 + * 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
 + *
 + *   https://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.manager.metrics.fate.meta;
 +
++import java.util.EnumMap;
 +import java.util.Map;
 +import java.util.Optional;
 +
 +import org.apache.accumulo.core.fate.FateInstanceType;
 +import org.apache.accumulo.core.fate.ReadOnlyFateStore;
++import org.apache.accumulo.core.fate.ReadOnlyFateStore.TStatus;
 +import org.apache.accumulo.manager.metrics.fate.FateMetricValues;
 +import org.apache.accumulo.manager.metrics.fate.FateMetrics;
 +import org.apache.accumulo.server.ServerContext;
 +import org.apache.zookeeper.KeeperException;
 +import org.apache.zookeeper.data.Stat;
 +import org.slf4j.Logger;
 +import org.slf4j.LoggerFactory;
 +
 +import com.google.common.base.Preconditions;
 +
 +public class MetaFateMetricValues extends FateMetricValues {
 +
 +  private static final Logger log = 
LoggerFactory.getLogger(MetaFateMetricValues.class);
 +
 +  private final long zkFateChildOpsTotal;
 +  private final long zkConnectionErrors;
 +
 +  protected MetaFateMetricValues(final long updateTime, final long 
currentFateOps,
 +      final long zkFateChildOpsTotal, final long zkConnectionErrors,
-       final Map<String,Long> txStateCounters, final Map<String,Long> 
opTypeCounters) {
++      final EnumMap<TStatus,Long> txStateCounters, final Map<String,Long> 
opTypeCounters) {
 +    super(updateTime, currentFateOps, txStateCounters, opTypeCounters);
 +
 +    this.zkFateChildOpsTotal = zkFateChildOpsTotal;
 +    this.zkConnectionErrors = zkConnectionErrors;
 +
 +  }
 +
 +  long getZkFateChildOpsTotal() {
 +    return zkFateChildOpsTotal;
 +  }
 +
 +  long getZkConnectionErrors() {
 +    return zkConnectionErrors;
 +  }
 +
 +  /**
 +   * The FATE transaction stores the transaction type as a debug string in 
the transaction zknode.
 +   * This method returns a map of counters of the current occurrences of each 
operation type that is
 +   * IN_PROGRESS
 +   *
 +   * @param context Accumulo context
 +   * @param fateRootPath the zookeeper path to fate info
 +   * @param metaFateStore a readonly MetaFateStore
 +   * @return the current FATE metric values.
 +   */
 +  public static MetaFateMetricValues getMetaStoreMetrics(final ServerContext 
context,
 +      final String fateRootPath,
 +      final ReadOnlyFateStore<FateMetrics<MetaFateMetricValues>> 
metaFateStore) {
 +    Preconditions.checkArgument(metaFateStore.type() == FateInstanceType.META,
 +        "Fate store must be of type %s", FateInstanceType.META);
 +
 +    Builder builder = null;
 +
 +    try {
 +      builder = getFateMetrics(metaFateStore, new Builder());
 +
 +      Stat node = 
context.getZooReaderWriter().getZooKeeper().exists(fateRootPath, false);
 +      builder.withZkFateChildOpsTotal(node.getCversion());
 +
 +      if (log.isTraceEnabled()) {
 +        log.trace(
 +            "ZkNodeStat: {czxid: {}, mzxid: {}, pzxid: {}, ctime: {}, mtime: 
{}, "
 +                + "version: {}, cversion: {}, num children: {}",
 +            node.getCzxid(), node.getMzxid(), node.getPzxid(), 
node.getCtime(), node.getMtime(),
 +            node.getVersion(), node.getCversion(), node.getNumChildren());
 +      }
 +
 +    } catch (KeeperException ex) {
 +      log.debug("Error connecting to ZooKeeper", ex);
 +      Optional.ofNullable(builder).ifPresent(Builder::incrZkConnectionErrors);
 +    } catch (InterruptedException ex) {
 +      Thread.currentThread().interrupt();
 +    }
 +
 +    return builder.build();
 +  }
 +
 +  @Override
 +  public String toString() {
 +    return "MetaFateMetricValues{updateTime=" + updateTime + ", 
currentFateOps=" + currentFateOps
 +        + ", zkFateChildOpsTotal=" + zkFateChildOpsTotal + ", 
zkConnectionErrors="
 +        + zkConnectionErrors + '}';
 +  }
 +
 +  static Builder builder() {
 +    return new Builder();
 +  }
 +
 +  static class Builder extends AbstractBuilder<Builder,MetaFateMetricValues> {
 +
 +    private long zkFateChildOpsTotal = 0;
 +    private long zkConnectionErrors = 0;
 +
 +    Builder() {
 +
 +    }
 +
 +    Builder withZkFateChildOpsTotal(final long value) {
 +      this.zkFateChildOpsTotal = value;
 +      return this;
 +    }
 +
 +    Builder incrZkConnectionErrors() {
 +      this.zkConnectionErrors += 1L;
 +      return this;
 +    }
 +
 +    Builder withZkConnectionErrors(final long value) {
 +      this.zkConnectionErrors = value;
 +      return this;
 +    }
 +
 +    @Override
 +    protected MetaFateMetricValues build() {
 +      return new MetaFateMetricValues(System.currentTimeMillis(), 
currentFateOps,
 +          zkFateChildOpsTotal, zkConnectionErrors, txStateCounters, 
opTypeCounters);
 +    }
 +  }
 +}
diff --cc 
server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/user/UserFateMetricValues.java
index 9d6c605a18,0000000000..bd2bf4d7b6
mode 100644,000000..100644
--- 
a/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/user/UserFateMetricValues.java
+++ 
b/server/manager/src/main/java/org/apache/accumulo/manager/metrics/fate/user/UserFateMetricValues.java
@@@ -1,63 -1,0 +1,64 @@@
 +/*
 + * 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
 + *
 + *   https://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.manager.metrics.fate.user;
 +
++import java.util.EnumMap;
 +import java.util.Map;
 +
 +import org.apache.accumulo.core.fate.FateInstanceType;
 +import org.apache.accumulo.core.fate.ReadOnlyFateStore;
 +import org.apache.accumulo.manager.metrics.fate.FateMetricValues;
 +import org.apache.accumulo.manager.metrics.fate.FateMetrics;
 +
 +import com.google.common.base.Preconditions;
 +
 +public class UserFateMetricValues extends FateMetricValues {
 +
 +  protected UserFateMetricValues(long updateTime, long currentFateOps,
-       Map<String,Long> txStateCounters, Map<String,Long> opTypeCounters) {
++      EnumMap<ReadOnlyFateStore.TStatus,Long> txStateCounters, 
Map<String,Long> opTypeCounters) {
 +    super(updateTime, currentFateOps, txStateCounters, opTypeCounters);
 +  }
 +
 +  public static UserFateMetricValues getUserStoreMetrics(
 +      final ReadOnlyFateStore<FateMetrics<UserFateMetricValues>> 
userFateStore) {
 +    Preconditions.checkArgument(userFateStore.type() == FateInstanceType.USER,
 +        "Fate store must be of type %s", FateInstanceType.USER);
 +    Builder builder = getFateMetrics(userFateStore, 
UserFateMetricValues.builder());
 +    return builder.build();
 +  }
 +
 +  @Override
 +  public String toString() {
 +    return "MetaFateMetricValues{updateTime=" + updateTime + ", 
currentFateOps=" + currentFateOps
 +        + '}';
 +  }
 +
 +  static Builder builder() {
 +    return new Builder();
 +  }
 +
 +  static class Builder extends AbstractBuilder<Builder,UserFateMetricValues> {
 +
 +    @Override
 +    protected UserFateMetricValues build() {
 +      return new UserFateMetricValues(System.currentTimeMillis(), 
currentFateOps, txStateCounters,
 +          opTypeCounters);
 +    }
 +  }
 +}

Reply via email to