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); + } + } +}