diff --git a/.github/workflows/ci-with-ratis.yml b/.github/workflows/ci-with-ratis.yml index 3c24f968eb7..0c69b0e28cf 100644 --- a/.github/workflows/ci-with-ratis.yml +++ b/.github/workflows/ci-with-ratis.yml @@ -50,3 +50,4 @@ jobs: secrets: inherit with: ratis_args: "-Dratis.version=${{ needs.ratis.outputs.ratis-version }} -Dratis.thirdparty.version=${{ needs.ratis.outputs.thirdparty-version }} -Dio.grpc.version=${{ needs.ratis.outputs.grpc-version }} -Dnetty.version=${{ needs.ratis.outputs.netty-version }} -Dgrpc.protobuf-compile.version=${{ needs.ratis.outputs.protobuf-version }}" + ref: ${{ github.event.inputs.ref }} diff --git a/.github/workflows/ci.yml b/.github/workflows/ci.yml index b8977160283..830707dce28 100644 --- a/.github/workflows/ci.yml +++ b/.github/workflows/ci.yml @@ -21,10 +21,15 @@ on: description: Version overrides from custom Ratis build default: '' required: false + ref: + type: string + description: Ozone ref (branch, tag or commit SHA) + default: '' + required: false env: FAIL_FAST: ${{ github.event_name == 'pull_request' }} MAVEN_OPTS: -Dhttp.keepAlive=false -Dmaven.wagon.http.pool=false -Dmaven.wagon.http.retryHandler.class=standard -Dmaven.wagon.http.retryHandler.count=3 - OZONE_WITH_COVERAGE: ${{ github.repository == 'apache/ozone' && github.event_name != 'pull_request' }} + OZONE_WITH_COVERAGE: ${{ github.repository == 'apache/ozone' && github.event_name == 'push' }} jobs: build-info: runs-on: ubuntu-20.04 @@ -41,18 +46,35 @@ jobs: needs-dependency-check: ${{ steps.selective-checks.outputs.needs-dependency-check }} needs-integration-tests: ${{ steps.selective-checks.outputs.needs-integration-tests }} needs-kubernetes-tests: ${{ steps.selective-checks.outputs.needs-kubernetes-tests }} + sha: ${{ steps.get-sha.outputs.sha }} steps: - - name: "Checkout ${{ github.ref }} ( ${{ github.sha }} )" + - name: "Checkout ${{ github.ref }} / ${{ github.sha }} (push)" uses: actions/checkout@v4 with: persist-credentials: false - - name: Fetch incoming commit ${{ github.sha }} with its parent + if: github.event_name == 'push' + - name: "Checkout ${{ github.sha }} with its parent (pull request)" uses: actions/checkout@v4 with: ref: ${{ github.sha }} fetch-depth: 2 persist-credentials: false if: github.event_name == 'pull_request' + - name: "Checkout ${{ inputs.ref }} given in workflow input (manual dispatch)" + uses: actions/checkout@v4 + with: + ref: ${{ inputs.ref }} + persist-credentials: false + if: github.event_name == 'workflow_dispatch' + - name: Get SHA of ${{ inputs.ref || github.ref }} + id: get-sha + run: | + if [[ "${GITHUB_EVENT_NAME}" == "workflow_dispatch" ]]; then + sha="$(git rev-parse --verify HEAD)" + else + sha="${GITHUB_SHA}" + fi + echo "sha=$sha" >> $GITHUB_OUTPUT - name: Selective checks id: selective-checks env: @@ -87,6 +109,8 @@ jobs: steps: - name: Checkout project uses: actions/checkout@v4 + with: + ref: ${{ needs.build-info.outputs.sha }} - name: Cache for npm dependencies uses: actions/cache@v4 with: @@ -211,11 +235,14 @@ jobs: steps: - name: Checkout project uses: actions/checkout@v4 + with: + ref: ${{ needs.build-info.outputs.sha }} if: matrix.check != 'bats' - name: Checkout project with history uses: actions/checkout@v4 with: fetch-depth: 0 + ref: ${{ needs.build-info.outputs.sha }} if: matrix.check == 'bats' - name: Cache for maven dependencies uses: actions/cache/restore@v4 @@ -264,6 +291,8 @@ jobs: steps: - name: Checkout project uses: actions/checkout@v4 + with: + ref: ${{ needs.build-info.outputs.sha }} - name: Cache for maven dependencies uses: actions/cache/restore@v4 with: @@ -310,6 +339,8 @@ jobs: steps: - name: Checkout project uses: actions/checkout@v4 + with: + ref: ${{ needs.build-info.outputs.sha }} - name: Download compiled Ozone binaries uses: actions/download-artifact@v4 with: @@ -339,6 +370,8 @@ jobs: steps: - name: Checkout project uses: actions/checkout@v4 + with: + ref: ${{ needs.build-info.outputs.sha }} - name: Cache for maven dependencies uses: actions/cache/restore@v4 with: @@ -384,6 +417,8 @@ jobs: steps: - name: Checkout project uses: actions/checkout@v4 + with: + ref: ${{ needs.build-info.outputs.sha }} - name: Download compiled Ozone binaries uses: actions/download-artifact@v4 with: @@ -426,6 +461,8 @@ jobs: steps: - name: Checkout project uses: actions/checkout@v4 + with: + ref: ${{ needs.build-info.outputs.sha }} - name: Download compiled Ozone binaries uses: actions/download-artifact@v4 with: @@ -475,6 +512,8 @@ jobs: steps: - name: Checkout project uses: actions/checkout@v4 + with: + ref: ${{ needs.build-info.outputs.sha }} - name: Cache for maven dependencies uses: actions/cache/restore@v4 with: @@ -520,8 +559,9 @@ jobs: coverage: runs-on: ubuntu-20.04 timeout-minutes: 30 - if: github.repository == 'apache/ozone' && github.event_name != 'pull_request' + if: github.repository == 'apache/ozone' && github.event_name == 'push' needs: + - build-info - acceptance - integration - native @@ -530,6 +570,7 @@ jobs: uses: actions/checkout@v4 with: fetch-depth: 0 + ref: ${{ needs.build-info.outputs.sha }} - name: Cache for maven dependencies uses: actions/cache/restore@v4 with: diff --git a/.mvn/extensions.xml b/.mvn/extensions.xml index 4a2117925f1..ac1c913fd50 100644 --- a/.mvn/extensions.xml +++ b/.mvn/extensions.xml @@ -24,7 +24,7 @@ com.gradle develocity-maven-extension - 1.21.4 + 1.21.5 com.gradle diff --git a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java index 5604de3f01f..330e94fda5a 100644 --- a/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java +++ b/hadoop-hdds/client/src/main/java/org/apache/hadoop/hdds/scm/XceiverClientRatis.java @@ -56,8 +56,10 @@ import org.apache.ratis.client.api.DataStreamApi; import org.apache.ratis.grpc.GrpcTlsConfig; import org.apache.ratis.proto.RaftProtos; +import org.apache.ratis.proto.RaftProtos.CommitInfoProto; import org.apache.ratis.proto.RaftProtos.ReplicationLevel; import org.apache.ratis.protocol.exceptions.GroupMismatchException; +import org.apache.ratis.protocol.exceptions.NotReplicatedException; import org.apache.ratis.protocol.exceptions.RaftException; import org.apache.ratis.protocol.RaftClientReply; import org.apache.ratis.retry.RetryPolicy; @@ -315,31 +317,46 @@ public XceiverClientReply watchForCommit(long index) throw e; } if (watchType == ReplicationLevel.ALL_COMMITTED) { - final RaftClientReply reply = getClient().async() - .watch(index, RaftProtos.ReplicationLevel.MAJORITY_COMMITTED) - .get(); - final XceiverClientReply clientReply = newWatchReply( - index, ReplicationLevel.MAJORITY_COMMITTED, index); - reply.getCommitInfos().stream() - .filter(i -> i.getCommitIndex() < index) - .forEach(proto -> { - UUID address = RatisHelper.toDatanodeId(proto.getServer()); - addDatanodetoReply(address, clientReply); - // since 3 way commit has failed, the updated map from now on will - // only store entries for those datanodes which have had successful - // replication. - commitInfoMap.remove(address); - LOG.info( - "Could not commit index {} on pipeline {} to all the nodes. " + - "Server {} has failed. Committed by majority.", - index, pipeline, address); - }); - return clientReply; + Throwable nre = + HddsClientUtils.containsException(e, NotReplicatedException.class); + Collection commitInfoProtoList; + if (nre instanceof NotReplicatedException) { + // If NotReplicatedException is thrown from the Datanode leader + // we can save one watch request round trip by using the CommitInfoProto + // in the NotReplicatedException + commitInfoProtoList = ((NotReplicatedException) nre).getCommitInfos(); + } else { + final RaftClientReply reply = getClient().async() + .watch(index, RaftProtos.ReplicationLevel.MAJORITY_COMMITTED) + .get(); + commitInfoProtoList = reply.getCommitInfos(); + } + return handleFailedAllCommit(index, commitInfoProtoList); } throw e; } } + private XceiverClientReply handleFailedAllCommit(long index, Collection commitInfoProtoList) { + final XceiverClientReply clientReply = newWatchReply( + index, ReplicationLevel.MAJORITY_COMMITTED, index); + commitInfoProtoList.stream() + .filter(i -> i.getCommitIndex() < index) + .forEach(proto -> { + UUID address = RatisHelper.toDatanodeId(proto.getServer()); + addDatanodetoReply(address, clientReply); + // since 3 way commit has failed, the updated map from now on will + // only store entries for those datanodes which have had successful + // replication. + commitInfoMap.remove(address); + LOG.info( + "Could not commit index {} on pipeline {} to all the nodes. " + + "Server {} has failed. Committed by majority.", + index, pipeline, address); + }); + return clientReply; + } + /** * Sends a given command to server gets a waitable future back. * diff --git a/hadoop-hdds/common/pom.xml b/hadoop-hdds/common/pom.xml index 41708e5c000..12bfe927b61 100644 --- a/hadoop-hdds/common/pom.xml +++ b/hadoop-hdds/common/pom.xml @@ -40,6 +40,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> info.picocli picocli + + com.github.stephenc.jcip + jcip-annotations + com.google.protobuf protobuf-java diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/StringUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/StringUtils.java index e7746d9d5ca..cb10dab51d5 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/StringUtils.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/StringUtils.java @@ -20,19 +20,9 @@ import java.nio.ByteBuffer; import java.nio.charset.Charset; import java.nio.charset.StandardCharsets; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Map; import com.google.common.base.Preconditions; -import org.apache.commons.lang3.SystemUtils; -import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.hdds.utils.SignalLogger; -import org.apache.hadoop.hdds.utils.VersionInfo; -import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.util.ShutdownHookManager; import org.apache.ratis.thirdparty.io.netty.buffer.Unpooled; -import org.slf4j.Logger; /** * Simple utility class to collection string conversion methods. @@ -44,11 +34,6 @@ private StringUtils() { private static final Charset UTF8 = StandardCharsets.UTF_8; - /** - * Priority of the StringUtils shutdown hook. - */ - private static final int SHUTDOWN_HOOK_PRIORITY = 0; - /** * Decode a specific range of bytes of the given byte array to a string * using UTF8. @@ -107,70 +92,6 @@ public static byte[] string2Bytes(String str) { return str.getBytes(UTF8); } - /** - * Return a message for logging. - * @param prefix prefix keyword for the message - * @param msg content of the message - * @return a message for logging - */ - public static String toStartupShutdownString(String prefix, String... msg) { - StringBuilder b = new StringBuilder(prefix); - b.append("\n/************************************************************"); - for (String s : msg) { - b.append("\n").append(prefix).append(s); - } - b.append("\n************************************************************/"); - return b.toString(); - } - - public static void startupShutdownMessage(VersionInfo versionInfo, - Class clazz, String[] args, Logger log, OzoneConfiguration conf) { - final String hostname = NetUtils.getHostname(); - final String className = clazz.getSimpleName(); - - if (log.isInfoEnabled()) { - log.info(createStartupShutdownMessage(versionInfo, className, hostname, - args, HddsUtils.processForLogging(conf))); - } - - if (SystemUtils.IS_OS_UNIX) { - try { - SignalLogger.INSTANCE.register(log); - } catch (Throwable t) { - log.warn("failed to register any UNIX signal loggers: ", t); - } - } - ShutdownHookManager.get().addShutdownHook( - () -> log.info(toStartupShutdownString("SHUTDOWN_MSG: ", - "Shutting down " + className + " at " + hostname)), - SHUTDOWN_HOOK_PRIORITY); - - } - - /** - * Generate the text for the startup/shutdown message of processes. - * @param className short name of the class - * @param hostname hostname - * @param args Command arguments - * @return a string to log. - */ - public static String createStartupShutdownMessage(VersionInfo versionInfo, - String className, String hostname, String[] args, - Map conf) { - return toStartupShutdownString("STARTUP_MSG: ", - "Starting " + className, - " host = " + hostname, - " args = " + (args != null ? Arrays.asList(args) : new ArrayList<>()), - " version = " + versionInfo.getVersion(), - " classpath = " + System.getProperty("java.class.path"), - " build = " + versionInfo.getUrl() + "/" - + versionInfo.getRevision() - + " ; compiled by '" + versionInfo.getUser() - + "' on " + versionInfo.getDate(), - " java = " + System.getProperty("java.version"), - " conf = " + conf); - } - public static String appendIfNotPresent(String str, char c) { Preconditions.checkNotNull(str, "Input string is null"); return str.isEmpty() || str.charAt(str.length() - 1) != c ? str + c : str; diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/DecommissionUtils.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/DecommissionUtils.java index 7d5b610b087..c176ad1464e 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/DecommissionUtils.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/client/DecommissionUtils.java @@ -126,7 +126,9 @@ public static Map getCountsMap(DatanodeDetails datanode, JsonNod Map countsMap, String errMsg) throws IOException { for (int i = 1; i <= numDecomNodes; i++) { - if (datanode.getHostName().equals(counts.get("tag.datanode." + i).asText())) { + String datanodeHostName = + (counts.get("tag.datanode." + i) != null) ? (counts.get("tag.datanode." + i).asText()) : ""; + if (datanode.getHostName().equals(datanodeHostName)) { JsonNode pipelinesDN = counts.get("PipelinesWaitingToCloseDN." + i); JsonNode underReplicatedDN = counts.get("UnderReplicatedDN." + i); JsonNode unclosedDN = counts.get("UnclosedContainersDN." + i); diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/fs/CachingSpaceUsageSource.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/fs/CachingSpaceUsageSource.java index b9a2f87a03d..9b9719386b3 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/fs/CachingSpaceUsageSource.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/fs/CachingSpaceUsageSource.java @@ -17,10 +17,12 @@ */ package org.apache.hadoop.hdds.fs; -import com.google.common.base.Preconditions; import com.google.common.util.concurrent.ThreadFactoryBuilder; import org.apache.hadoop.hdds.annotation.InterfaceAudience; import org.apache.hadoop.hdds.annotation.InterfaceStability; +import org.apache.ratis.util.AutoCloseableLock; +import org.apache.ratis.util.AutoCloseableReadWriteLock; +import org.apache.ratis.util.Preconditions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,7 +33,6 @@ import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ScheduledFuture; import java.util.concurrent.atomic.AtomicBoolean; -import java.util.concurrent.atomic.AtomicLong; import static java.util.concurrent.TimeUnit.MILLISECONDS; @@ -46,12 +47,16 @@ public class CachingSpaceUsageSource implements SpaceUsageSource { LoggerFactory.getLogger(CachingSpaceUsageSource.class); private final ScheduledExecutorService executor; - private final AtomicLong cachedValue = new AtomicLong(); + private final AutoCloseableReadWriteLock lock; + private long cachedUsedSpace; + private long cachedAvailable; + private long cachedCapacity; private final Duration refresh; private final SpaceUsageSource source; private final SpaceUsagePersistence persistence; private boolean running; - private ScheduledFuture scheduledFuture; + private ScheduledFuture updateUsedSpaceFuture; + private ScheduledFuture updateAvailableFuture; private final AtomicBoolean isRefreshRunning; public CachingSpaceUsageSource(SpaceUsageCheckParams params) { @@ -60,15 +65,16 @@ public CachingSpaceUsageSource(SpaceUsageCheckParams params) { CachingSpaceUsageSource(SpaceUsageCheckParams params, ScheduledExecutorService executor) { - Preconditions.checkArgument(params != null, "params == null"); + Preconditions.assertNotNull(params, "params == null"); refresh = params.getRefresh(); source = params.getSource(); + lock = new AutoCloseableReadWriteLock(source.toString()); persistence = params.getPersistence(); this.executor = executor; isRefreshRunning = new AtomicBoolean(); - Preconditions.checkArgument(refresh.isZero() == (executor == null), + Preconditions.assertTrue(refresh.isZero() == (executor == null), "executor should be provided if and only if refresh is requested"); loadInitialValue(); @@ -76,45 +82,81 @@ public CachingSpaceUsageSource(SpaceUsageCheckParams params) { @Override public long getCapacity() { - return source.getCapacity(); + try (AutoCloseableLock ignored = lock.readLock(null, null)) { + return cachedCapacity; + } } @Override public long getAvailable() { - return source.getAvailable(); + try (AutoCloseableLock ignored = lock.readLock(null, null)) { + return cachedAvailable; + } } @Override public long getUsedSpace() { - return cachedValue.get(); + try (AutoCloseableLock ignored = lock.readLock(null, null)) { + return cachedUsedSpace; + } + } + + @Override + public SpaceUsageSource snapshot() { + try (AutoCloseableLock ignored = lock.readLock(null, null)) { + return new Fixed(cachedCapacity, cachedAvailable, cachedUsedSpace); + } } public void incrementUsedSpace(long usedSpace) { - cachedValue.addAndGet(usedSpace); + if (usedSpace == 0) { + return; + } + Preconditions.assertTrue(usedSpace > 0, () -> usedSpace + " < 0"); + final long current, change; + try (AutoCloseableLock ignored = lock.writeLock(null, null)) { + current = cachedAvailable; + change = Math.min(current, usedSpace); + cachedAvailable -= change; + cachedUsedSpace += change; + } + + if (change != usedSpace) { + LOG.warn("Attempted to decrement available space to a negative value. Current: {}, Decrement: {}, Source: {}", + current, usedSpace, source); + } } public void decrementUsedSpace(long reclaimedSpace) { - cachedValue.updateAndGet(current -> { - long newValue = current - reclaimedSpace; - if (newValue < 0) { - if (current > 0) { - LOG.warn("Attempted to decrement used space to a negative value. " + - "Current: {}, Decrement: {}, Source: {}", - current, reclaimedSpace, source); - } - return 0; - } else { - return newValue; - } - }); + if (reclaimedSpace == 0) { + return; + } + Preconditions.assertTrue(reclaimedSpace > 0, () -> reclaimedSpace + " < 0"); + final long current, change; + try (AutoCloseableLock ignored = lock.writeLock(null, null)) { + current = cachedUsedSpace; + change = Math.min(current, reclaimedSpace); + cachedUsedSpace -= change; + cachedAvailable += change; + } + + if (change != reclaimedSpace) { + LOG.warn("Attempted to decrement used space to a negative value. Current: {}, Decrement: {}, Source: {}", + current, reclaimedSpace, source); + } } public void start() { if (executor != null) { - long initialDelay = cachedValue.get() > 0 ? refresh.toMillis() : 0; + long initialDelay = getUsedSpace() > 0 ? refresh.toMillis() : 0; if (!running) { - scheduledFuture = executor.scheduleWithFixedDelay( + updateUsedSpaceFuture = executor.scheduleWithFixedDelay( this::refresh, initialDelay, refresh.toMillis(), MILLISECONDS); + + long availableUpdateDelay = Math.min(refresh.toMillis(), Duration.ofMinutes(1).toMillis()); + updateAvailableFuture = executor.scheduleWithFixedDelay( + this::updateAvailable, availableUpdateDelay, availableUpdateDelay, MILLISECONDS); + running = true; } } else { @@ -126,8 +168,13 @@ public void shutdown() { persistence.save(this); // save cached value if (executor != null) { - if (running && scheduledFuture != null) { - scheduledFuture.cancel(true); + if (running) { + if (updateUsedSpaceFuture != null) { + updateUsedSpaceFuture.cancel(true); + } + if (updateAvailableFuture != null) { + updateAvailableFuture.cancel(true); + } } running = false; @@ -135,21 +182,48 @@ public void shutdown() { } } + /** Schedule immediate refresh. */ public void refreshNow() { - //refresh immediately executor.schedule(this::refresh, 0, MILLISECONDS); } + /** Loads {@code usedSpace} value from persistent source, if present. + * Also updates {@code available} and {@code capacity} from the {@code source}. */ private void loadInitialValue() { final OptionalLong initialValue = persistence.load(); - initialValue.ifPresent(cachedValue::set); + updateCachedValues(initialValue.orElse(0)); + } + + /** Updates {@code available} and {@code capacity} from the {@code source}. */ + private void updateAvailable() { + final long capacity = source.getCapacity(); + final long available = source.getAvailable(); + + try (AutoCloseableLock ignored = lock.writeLock(null, null)) { + cachedAvailable = available; + cachedCapacity = capacity; + } + } + + /** Updates {@code available} and {@code capacity} from the {@code source}, + * sets {@code usedSpace} to the specified {@code used} value. */ + private void updateCachedValues(long used) { + final long capacity = source.getCapacity(); + final long available = source.getAvailable(); + + try (AutoCloseableLock ignored = lock.writeLock(null, null)) { + cachedAvailable = available; + cachedCapacity = capacity; + cachedUsedSpace = used; + } } + /** Refreshes all 3 values. */ private void refresh() { //only one `refresh` can be running at a certain moment if (isRefreshRunning.compareAndSet(false, true)) { try { - cachedValue.set(source.getUsedSpace()); + updateCachedValues(source.getUsedSpace()); } catch (RuntimeException e) { LOG.warn("Error refreshing space usage for {}", source, e); } finally { diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/recon/ReconConfigKeys.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/recon/ReconConfigKeys.java index 3571d39bc8a..3ed9f4e58e1 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/recon/ReconConfigKeys.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/recon/ReconConfigKeys.java @@ -36,7 +36,8 @@ private ReconConfigKeys() { public static final String OZONE_RECON_DB_DIRS_PERMISSIONS = "ozone.recon.db.dirs.permissions"; - + public static final String OZONE_RECON_DB_DIRS_PERMISSIONS_DEFAULT = + "750"; public static final String OZONE_RECON_DATANODE_ADDRESS_KEY = "ozone.recon.datanode.address"; public static final String OZONE_RECON_ADDRESS_KEY = diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java index 2fc04e00f23..3ef9317ced0 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/ScmConfig.java @@ -63,51 +63,43 @@ public class ScmConfig extends ReconfigurableConfig { ) private String action; + private static final String DESCRIPTION_COMMON_CHOICES_OF_PIPELINE_CHOOSE_POLICY_IMPL = + "One of the following values can be used: " + + "(1) org.apache.hadoop.hdds.scm.pipeline.choose.algorithms.RandomPipelineChoosePolicy" + + " : chooses a pipeline randomly. " + + "(2) org.apache.hadoop.hdds.scm.pipeline.choose.algorithms.HealthyPipelineChoosePolicy" + + " : chooses a healthy pipeline randomly. " + + "(3) org.apache.hadoop.hdds.scm.pipeline.choose.algorithms.CapacityPipelineChoosePolicy" + + " : chooses the pipeline with lower utilization from two random pipelines. Note that" + + " random choose method will be executed twice in this policy." + + "(4) org.apache.hadoop.hdds.scm.pipeline.choose.algorithms.RoundRobinPipelineChoosePolicy" + + " : chooses a pipeline in a round robin fashion. Intended for troubleshooting and testing purposes only."; + + // hdds.scm.pipeline.choose.policy.impl @Config(key = "pipeline.choose.policy.impl", type = ConfigType.STRING, - defaultValue = "org.apache.hadoop.hdds.scm.pipeline.choose.algorithms" + - ".RandomPipelineChoosePolicy", + defaultValue = "org.apache.hadoop.hdds.scm.pipeline.choose.algorithms.RandomPipelineChoosePolicy", tags = { ConfigTag.SCM, ConfigTag.PIPELINE }, description = - "The full name of class which implements " - + "org.apache.hadoop.hdds.scm.PipelineChoosePolicy. " - + "The class decides which pipeline will be used to find or " - + "allocate Ratis containers. If not set, " - + "org.apache.hadoop.hdds.scm.pipeline.choose.algorithms." - + "RandomPipelineChoosePolicy will be used as default value. " - + "The following values can be used, " - + "(1) org.apache.hadoop.hdds.scm.pipeline.choose.algorithms." - + "RandomPipelineChoosePolicy : random choose one pipeline. " - + "(2) org.apache.hadoop.hdds.scm.pipeline.choose.algorithms." - + "HealthyPipelineChoosePolicy : random choose one healthy pipeline. " - + "(3) org.apache.hadoop.hdds.scm.pipeline.choose.algorithms." - + "CapacityPipelineChoosePolicy : choose the pipeline with lower " - + "utilization from the two pipelines. Note that random choose " - + "method will be executed twice in this policy." + "Sets the policy for choosing a pipeline for a Ratis container. The value should be " + + "the full name of a class which implements org.apache.hadoop.hdds.scm.PipelineChoosePolicy. " + + "The class decides which pipeline will be used to find or allocate Ratis containers. If not set, " + + "org.apache.hadoop.hdds.scm.pipeline.choose.algorithms.RandomPipelineChoosePolicy" + + " will be used as default value. " + DESCRIPTION_COMMON_CHOICES_OF_PIPELINE_CHOOSE_POLICY_IMPL ) private String pipelineChoosePolicyName; + // hdds.scm.ec.pipeline.choose.policy.impl @Config(key = "ec.pipeline.choose.policy.impl", type = ConfigType.STRING, - defaultValue = "org.apache.hadoop.hdds.scm.pipeline.choose.algorithms" + - ".RandomPipelineChoosePolicy", + defaultValue = "org.apache.hadoop.hdds.scm.pipeline.choose.algorithms.RandomPipelineChoosePolicy", tags = { ConfigTag.SCM, ConfigTag.PIPELINE }, description = - "The full name of class which implements " - + "org.apache.hadoop.hdds.scm.PipelineChoosePolicy. " - + "The class decides which pipeline will be used when " - + "selecting an EC Pipeline. If not set, " - + "org.apache.hadoop.hdds.scm.pipeline.choose.algorithms." - + "RandomPipelineChoosePolicy will be used as default value. " - + "The following values can be used, " - + "(1) org.apache.hadoop.hdds.scm.pipeline.choose.algorithms." - + "RandomPipelineChoosePolicy : random choose one pipeline. " - + "(2) org.apache.hadoop.hdds.scm.pipeline.choose.algorithms." - + "HealthyPipelineChoosePolicy : random choose one healthy pipeline. " - + "(3) org.apache.hadoop.hdds.scm.pipeline.choose.algorithms." - + "CapacityPipelineChoosePolicy : choose the pipeline with lower " - + "utilization from the two pipelines. Note that random choose " - + "method will be executed twice in this policy." + "Sets the policy for choosing an EC pipeline. The value should be " + + "the full name of a class which implements org.apache.hadoop.hdds.scm.PipelineChoosePolicy. " + + "The class decides which pipeline will be used when selecting an EC Pipeline. If not set, " + + "org.apache.hadoop.hdds.scm.pipeline.choose.algorithms.RandomPipelineChoosePolicy" + + " will be used as default value. " + DESCRIPTION_COMMON_CHOICES_OF_PIPELINE_CHOOSE_POLICY_IMPL ) private String ecPipelineChoosePolicyName; diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/BackgroundService.java b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/BackgroundService.java index c7edaaeeaa4..8f75da98e1c 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/BackgroundService.java +++ b/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/BackgroundService.java @@ -141,8 +141,11 @@ public synchronized void run() { if (LOG.isDebugEnabled()) { LOG.debug("task execution result size {}", result.getSize()); } - } catch (Exception e) { - LOG.warn("Background task execution failed", e); + } catch (Throwable e) { + LOG.error("Background task execution failed", e); + if (e instanceof Error) { + throw (Error) e; + } } finally { long endTime = System.nanoTime(); if (endTime - startTime > serviceTimeoutInNanos) { diff --git a/hadoop-hdds/common/src/main/resources/ozone-default.xml b/hadoop-hdds/common/src/main/resources/ozone-default.xml index 9b864a65c7e..fe6ef0d5e67 100644 --- a/hadoop-hdds/common/src/main/resources/ozone-default.xml +++ b/hadoop-hdds/common/src/main/resources/ozone-default.xml @@ -3057,7 +3057,7 @@ hdds.datanode.http.auth.type simple DATANODE, SECURITY, KERBEROS - simple or kerberos. If kerberos is set, Kerberos SPNEOGO + simple or kerberos. If kerberos is set, SPNEGO will be used for http authentication. @@ -3065,7 +3065,7 @@ ozone.freon.http.auth.type simple FREON, SECURITY - simple or kerberos. If kerberos is set, Kerberos SPNEOGO + simple or kerberos. If kerberos is set, SPNEGO will be used for http authentication. @@ -3073,7 +3073,7 @@ ozone.om.http.auth.type simple OM, SECURITY, KERBEROS - simple or kerberos. If kerberos is set, Kerberos SPNEOGO + simple or kerberos. If kerberos is set, SPNEGO will be used for http authentication. @@ -3081,7 +3081,7 @@ hdds.scm.http.auth.type simple OM, SECURITY, KERBEROS - simple or kerberos. If kerberos is set, Kerberos SPNEOGO + simple or kerberos. If kerberos is set, SPNEGO will be used for http authentication. @@ -3089,7 +3089,7 @@ ozone.recon.http.auth.type simple RECON, SECURITY, KERBEROS - simple or kerberos. If kerberos is set, Kerberos SPNEOGO + simple or kerberos. If kerberos is set, SPNEGO will be used for http authentication. @@ -3097,7 +3097,7 @@ ozone.s3g.http.auth.type simple S3GATEWAY, SECURITY, KERBEROS - simple or kerberos. If kerberos is set, Kerberos SPNEOGO + simple or kerberos. If kerberos is set, SPNEGO will be used for http authentication. diff --git a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/fs/TestCachingSpaceUsageSource.java b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/fs/TestCachingSpaceUsageSource.java index 8523861000e..b84ca130f27 100644 --- a/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/fs/TestCachingSpaceUsageSource.java +++ b/hadoop-hdds/common/src/test/java/org/apache/hadoop/hdds/fs/TestCachingSpaceUsageSource.java @@ -36,19 +36,20 @@ import static org.mockito.Mockito.anyLong; import static org.mockito.Mockito.eq; import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; import static org.mockito.Mockito.verify; import static org.mockito.Mockito.when; /** * Tests for {@link CachingSpaceUsageSource}. */ -public class TestCachingSpaceUsageSource { +class TestCachingSpaceUsageSource { @TempDir private static File dir; @Test - public void providesInitialValueUntilStarted() { + void providesInitialValueUntilStarted() { final long initialValue = validInitialValue(); SpaceUsageCheckParams params = paramsBuilder(new AtomicLong(initialValue)) .withRefresh(Duration.ZERO) @@ -57,10 +58,11 @@ public void providesInitialValueUntilStarted() { SpaceUsageSource subject = new CachingSpaceUsageSource(params); assertEquals(initialValue, subject.getUsedSpace()); + assertAvailableWasUpdated(params.getSource(), subject); } @Test - public void ignoresMissingInitialValue() { + void ignoresMissingInitialValue() { SpaceUsageCheckParams params = paramsBuilder() .withRefresh(Duration.ZERO) .build(); @@ -68,10 +70,11 @@ public void ignoresMissingInitialValue() { SpaceUsageSource subject = new CachingSpaceUsageSource(params); assertEquals(0, subject.getUsedSpace()); + assertAvailableWasUpdated(params.getSource(), subject); } @Test - public void updatesValueFromSourceUponStartIfPeriodicRefreshNotConfigured() { + void updatesValueFromSourceUponStartIfPeriodicRefreshNotConfigured() { AtomicLong savedValue = new AtomicLong(validInitialValue()); SpaceUsageCheckParams params = paramsBuilder(savedValue) .withRefresh(Duration.ZERO).build(); @@ -79,11 +82,11 @@ public void updatesValueFromSourceUponStartIfPeriodicRefreshNotConfigured() { CachingSpaceUsageSource subject = new CachingSpaceUsageSource(params); subject.start(); - assertSubjectWasRefreshed(params.getSource().getUsedSpace(), subject); + assertSubjectWasRefreshed(params.getSource(), subject); } @Test - public void schedulesRefreshWithDelayIfConfigured() { + void schedulesRefreshWithDelayIfConfigured() { long initialValue = validInitialValue(); AtomicLong savedValue = new AtomicLong(initialValue); SpaceUsageCheckParams params = paramsBuilder(savedValue) @@ -96,13 +99,13 @@ public void schedulesRefreshWithDelayIfConfigured() { subject.start(); verifyRefreshWasScheduled(executor, refresh.toMillis(), refresh); - assertSubjectWasRefreshed(params.getSource().getUsedSpace(), subject); + assertSubjectWasRefreshed(params.getSource(), subject); assertEquals(initialValue, savedValue.get(), "value should not have been saved to file yet"); } @Test - public void schedulesImmediateRefreshIfInitialValueMissing() { + void schedulesImmediateRefreshIfInitialValueMissing() { final long initialValue = missingInitialValue(); AtomicLong savedValue = new AtomicLong(initialValue); SpaceUsageCheckParams params = paramsBuilder(savedValue).build(); @@ -113,13 +116,13 @@ public void schedulesImmediateRefreshIfInitialValueMissing() { subject.start(); verifyRefreshWasScheduled(executor, 0L, params.getRefresh()); - assertSubjectWasRefreshed(params.getSource().getUsedSpace(), subject); + assertSubjectWasRefreshed(params.getSource(), subject); assertEquals(initialValue, savedValue.get(), "value should not have been saved to file yet"); } @Test - public void savesValueOnShutdown() { + void savesValueOnShutdown() { AtomicLong savedValue = new AtomicLong(validInitialValue()); SpaceUsageSource source = mock(SpaceUsageSource.class); final long usedSpace = 4L; @@ -138,22 +141,44 @@ public void savesValueOnShutdown() { "value should have been saved to file"); assertEquals(usedSpace, subject.getUsedSpace(), "no further updates from source expected"); - verify(future).cancel(true); + verify(future, times(2)).cancel(true); verify(executor).shutdown(); } @Test - public void testDecrementDoesNotGoNegative() { + void decrementUsedSpaceMoreThanCurrent() { SpaceUsageCheckParams params = paramsBuilder(new AtomicLong(50)) .withRefresh(Duration.ZERO) .build(); CachingSpaceUsageSource subject = new CachingSpaceUsageSource(params); + SpaceUsageSource original = subject.snapshot(); // Try to decrement more than the current value - subject.decrementUsedSpace(100); + final long change = original.getUsedSpace() * 2; + subject.decrementUsedSpace(change); - // Check that the value has been set to 0 + // should not drop below 0 assertEquals(0, subject.getUsedSpace()); + // available and used change by same amount (in opposite directions) + assertEquals(original.getAvailable() + original.getUsedSpace(), subject.getAvailable()); + } + + @Test + void decrementAvailableSpaceMoreThanCurrent() { + SpaceUsageCheckParams params = paramsBuilder(new AtomicLong(50)) + .withRefresh(Duration.ZERO) + .build(); + CachingSpaceUsageSource subject = new CachingSpaceUsageSource(params); + SpaceUsageSource original = subject.snapshot(); + + // Try to decrement more than the current value + final long change = original.getAvailable() * 2; + subject.incrementUsedSpace(change); + + // should not drop below 0 + assertEquals(0, subject.getAvailable()); + // available and used change by same amount (in opposite directions) + assertEquals(original.getUsedSpace() + original.getAvailable(), subject.getUsedSpace()); } private static long missingInitialValue() { @@ -197,14 +222,29 @@ private static void verifyRefreshWasScheduled( ScheduledExecutorService executor, long expectedInitialDelay, Duration refresh) { + // refresh usedSpace verify(executor).scheduleWithFixedDelay(any(), eq(expectedInitialDelay), eq(refresh.toMillis()), eq(TimeUnit.MILLISECONDS)); + + // update available/capacity + final long oneMinute = Duration.ofMinutes(1).toMillis(); + final long delay = Math.min(refresh.toMillis(), oneMinute); + verify(executor).scheduleWithFixedDelay(any(), eq(delay), + eq(delay), eq(TimeUnit.MILLISECONDS)); + } + + private static void assertAvailableWasUpdated(SpaceUsageSource source, + SpaceUsageSource subject) { + + assertEquals(source.getCapacity(), subject.getCapacity()); + assertEquals(source.getAvailable(), subject.getAvailable()); } - private static void assertSubjectWasRefreshed(long expected, + private static void assertSubjectWasRefreshed(SpaceUsageSource source, SpaceUsageSource subject) { - assertEquals(expected, subject.getUsedSpace(), + assertAvailableWasUpdated(source, subject); + assertEquals(source.getUsedSpace(), subject.getUsedSpace(), "subject should have been refreshed"); } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java index 55125e5fb37..02c6160545d 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/HddsDatanodeService.java @@ -33,7 +33,6 @@ import org.apache.hadoop.hdds.DFSConfigKeysLegacy; import org.apache.hadoop.hdds.DatanodeVersion; import org.apache.hadoop.hdds.HddsUtils; -import org.apache.hadoop.hdds.StringUtils; import org.apache.hadoop.hdds.cli.GenericCli; import org.apache.hadoop.hdds.cli.HddsVersionProvider; import org.apache.hadoop.hdds.conf.OzoneConfiguration; @@ -172,7 +171,7 @@ public static Logger getLogger() { public Void call() throws Exception { OzoneConfiguration configuration = createOzoneConfiguration(); if (printBanner) { - StringUtils.startupShutdownMessage(HddsVersionInfo.HDDS_VERSION_INFO, + HddsServerUtil.startupShutdownMessage(HddsVersionInfo.HDDS_VERSION_INFO, HddsDatanodeService.class, args, LOG, configuration); } start(configuration); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/CSMMetrics.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/CSMMetrics.java index 6bd15946184..3634ae34ac8 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/CSMMetrics.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/CSMMetrics.java @@ -39,6 +39,7 @@ public class CSMMetrics { public static final String SOURCE_NAME = CSMMetrics.class.getSimpleName(); + private RaftGroupId gid; // ratis op metrics metrics private @Metric MutableCounterLong numWriteStateMachineOps; @@ -50,6 +51,7 @@ public class CSMMetrics { private @Metric MutableRate transactionLatencyMs; private final EnumMap opsLatencyMs; + private final EnumMap opsQueueingDelay; private MetricsRegistry registry = null; // Failure Metrics @@ -64,15 +66,22 @@ public class CSMMetrics { private @Metric MutableCounterLong numDataCacheMiss; private @Metric MutableCounterLong numDataCacheHit; private @Metric MutableCounterLong numEvictedCacheCount; + private @Metric MutableCounterLong pendingApplyTransactions; private @Metric MutableRate applyTransactionNs; private @Metric MutableRate writeStateMachineDataNs; + private @Metric MutableRate writeStateMachineQueueingLatencyNs; + private @Metric MutableRate untilApplyTransactionNs; + private @Metric MutableRate startTransactionCompleteNs; - public CSMMetrics() { + public CSMMetrics(RaftGroupId gid) { + this.gid = gid; this.opsLatencyMs = new EnumMap<>(ContainerProtos.Type.class); + this.opsQueueingDelay = new EnumMap<>(ContainerProtos.Type.class); this.registry = new MetricsRegistry(CSMMetrics.class.getSimpleName()); for (ContainerProtos.Type type : ContainerProtos.Type.values()) { opsLatencyMs.put(type, registry.newRate(type.toString() + "Ms", type + " op")); + opsQueueingDelay.put(type, registry.newRate("queueingDelay" + type.toString() + "Ns", type + " op")); } } @@ -80,7 +89,12 @@ public static CSMMetrics create(RaftGroupId gid) { MetricsSystem ms = DefaultMetricsSystem.instance(); return ms.register(SOURCE_NAME + gid.toString(), "Container State Machine", - new CSMMetrics()); + new CSMMetrics(gid)); + } + + @Metric + public String getRaftGroupId() { + return gid.toString(); } public void incNumWriteStateMachineOps() { @@ -189,6 +203,11 @@ public void incPipelineLatencyMs(ContainerProtos.Type type, transactionLatencyMs.add(latencyMillis); } + public void recordQueueingDelay(ContainerProtos.Type type, + long latencyNanos) { + opsQueueingDelay.get(type).add(latencyNanos); + } + public void incNumStartTransactionVerifyFailures() { numStartTransactionVerifyFailures.incr(); } @@ -205,6 +224,19 @@ public void recordWriteStateMachineCompletionNs(long latencyNanos) { writeStateMachineDataNs.add(latencyNanos); } + + public void recordWriteStateMachineQueueingLatencyNs(long latencyNanos) { + writeStateMachineQueueingLatencyNs.add(latencyNanos); + } + + public void recordUntilApplyTransactionNs(long latencyNanos) { + untilApplyTransactionNs.add(latencyNanos); + } + + public void recordStartTransactionCompleteNs(long latencyNanos) { + startTransactionCompleteNs.add(latencyNanos); + } + public void incNumDataCacheMiss() { numDataCacheMiss.incr(); } @@ -216,8 +248,16 @@ public void incNumEvictedCacheCount() { numEvictedCacheCount.incr(); } + public void incPendingApplyTransactions() { + pendingApplyTransactions.incr(); + } + + public void decPendingApplyTransactions() { + pendingApplyTransactions.incr(-1); + } + public void unRegister() { MetricsSystem ms = DefaultMetricsSystem.instance(); - ms.unregisterSource(SOURCE_NAME); + ms.unregisterSource(SOURCE_NAME + gid.toString()); } } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java index 6351c746fe5..f583eadd7ea 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/ContainerStateMachine.java @@ -402,6 +402,7 @@ public TransactionContext startTransaction(LogEntryProto entry, RaftPeerRole rol @Override public TransactionContext startTransaction(RaftClientRequest request) throws IOException { + long startTime = Time.monotonicNowNanos(); final ContainerCommandRequestProto proto = message2ContainerCommandRequestProto(request.getMessage()); Preconditions.checkArgument(request.getRaftGroupId().equals(gid)); @@ -411,6 +412,8 @@ public TransactionContext startTransaction(RaftClientRequest request) .setStateMachine(this) .setServerRole(RaftPeerRole.LEADER); + metrics.incPendingApplyTransactions(); + try { dispatcher.validateContainerCommand(proto); } catch (IOException ioe) { @@ -440,9 +443,11 @@ public TransactionContext startTransaction(RaftClientRequest request) builder.setStateMachineData(write.getData()); } final ContainerCommandRequestProto containerCommandRequestProto = protoBuilder.build(); - return builder.setStateMachineContext(new Context(proto, containerCommandRequestProto)) + TransactionContext txnContext = builder.setStateMachineContext(new Context(proto, containerCommandRequestProto)) .setLogData(containerCommandRequestProto.toByteString()) .build(); + metrics.recordStartTransactionCompleteNs(Time.monotonicNowNanos() - startTime); + return txnContext; } private static ContainerCommandRequestProto getContainerCommandRequestProto( @@ -521,6 +526,8 @@ private CompletableFuture writeStateMachineData( CompletableFuture writeChunkFuture = CompletableFuture.supplyAsync(() -> { try { + metrics.recordWriteStateMachineQueueingLatencyNs( + Time.monotonicNowNanos() - startTime); return dispatchCommand(requestProto, context); } catch (Exception e) { LOG.error("{}: writeChunk writeStateMachineData failed: blockId" + @@ -884,6 +891,11 @@ private CompletableFuture applyTransaction( final CheckedSupplier task = () -> { try { + long timeNow = Time.monotonicNowNanos(); + long queueingDelay = timeNow - context.getStartTime(); + metrics.recordQueueingDelay(request.getCmdType(), queueingDelay); + // TODO: add a counter to track number of executing applyTransaction + // and queue size return dispatchCommand(request, context); } catch (Exception e) { exceptionHandler.accept(e); @@ -932,11 +944,13 @@ public CompletableFuture applyTransaction(TransactionContext trx) { .setTerm(trx.getLogEntry().getTerm()) .setLogIndex(index); + final Context context = (Context) trx.getStateMachineContext(); long applyTxnStartTime = Time.monotonicNowNanos(); + metrics.recordUntilApplyTransactionNs(applyTxnStartTime - context.getStartTime()); applyTransactionSemaphore.acquire(); metrics.incNumApplyTransactionsOps(); - final Context context = (Context) trx.getStateMachineContext(); + Objects.requireNonNull(context, "context == null"); final ContainerCommandRequestProto requestProto = context.getLogProto(); final Type cmdType = requestProto.getCmdType(); @@ -1021,6 +1035,9 @@ public CompletableFuture applyTransaction(TransactionContext trx) { applyTransactionSemaphore.release(); metrics.recordApplyTransactionCompletionNs( Time.monotonicNowNanos() - applyTxnStartTime); + if (trx.getServerRole() == RaftPeerRole.LEADER) { + metrics.decPendingApplyTransactions(); + } }); return applyTransactionFuture; } catch (InterruptedException e) { diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/DispatcherContext.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/DispatcherContext.java index d6c976cb389..15af2645352 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/DispatcherContext.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/DispatcherContext.java @@ -19,6 +19,7 @@ import org.apache.hadoop.hdds.annotation.InterfaceAudience; import org.apache.hadoop.hdds.annotation.InterfaceStability; +import org.apache.hadoop.util.Time; import org.apache.ratis.server.protocol.TermIndex; import java.util.Map; @@ -118,12 +119,15 @@ public static Op op(DispatcherContext context) { private final Map container2BCSIDMap; + private long startTime; + private DispatcherContext(Builder b) { this.op = Objects.requireNonNull(b.op, "op == null"); this.term = b.term; this.logIndex = b.logIndex; this.stage = b.stage; this.container2BCSIDMap = b.container2BCSIDMap; + this.startTime = Time.monotonicNowNanos(); } /** Use {@link DispatcherContext#op(DispatcherContext)} for handling null. */ @@ -147,6 +151,10 @@ public Map getContainer2BCSIDMap() { return container2BCSIDMap; } + public long getStartTime() { + return startTime; + } + @Override public String toString() { return op + "-" + stage + TermIndex.valueOf(term, logIndex); diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java index 75f0482a6ca..e1df809c8ae 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/transport/server/ratis/XceiverServerRatis.java @@ -184,7 +184,7 @@ private XceiverServerRatis(DatanodeDetails dd, StateContext context, ConfigurationSource conf, Parameters parameters) throws IOException { this.conf = conf; - Objects.requireNonNull(dd, "id == null"); + Objects.requireNonNull(dd, "DatanodeDetails == null"); datanodeDetails = dd; ratisServerConfig = conf.getObject(DatanodeRatisServerConfig.class); assignPorts(); @@ -372,12 +372,11 @@ public RaftProperties newRaftProperties() { } private void setRatisLeaderElectionTimeout(RaftProperties properties) { - long duration; TimeUnit leaderElectionMinTimeoutUnit = OzoneConfigKeys. HDDS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_DEFAULT .getUnit(); - duration = conf.getTimeDuration( + long duration = conf.getTimeDuration( OzoneConfigKeys.HDDS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_KEY, OzoneConfigKeys. HDDS_RATIS_LEADER_ELECTION_MINIMUM_TIMEOUT_DURATION_DEFAULT @@ -393,12 +392,10 @@ private void setRatisLeaderElectionTimeout(RaftProperties properties) { } private void setTimeoutForRetryCache(RaftProperties properties) { - TimeUnit timeUnit; - long duration; - timeUnit = + TimeUnit timeUnit = OzoneConfigKeys.HDDS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_DEFAULT .getUnit(); - duration = conf.getTimeDuration( + long duration = conf.getTimeDuration( OzoneConfigKeys.HDDS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_KEY, OzoneConfigKeys.HDDS_RATIS_SERVER_RETRY_CACHE_TIMEOUT_DURATION_DEFAULT .getDuration(), timeUnit); @@ -515,7 +512,6 @@ private RpcType setRpcType(RaftProperties properties) { } private void setPendingRequestsLimits(RaftProperties properties) { - long pendingRequestsBytesLimit = (long) conf.getStorageSize( OzoneConfigKeys.HDDS_CONTAINER_RATIS_LEADER_PENDING_BYTES_LIMIT, OzoneConfigKeys.HDDS_CONTAINER_RATIS_LEADER_PENDING_BYTES_LIMIT_DEFAULT, @@ -819,17 +815,6 @@ public List getPipelineReport() { } } - @VisibleForTesting - public List getPipelineIds() { - Iterable gids = server.getGroupIds(); - List pipelineIDs = new ArrayList<>(); - for (RaftGroupId groupId : gids) { - pipelineIDs.add(PipelineID.valueOf(groupId.getUuid())); - LOG.info("pipeline id {}", PipelineID.valueOf(groupId.getUuid())); - } - return pipelineIDs; - } - @Override public void addGroup(HddsProtos.PipelineID pipelineId, List peers) throws IOException { @@ -939,10 +924,9 @@ public void handleNodeLogFailure(RaftGroupId groupId, Throwable t) { } public long getMinReplicatedIndex(PipelineID pipelineID) throws IOException { - Long minIndex; GroupInfoReply reply = getServer() .getGroupInfo(createGroupInfoRequest(pipelineID.getProtobuf())); - minIndex = RatisHelper.getMinReplicatedIndex(reply.getCommitInfos()); + Long minIndex = RatisHelper.getMinReplicatedIndex(reply.getCommitInfos()); return minIndex == null ? -1 : minIndex; } diff --git a/hadoop-hdds/docs/content/feature/Nonrolling-Upgrade.md b/hadoop-hdds/docs/content/feature/Nonrolling-Upgrade.md index b18f585aebb..ac365d4f9b7 100644 --- a/hadoop-hdds/docs/content/feature/Nonrolling-Upgrade.md +++ b/hadoop-hdds/docs/content/feature/Nonrolling-Upgrade.md @@ -92,17 +92,3 @@ At this point, the cluster is upgraded to a pre-finalized state and fully operat ``` At this point, the cluster is finalized and the upgrade is complete. - -## Features Requiring Finalization - -Below is a list of backwards incompatible features and the version in which they were introduced. These features can only be used on a finalized ozone cluster with at least the specified version. Run `ozone version` to get the current version of your Ozone cluster. - -### Version 1.2.0 - -- [Prefix based File System Optimization]({{< relref "PrefixFSO.md" >}}) - - Although new 1.2.0 clusters can use this feature, it is currently not supported for clusters upgraded to 1.2.0, even after finalizing. - -- [SCM High Availability]({{< relref "SCM-HA.md" >}}) - - Although new 1.2.0 clusters can use this feature, it is currently not supported for clusters upgraded to 1.2.0, even after finalizing. - - diff --git a/hadoop-hdds/framework/pom.xml b/hadoop-hdds/framework/pom.xml index af0887ad16b..4a134e1c338 100644 --- a/hadoop-hdds/framework/pom.xml +++ b/hadoop-hdds/framework/pom.xml @@ -140,6 +140,10 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> com.fasterxml.jackson.datatype jackson-datatype-jsr310 + + com.github.jnr + jnr-posix + org.apache.hadoop hadoop-hdfs-client diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/BaseApprover.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/BaseApprover.java deleted file mode 100644 index 0ea2c60f020..00000000000 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/BaseApprover.java +++ /dev/null @@ -1,244 +0,0 @@ -/* - * 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.hadoop.hdds.security.x509.certificate.authority; - -import org.apache.hadoop.hdds.security.SecurityConfig; -import org.apache.hadoop.hdds.security.exception.SCMSecurityException; -import org.apache.hadoop.hdds.security.x509.certificate.authority.profile.PKIProfile; -import org.bouncycastle.asn1.ASN1Encodable; -import org.bouncycastle.asn1.ASN1ObjectIdentifier; -import org.bouncycastle.asn1.pkcs.Attribute; -import org.bouncycastle.asn1.pkcs.PKCSObjectIdentifiers; -import org.bouncycastle.asn1.x500.RDN; -import org.bouncycastle.asn1.x509.Extension; -import org.bouncycastle.asn1.x509.Extensions; -import org.bouncycastle.operator.ContentVerifierProvider; -import org.bouncycastle.operator.OperatorCreationException; -import org.bouncycastle.operator.jcajce.JcaContentVerifierProviderBuilder; -import org.bouncycastle.pkcs.PKCS10CertificationRequest; -import org.bouncycastle.pkcs.PKCSException; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -import java.util.ArrayList; -import java.util.List; -import java.util.Objects; -import java.util.concurrent.CompletableFuture; - -/** - * A base approver class for certificate approvals. - */ -public abstract class BaseApprover implements CertificateApprover { - private static final Logger LOG = - LoggerFactory.getLogger(BaseApprover.class); - private final PKIProfile profile; - private final SecurityConfig securityConfig; - - public BaseApprover(PKIProfile pkiProfile, SecurityConfig config) { - this.profile = Objects.requireNonNull(pkiProfile); - this.securityConfig = Objects.requireNonNull(config); - } - - /** - * Returns the PKI policy profile. - * - * @return PKIProfile - */ - public PKIProfile getProfile() { - return profile; - } - - /** - * Returns the Security config. - * - * @return SecurityConfig - */ - public SecurityConfig getSecurityConfig() { - return securityConfig; - } - - /** - * Returns the Attribute array that encodes extensions. - * - * @param request - Certificate Request - * @return - An Array of Attributes that encode various extensions requested - * in this certificate. - */ - Attribute[] getAttributes(PKCS10CertificationRequest request) { - Objects.requireNonNull(request); - return - request.getAttributes(PKCSObjectIdentifiers.pkcs_9_at_extensionRequest); - } - - /** - * Returns a list of Extensions encoded in a given attribute. - * - * @param attribute - Attribute to decode. - * @return - List of Extensions. - */ - List getExtensionsList(Attribute attribute) { - Objects.requireNonNull(attribute); - List extensionsList = new ArrayList<>(); - for (ASN1Encodable value : attribute.getAttributeValues()) { - if (value != null) { - Extensions extensions = Extensions.getInstance(value); - extensionsList.add(extensions); - } - } - return extensionsList; - } - - /** - * Returns the Extension decoded into a Java Collection. - * @param extensions - A set of Extensions in ASN.1. - * @return List of Decoded Extensions. - */ - List getIndividualExtension(Extensions extensions) { - Objects.requireNonNull(extensions); - List extenList = new ArrayList<>(); - for (ASN1ObjectIdentifier id : extensions.getExtensionOIDs()) { - if (id != null) { - Extension ext = extensions.getExtension(id); - if (ext != null) { - extenList.add(ext); - } - } - } - return extenList; - } - - - - /** - * This function verifies all extensions in the certificate. - * - * @param request - CSR - * @return - true if the extensions are acceptable by the profile, false - * otherwise. - */ - boolean verfiyExtensions(PKCS10CertificationRequest request) { - Objects.requireNonNull(request); - /* - * Inside a CSR we have - * 1. A list of Attributes - * 2. Inside each attribute a list of extensions. - * 3. We need to walk thru the each extension and verify they - * are expected and we can put that into a certificate. - */ - - for (Attribute attr : getAttributes(request)) { - for (Extensions extensionsList : getExtensionsList(attr)) { - for (Extension extension : getIndividualExtension(extensionsList)) { - if (!profile.validateExtension(extension)) { - LOG.error("Failed to verify extension. {}", - extension.getExtnId().getId()); - return false; - } - } - } - } - return true; - } - - /** - * Verifies the Signature on the CSR is valid. - * - * @param pkcs10Request - PCKS10 Request. - * @return True if it is valid, false otherwise. - * @throws OperatorCreationException - On Error. - * @throws PKCSException - on Error. - */ - boolean verifyPkcs10Request(PKCS10CertificationRequest pkcs10Request) - throws OperatorCreationException, PKCSException { - ContentVerifierProvider verifierProvider = new - JcaContentVerifierProviderBuilder() - .setProvider(this.securityConfig.getProvider()) - .build(pkcs10Request.getSubjectPublicKeyInfo()); - return - pkcs10Request.isSignatureValid(verifierProvider); - } - - /** - * {@inheritDoc} - */ - @Override - public CompletableFuture inspectCSR(PKCS10CertificationRequest csr) { - /** - * The base approver executes the following algorithm to verify that a - * CSR meets the PKI Profile criteria. - * - * 0. For time being (Until we have SCM HA) we will deny all request to - * become an intermediary CA. So we will not need to verify using CA - * profile, right now. - * - * 1. We verify the proof of possession. That is we verify the entity - * that sends us the CSR indeed has the private key for the said public key. - * - * 2. Then we will verify the RDNs meet the format and the Syntax that - * PKI profile dictates. - * - * 3. Then we decode each and every extension and ask if the PKI profile - * approves of these extension requests. - * - * 4. If all of these pass, We will return a Future which will point to - * the Certificate when finished. - */ - - CompletableFuture response = new CompletableFuture<>(); - try { - // Step 0: Verify this is not a CA Certificate. - // Will be done by the Ozone PKI profile for time being. - // If there are any basicConstraints, they will flagged as not - // supported for time being. - - // Step 1: Let us verify that Certificate is indeed signed by someone - // who has access to the private key. - if (!verifyPkcs10Request(csr)) { - LOG.error("Failed to verify the signature in CSR."); - response.completeExceptionally(new SCMSecurityException("Failed to " + - "verify the CSR.")); - } - - // Step 2: Verify the RDNs are in the correct format. - // TODO: Ozone Profile does not verify RDN now, so this call will pass. - for (RDN rdn : csr.getSubject().getRDNs()) { - if (!profile.validateRDN(rdn)) { - LOG.error("Failed in verifying RDNs"); - response.completeExceptionally(new SCMSecurityException("Failed to " + - "verify the RDNs. Please check the subject name.")); - } - } - - // Step 3: Verify the Extensions. - if (!verfiyExtensions(csr)) { - LOG.error("failed in verification of extensions."); - response.completeExceptionally(new SCMSecurityException("Failed to " + - "verify extensions.")); - } - - } catch (OperatorCreationException | PKCSException e) { - LOG.error("Approval Failure.", e); - response.completeExceptionally(new SCMSecurityException(e)); - } - return response; - } - - -} diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/CertificateApprover.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/CertificateApprover.java index 63c69bc06e7..3282f72a5dc 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/CertificateApprover.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/CertificateApprover.java @@ -20,7 +20,6 @@ package org.apache.hadoop.hdds.security.x509.certificate.authority; import org.apache.hadoop.hdds.security.SecurityConfig; -import org.bouncycastle.operator.OperatorCreationException; import org.bouncycastle.pkcs.PKCS10CertificationRequest; import java.io.IOException; @@ -55,7 +54,7 @@ public interface CertificateApprover { * @param certSerialId - the new certificate id. * @return Signed Certificate. * @throws IOException - On Error - * @throws OperatorCreationException - on Error. + * @throws CertificateException - on Error. */ @SuppressWarnings("ParameterNumber") X509Certificate sign( @@ -68,7 +67,7 @@ X509Certificate sign( String scmId, String clusterId, String certSerialId) - throws IOException, OperatorCreationException, CertificateException; + throws IOException, CertificateException; /** diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/DefaultApprover.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/DefaultApprover.java index e240856eae8..b2d62443b77 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/DefaultApprover.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/security/x509/certificate/authority/DefaultApprover.java @@ -19,10 +19,15 @@ package org.apache.hadoop.hdds.security.x509.certificate.authority; +import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.hdds.security.SecurityConfig; import org.apache.hadoop.hdds.security.exception.SCMSecurityException; import org.apache.hadoop.hdds.security.x509.certificate.authority.profile.PKIProfile; +import org.bouncycastle.asn1.ASN1Encodable; import org.bouncycastle.asn1.ASN1ObjectIdentifier; +import org.bouncycastle.asn1.pkcs.Attribute; +import org.bouncycastle.asn1.pkcs.PKCSObjectIdentifiers; +import org.bouncycastle.asn1.x500.RDN; import org.bouncycastle.asn1.x500.X500Name; import org.bouncycastle.asn1.x500.style.BCStyle; import org.bouncycastle.asn1.x509.AlgorithmIdentifier; @@ -37,11 +42,14 @@ import org.bouncycastle.crypto.util.PrivateKeyFactory; import org.bouncycastle.crypto.util.PublicKeyFactory; import org.bouncycastle.operator.ContentSigner; +import org.bouncycastle.operator.ContentVerifierProvider; import org.bouncycastle.operator.DefaultDigestAlgorithmIdentifierFinder; import org.bouncycastle.operator.DefaultSignatureAlgorithmIdentifierFinder; import org.bouncycastle.operator.OperatorCreationException; import org.bouncycastle.operator.bc.BcRSAContentSignerBuilder; +import org.bouncycastle.operator.jcajce.JcaContentVerifierProviderBuilder; import org.bouncycastle.pkcs.PKCS10CertificationRequest; +import org.bouncycastle.pkcs.PKCSException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -50,8 +58,12 @@ import java.security.PrivateKey; import java.security.cert.CertificateException; import java.security.cert.X509Certificate; +import java.util.ArrayList; import java.util.Arrays; import java.util.Date; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.CompletableFuture; import java.util.stream.Collectors; import static org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateSignRequest.getDistinguishedNameWithSN; @@ -60,10 +72,13 @@ /** * Default Approver used the by the DefaultCA. */ -public class DefaultApprover extends BaseApprover { +public class DefaultApprover implements CertificateApprover { + + private static final Logger LOG = LoggerFactory.getLogger(DefaultApprover.class); + + private final PKIProfile profile; + private final SecurityConfig securityConfig; - private static final Logger LOG = - LoggerFactory.getLogger(DefaultApprover.class); /** * Constructs the Default Approver. * @@ -71,7 +86,8 @@ public class DefaultApprover extends BaseApprover { * @param config - Security Config */ public DefaultApprover(PKIProfile pkiProfile, SecurityConfig config) { - super(pkiProfile, config); + this.profile = Objects.requireNonNull(pkiProfile); + this.securityConfig = Objects.requireNonNull(config); } /** @@ -101,8 +117,7 @@ public X509Certificate sign( PKCS10CertificationRequest certificationRequest, String scmId, String clusterId, - String certSerialId) throws IOException, - OperatorCreationException, CertificateException { + String certSerialId) throws IOException, CertificateException { AlgorithmIdentifier sigAlgId = new DefaultSignatureAlgorithmIdentifierFinder().find( @@ -161,22 +176,190 @@ public X509Certificate sign( .map(ASN1ObjectIdentifier::getId) .collect(Collectors.joining(", "))); LOG.info("Extensions to add to the certificate if they present in CSR: {}", - Arrays.stream(getProfile().getSupportedExtensions()) + Arrays.stream(profile.getSupportedExtensions()) .map(oid -> oid == null ? "null" : oid.getId()) .collect(Collectors.joining(", "))); - for (ASN1ObjectIdentifier extId : getProfile().getSupportedExtensions()) { + for (ASN1ObjectIdentifier extId : profile.getSupportedExtensions()) { Extension ext = exts.getExtension(extId); if (ext != null) { certificateGenerator.addExtension(ext); } } - ContentSigner sigGen = new BcRSAContentSignerBuilder(sigAlgId, digAlgId) - .build(asymmetricKP); + try { + ContentSigner sigGen = new BcRSAContentSignerBuilder(sigAlgId, digAlgId) + .build(asymmetricKP); + + //TODO: as part of HDDS-10743 ensure that converter is instantiated only once + return new JcaX509CertificateConverter().getCertificate(certificateGenerator.build(sigGen)); + } catch (OperatorCreationException oce) { + throw new CertificateException(oce); + } + } + + /** + * Returns the Attribute array that encodes extensions. + * + * @param request - Certificate Request + * @return - An Array of Attributes that encode various extensions requested + * in this certificate. + */ + private Attribute[] getAttributes(PKCS10CertificationRequest request) { + Objects.requireNonNull(request); - //TODO: as part of HDDS-10743 ensure that converter is instantiated only once - return new JcaX509CertificateConverter().getCertificate(certificateGenerator.build(sigGen)); + return request.getAttributes(PKCSObjectIdentifiers.pkcs_9_at_extensionRequest); + } + + /** + * Returns a list of Extensions encoded in a given attribute. + * + * @param attribute - Attribute to decode. + * @return - List of Extensions. + */ + private List getExtensionsList(Attribute attribute) { + Objects.requireNonNull(attribute); + List extensionsList = new ArrayList<>(); + for (ASN1Encodable value : attribute.getAttributeValues()) { + if (value != null) { + Extensions extensions = Extensions.getInstance(value); + extensionsList.add(extensions); + } + } + return extensionsList; + } + /** + * Returns the Extension decoded into a Java Collection. + * @param extensions - A set of Extensions in ASN.1. + * @return List of Decoded Extensions. + */ + private List getIndividualExtension(Extensions extensions) { + Objects.requireNonNull(extensions); + List extenList = new ArrayList<>(); + for (ASN1ObjectIdentifier id : extensions.getExtensionOIDs()) { + if (id != null) { + Extension ext = extensions.getExtension(id); + if (ext != null) { + extenList.add(ext); + } + } + } + return extenList; } + /** + * This function verifies all extensions in the certificate. + * + * @param request - CSR + * @return - true if the extensions are acceptable by the profile, false + * otherwise. + */ + @VisibleForTesting + boolean verfiyExtensions(PKCS10CertificationRequest request) { + Objects.requireNonNull(request); + /* + * Inside a CSR we have + * 1. A list of Attributes + * 2. Inside each attribute a list of extensions. + * 3. We need to walk thru the each extension and verify they + * are expected and we can put that into a certificate. + */ + + for (Attribute attr : getAttributes(request)) { + for (Extensions extensionsList : getExtensionsList(attr)) { + for (Extension extension : getIndividualExtension(extensionsList)) { + if (!profile.validateExtension(extension)) { + LOG.error("Failed to verify extension. {}", + extension.getExtnId().getId()); + return false; + } + } + } + } + return true; + } + + /** + * Verifies the Signature on the CSR is valid. + * + * @param pkcs10Request - PCKS10 Request. + * @return True if it is valid, false otherwise. + * @throws OperatorCreationException - On Error. + * @throws PKCSException - on Error. + */ + @VisibleForTesting + boolean verifyPkcs10Request(PKCS10CertificationRequest pkcs10Request) + throws OperatorCreationException, PKCSException { + ContentVerifierProvider verifierProvider = new + JcaContentVerifierProviderBuilder() + .setProvider(this.securityConfig.getProvider()) + .build(pkcs10Request.getSubjectPublicKeyInfo()); + return + pkcs10Request.isSignatureValid(verifierProvider); + } + + /** + * {@inheritDoc} + */ + @Override + public CompletableFuture inspectCSR(PKCS10CertificationRequest csr) { + /** + * The base approver executes the following algorithm to verify that a + * CSR meets the PKI Profile criteria. + * + * 0. For time being (Until we have SCM HA) we will deny all request to + * become an intermediary CA. So we will not need to verify using CA + * profile, right now. + * + * 1. We verify the proof of possession. That is we verify the entity + * that sends us the CSR indeed has the private key for the said public key. + * + * 2. Then we will verify the RDNs meet the format and the Syntax that + * PKI profile dictates. + * + * 3. Then we decode each and every extension and ask if the PKI profile + * approves of these extension requests. + * + * 4. If all of these pass, We will return a Future which will point to + * the Certificate when finished. + */ + + CompletableFuture response = new CompletableFuture<>(); + try { + // Step 0: Verify this is not a CA Certificate. + // Will be done by the Ozone PKI profile for time being. + // If there are any basicConstraints, they will flagged as not + // supported for time being. + + // Step 1: Let us verify that Certificate is indeed signed by someone + // who has access to the private key. + if (!verifyPkcs10Request(csr)) { + LOG.error("Failed to verify the signature in CSR."); + response.completeExceptionally(new SCMSecurityException("Failed to " + + "verify the CSR.")); + } + + // Step 2: Verify the RDNs are in the correct format. + // TODO: Ozone Profile does not verify RDN now, so this call will pass. + for (RDN rdn : csr.getSubject().getRDNs()) { + if (!profile.validateRDN(rdn)) { + LOG.error("Failed in verifying RDNs"); + response.completeExceptionally(new SCMSecurityException("Failed to " + + "verify the RDNs. Please check the subject name.")); + } + } + + // Step 3: Verify the Extensions. + if (!verfiyExtensions(csr)) { + LOG.error("failed in verification of extensions."); + response.completeExceptionally(new SCMSecurityException("Failed to " + + "verify extensions.")); + } + + } catch (OperatorCreationException | PKCSException e) { + LOG.error("Approval Failure.", e); + response.completeExceptionally(new SCMSecurityException(e)); + } + return response; + } } diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/HddsServerUtil.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/HddsServerUtil.java index 2b83c4d0594..c45e772c241 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/HddsServerUtil.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/HddsServerUtil.java @@ -25,8 +25,10 @@ import java.nio.file.Files; import java.nio.file.Path; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.List; +import java.util.Map; import java.util.Optional; import java.util.OptionalInt; import java.util.concurrent.TimeUnit; @@ -39,8 +41,10 @@ import org.apache.commons.compress.archivers.ArchiveOutputStream; import org.apache.commons.compress.archivers.tar.TarArchiveOutputStream; import org.apache.commons.compress.utils.IOUtils; +import org.apache.commons.lang3.SystemUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.hdds.DFSConfigKeysLegacy; +import org.apache.hadoop.hdds.HddsUtils; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocol.SCMSecurityProtocol; @@ -99,6 +103,7 @@ import static org.apache.hadoop.hdds.server.ServerUtils.sanitizeUserArgs; import static org.apache.hadoop.ozone.OzoneConfigKeys.HDDS_DATANODE_CONTAINER_DB_DIR; +import org.apache.hadoop.util.ShutdownHookManager; import org.rocksdb.RocksDBException; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -111,6 +116,8 @@ public final class HddsServerUtil { private HddsServerUtil() { } + private static final int SHUTDOWN_HOOK_PRIORITY = 0; + public static final String OZONE_RATIS_SNAPSHOT_COMPLETE_FLAG_NAME = "OZONE_RATIS_SNAPSHOT_COMPLETE"; @@ -678,4 +685,68 @@ public static IOException toIOException(String msg, RocksDBException e) { public static void addSuppressedLoggingExceptions(RPC.Server server) { server.addSuppressedLoggingExceptions(ServerNotLeaderException.class); } + + public static void startupShutdownMessage(VersionInfo versionInfo, + Class clazz, String[] args, Logger log, OzoneConfiguration conf) { + final String hostname = NetUtils.getHostname(); + final String className = clazz.getSimpleName(); + + if (log.isInfoEnabled()) { + log.info(createStartupShutdownMessage(versionInfo, className, hostname, + args, HddsUtils.processForLogging(conf))); + } + + if (SystemUtils.IS_OS_UNIX) { + try { + SignalLogger.INSTANCE.register(log); + } catch (Throwable t) { + log.warn("failed to register any UNIX signal loggers: ", t); + } + } + ShutdownHookManager.get().addShutdownHook( + () -> log.info(toStartupShutdownString("SHUTDOWN_MSG: ", + "Shutting down " + className + " at " + hostname)), + SHUTDOWN_HOOK_PRIORITY); + } + + /** + * Return a message for logging. + * @param prefix prefix keyword for the message + * @param msg content of the message + * @return a message for logging + */ + public static String toStartupShutdownString(String prefix, String... msg) { + StringBuilder b = new StringBuilder(prefix); + b.append("\n/************************************************************"); + for (String s : msg) { + b.append("\n").append(prefix).append(s); + } + b.append("\n************************************************************/"); + return b.toString(); + } + + /** + * Generate the text for the startup/shutdown message of processes. + * @param className short name of the class + * @param hostname hostname + * @param args Command arguments + * @return a string to log. + */ + public static String createStartupShutdownMessage(VersionInfo versionInfo, + String className, String hostname, String[] args, + Map conf) { + return toStartupShutdownString("STARTUP_MSG: ", + "Starting " + className, + " host = " + hostname, + " args = " + (args != null ? Arrays.asList(args) : new ArrayList<>()), + " version = " + versionInfo.getVersion(), + " classpath = " + System.getProperty("java.class.path"), + " build = " + versionInfo.getUrl() + "/" + + versionInfo.getRevision() + + " ; compiled by '" + versionInfo.getUser() + + "' on " + versionInfo.getDate(), + " java = " + System.getProperty("java.version"), + " conf = " + conf); + } + } diff --git a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/SignalLogger.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/SignalLogger.java similarity index 74% rename from hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/SignalLogger.java rename to hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/SignalLogger.java index 5782f806d54..df6f18cf6ee 100644 --- a/hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/utils/SignalLogger.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/SignalLogger.java @@ -18,10 +18,18 @@ package org.apache.hadoop.hdds.utils; +import jnr.constants.platform.Signal; +import jnr.posix.POSIX; +import jnr.posix.POSIXFactory; +import jnr.posix.SignalHandler; +import jnr.posix.util.DefaultPOSIXHandler; import org.apache.hadoop.hdds.annotation.InterfaceAudience; import org.apache.hadoop.hdds.annotation.InterfaceStability; import org.slf4j.Logger; +import java.util.EnumSet; +import java.util.Set; + /** * This class logs a message whenever we're about to exit on a UNIX signal. * This is helpful for determining the root cause of a process' exit. @@ -34,20 +42,18 @@ public enum SignalLogger { INSTANCE; - private static final String[] SIGNALS = new String[] {"TERM", "HUP", "INT"}; + private static final Set SIGNALS = EnumSet.of(Signal.SIGHUP, Signal.SIGINT, Signal.SIGTERM); private boolean registered = false; /** * Our signal handler. */ - private static class Handler implements sun.misc.SignalHandler { + private static class Handler implements SignalHandler { private final Logger log; - private final sun.misc.SignalHandler prevHandler; - Handler(String name, Logger log) { + Handler(Logger log) { this.log = log; - prevHandler = sun.misc.Signal.handle(new sun.misc.Signal(name), this); } /** @@ -56,10 +62,8 @@ private static class Handler implements sun.misc.SignalHandler { * @param signal The incoming signal */ @Override - public void handle(sun.misc.Signal signal) { - log.error("RECEIVED SIGNAL {}: SIG{}", - signal.getNumber(), signal.getName()); - prevHandler.handle(signal); + public void handle(int signal) { + log.error("RECEIVED SIGNAL {}: {}", signal, Signal.valueOf(signal)); } } @@ -76,10 +80,12 @@ public void register(final Logger log) { StringBuilder bld = new StringBuilder(); bld.append("registered UNIX signal handlers for ["); String separator = ""; - for (String signalName : SIGNALS) { + final POSIX posix = POSIXFactory.getPOSIX(new DefaultPOSIXHandler(), true); + final Handler handler = new Handler(log); + for (Signal signal : SIGNALS) { try { - new Handler(signalName, log); - bld.append(separator).append(signalName); + posix.signal(signal, handler); + bld.append(separator).append(signal.name()); separator = ", "; } catch (Exception e) { log.debug("", e); diff --git a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDatabase.java b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDatabase.java index 957e9d27c38..c441ec929c7 100644 --- a/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDatabase.java +++ b/hadoop-hdds/framework/src/main/java/org/apache/hadoop/hdds/utils/db/RocksDatabase.java @@ -681,7 +681,7 @@ Integer get(ColumnFamily family, ByteBuffer key, ByteBuffer outValue) try (UncheckedAutoCloseable ignored = acquire()) { final int size = db.get().get(family.getHandle(), DEFAULT_READ_OPTION, key, outValue); - LOG.debug("get: size={}, remaining={}", + LOG.trace("get: size={}, remaining={}", size, outValue.asReadOnlyBuffer().remaining()); return size == ManagedRocksDB.NOT_FOUND ? null : size; } catch (RocksDBException e) { diff --git a/hadoop-hdds/rocks-native/pom.xml b/hadoop-hdds/rocks-native/pom.xml index 60775a88e6d..1c8f01bff93 100644 --- a/hadoop-hdds/rocks-native/pom.xml +++ b/hadoop-hdds/rocks-native/pom.xml @@ -46,11 +46,6 @@ test - - - 8 - 8 - diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ScmUtils.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ScmUtils.java index cc6147c7a64..3f6b90b05e2 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ScmUtils.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/ScmUtils.java @@ -23,10 +23,8 @@ import java.util.concurrent.BlockingQueue; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.conf.OzoneConfiguration; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmOps; import org.apache.hadoop.hdds.scm.events.SCMEvents; import org.apache.hadoop.hdds.scm.exceptions.SCMException; -import org.apache.hadoop.hdds.scm.safemode.Precheck; import org.apache.hadoop.hdds.scm.security.RootCARotationManager; import org.apache.hadoop.hdds.scm.server.ContainerReportQueue; @@ -39,7 +37,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.io.File; import java.net.InetSocketAddress; import java.util.Optional; import java.util.OptionalInt; @@ -75,30 +72,6 @@ public final class ScmUtils { private ScmUtils() { } - /** - * Perform all prechecks for given scm operation. - * - * @param operation - * @param preChecks prechecks to be performed - */ - public static void preCheck(ScmOps operation, Precheck... preChecks) - throws SCMException { - for (Precheck preCheck : preChecks) { - preCheck.check(operation); - } - } - - /** - * Create SCM directory file based on given path. - */ - public static File createSCMDir(String dirPath) { - File dirFile = new File(dirPath); - if (!dirFile.mkdirs() && !dirFile.exists()) { - throw new IllegalArgumentException("Unable to create path: " + dirFile); - } - return dirFile; - } - public static InetSocketAddress getScmBlockProtocolServerAddress( OzoneConfiguration conf, String localScmServiceId, String nodeId) { String bindHostKey = ConfUtils.addKeySuffixes( diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/choose/algorithms/RoundRobinPipelineChoosePolicy.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/choose/algorithms/RoundRobinPipelineChoosePolicy.java new file mode 100644 index 00000000000..0fb1a1243d7 --- /dev/null +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/choose/algorithms/RoundRobinPipelineChoosePolicy.java @@ -0,0 +1,63 @@ +/** + * 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.hadoop.hdds.scm.pipeline.choose.algorithms; + +import org.apache.hadoop.hdds.scm.PipelineChoosePolicy; +import org.apache.hadoop.hdds.scm.PipelineRequestInformation; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.List; + +/** + * Round-robin choose policy that chooses pipeline in a round-robin fashion. + * Only useful for debugging and testing purposes, at least for now. + */ +public class RoundRobinPipelineChoosePolicy implements PipelineChoosePolicy { + + public static final Logger LOG = LoggerFactory.getLogger(RoundRobinPipelineChoosePolicy.class); + + // Stores the index of the next pipeline to be returned. + private int nextPipelineIndex = 0; + + @Override + public Pipeline choosePipeline(List pipelineList, + PipelineRequestInformation pri) { + return pipelineList.get(choosePipelineIndex(pipelineList, pri)); + } + + /** + * Given a list of pipelines, return the index of the chosen pipeline. + * @param pipelineList List of pipelines + * @param pri PipelineRequestInformation + * @return Index in the list of the chosen pipeline. + */ + @Override + public int choosePipelineIndex(List pipelineList, + PipelineRequestInformation pri) { + final int numPipelines = pipelineList.size(); + int chosenIndex; + synchronized (this) { + nextPipelineIndex = nextPipelineIndex % numPipelines; + chosenIndex = nextPipelineIndex++; + } + LOG.debug("chosenIndex = {}, numPipelines = {}", chosenIndex, numPipelines); + return chosenIndex; + } +} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/Precheck.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/Precheck.java deleted file mode 100644 index 12c6c317542..00000000000 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/Precheck.java +++ /dev/null @@ -1,29 +0,0 @@ -/* - * 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.hadoop.hdds.scm.safemode; - -import org.apache.hadoop.hdds.scm.exceptions.SCMException; - -/** - * Precheck for SCM operations. - * */ -public interface Precheck { - boolean check(T t) throws SCMException; - String type(); -} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModePrecheck.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModePrecheck.java deleted file mode 100644 index 6a0001c673c..00000000000 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModePrecheck.java +++ /dev/null @@ -1,69 +0,0 @@ -/* - * 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.hadoop.hdds.scm.safemode; - -import java.util.concurrent.atomic.AtomicBoolean; -import org.apache.hadoop.hdds.conf.ConfigurationSource; -import org.apache.hadoop.hdds.HddsConfigKeys; -import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmOps; -import org.apache.hadoop.hdds.scm.exceptions.SCMException; -import org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes; - -/** - * Safe mode pre-check for SCM operations. - * */ -public class SafeModePrecheck implements Precheck { - - private AtomicBoolean inSafeMode; - public static final String PRECHECK_TYPE = "SafeModePrecheck"; - - public SafeModePrecheck(ConfigurationSource conf) { - boolean safeModeEnabled = conf.getBoolean( - HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED, - HddsConfigKeys.HDDS_SCM_SAFEMODE_ENABLED_DEFAULT); - if (safeModeEnabled) { - inSafeMode = new AtomicBoolean(true); - } else { - inSafeMode = new AtomicBoolean(false); - } - } - - @Override - public boolean check(ScmOps op) throws SCMException { - if (inSafeMode.get() && SafeModeRestrictedOps - .isRestrictedInSafeMode(op)) { - throw new SCMException("SafeModePrecheck failed for " + op, - ResultCodes.SAFE_MODE_EXCEPTION); - } - return inSafeMode.get(); - } - - @Override - public String type() { - return PRECHECK_TYPE; - } - - public boolean isInSafeMode() { - return inSafeMode.get(); - } - - public void setInSafeMode(boolean inSafeMode) { - this.inSafeMode.set(inSafeMode); - } -} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeRestrictedOps.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeRestrictedOps.java deleted file mode 100644 index b46611f6ee5..00000000000 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeRestrictedOps.java +++ /dev/null @@ -1,41 +0,0 @@ -/** - * 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.hadoop.hdds.scm.safemode; - -import java.util.EnumSet; - -import org.apache.hadoop.hdds.protocol.proto.HddsProtos.ScmOps; - -/** - * Operations restricted in SCM safe mode. - */ -public final class SafeModeRestrictedOps { - private static EnumSet restrictedOps = EnumSet.noneOf(ScmOps.class); - - private SafeModeRestrictedOps() { - } - - static { - restrictedOps.add(ScmOps.allocateBlock); - restrictedOps.add(ScmOps.allocateContainer); - } - - public static boolean isRestrictedInSafeMode(ScmOps opName) { - return restrictedOps.contains(opName); - } -} diff --git a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManagerStarter.java b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManagerStarter.java index 916578796ff..e258c8ee66e 100644 --- a/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManagerStarter.java +++ b/hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/server/StorageContainerManagerStarter.java @@ -21,11 +21,11 @@ */ package org.apache.hadoop.hdds.scm.server; -import org.apache.hadoop.hdds.StringUtils; import org.apache.hadoop.hdds.cli.GenericCli; import org.apache.hadoop.hdds.cli.HddsVersionProvider; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.tracing.TracingUtil; +import org.apache.hadoop.hdds.utils.HddsServerUtil; import org.apache.hadoop.hdds.utils.HddsVersionInfo; import org.apache.hadoop.ozone.common.StorageInfo; import org.apache.hadoop.ozone.util.OzoneNetUtils; @@ -155,7 +155,7 @@ private void commonInit() { String[] originalArgs = getCmd().getParseResult().originalArgs() .toArray(new String[0]); - StringUtils.startupShutdownMessage(HddsVersionInfo.HDDS_VERSION_INFO, + HddsServerUtil.startupShutdownMessage(HddsVersionInfo.HDDS_VERSION_INFO, StorageContainerManager.class, originalArgs, LOG, conf); } diff --git a/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/choose/algorithms/TestRoundRobinPipelineChoosePolicy.java b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/choose/algorithms/TestRoundRobinPipelineChoosePolicy.java new file mode 100644 index 00000000000..2dc7958631f --- /dev/null +++ b/hadoop-hdds/server-scm/src/test/java/org/apache/hadoop/hdds/scm/pipeline/choose/algorithms/TestRoundRobinPipelineChoosePolicy.java @@ -0,0 +1,170 @@ +/** + * 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.hadoop.hdds.scm.pipeline.choose.algorithms; + +import org.apache.hadoop.hdds.protocol.DatanodeDetails; +import org.apache.hadoop.hdds.protocol.MockDatanodeDetails; +import org.apache.hadoop.hdds.scm.PipelineChoosePolicy; +import org.apache.hadoop.hdds.scm.node.NodeManager; +import org.apache.hadoop.hdds.scm.pipeline.MockPipeline; +import org.apache.hadoop.hdds.scm.pipeline.MockRatisPipelineProvider; +import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.mockito.Mockito.mock; + +/** + * Test for the round-robin pipeline choose policy. + */ +public class TestRoundRobinPipelineChoosePolicy { + private static final int NUM_DATANODES = 4; + private static final int NUM_PIPELINES = 4; + private PipelineChoosePolicy policy; + private List allPipelines; + + @BeforeEach + public void setup() throws Exception { + + List datanodes = new ArrayList<>(); + for (int i = 0; i < NUM_DATANODES; i++) { + datanodes.add(MockDatanodeDetails.randomDatanodeDetails()); + } + + NodeManager mockNodeManager = mock(NodeManager.class); + policy = new RoundRobinPipelineChoosePolicy().init(mockNodeManager); + + // 4 pipelines with each pipeline having 3 datanodes + // + // pipeline0 dn1 dn2 dn3 + // pipeline1 dn0 dn2 dn3 + // pipeline2 dn0 dn1 dn3 + // pipeline3 dn0 dn1 dn2 + // + allPipelines = new ArrayList<>(); + for (int i = 0; i < NUM_PIPELINES; i++) { + List dns = new ArrayList<>(); + for (int j = 0; j < datanodes.size(); j++) { + if (i != j) { + dns.add(datanodes.get(j)); + } + } + Pipeline pipeline = MockPipeline.createPipeline(dns); + MockRatisPipelineProvider.markPipelineHealthy(pipeline); + allPipelines.add(pipeline); + } + + } + + private void verifySelectedCountMap(Map selectedCountMap, int[] arrExpectCount) { + for (int i = 0; i < NUM_PIPELINES; i++) { + assertEquals(arrExpectCount[i], selectedCountMap.getOrDefault(allPipelines.get(i), 0)); + } + } + + @Test + public void testChoosePipeline() { + Map selectedCountMap = new HashMap<>(); + + final int numContainers = 100; + for (int i = 0; i < numContainers; i++) { + Pipeline pipeline = policy.choosePipeline(allPipelines, null); + assertNotNull(pipeline); + assertEquals(allPipelines.get(i % NUM_PIPELINES), pipeline); + selectedCountMap.compute(pipeline, (k, v) -> v == null ? 1 : v + 1); + } + + // Each pipeline would be chosen 100 / 4 = 25 times + verifySelectedCountMap(selectedCountMap, new int[] {25, 25, 25, 25}); + } + + @Test + public void testChoosePipelineListVaries() { + Map selectedCountMap; + + // A pipeline list that holds only a subset of the pipelines for this test + List availablePipelines = new ArrayList<>(); + int numAvailablePipeline; + + // Case 1. Only pipeline0 is available + availablePipelines.add(allPipelines.get(0)); + numAvailablePipeline = availablePipelines.size(); + selectedCountMap = new HashMap<>(); + for (int i = 0; i < 10; i++) { + final Pipeline pipeline = policy.choosePipeline(availablePipelines, null); + assertEquals(allPipelines.get(i % numAvailablePipeline), pipeline); + selectedCountMap.compute(pipeline, (k, v) -> v == null ? 1 : v + 1); + } + // pipeline0 is selected 10 times + verifySelectedCountMap(selectedCountMap, new int[] {10, 0, 0, 0}); + + // Case 2. pipeline0 and pipeline1 are available + availablePipelines.add(allPipelines.get(1)); + numAvailablePipeline = availablePipelines.size(); + selectedCountMap = new HashMap<>(); + for (int i = 0; i < 10; i++) { + final Pipeline pipeline = policy.choosePipeline(availablePipelines, null); + assertEquals(availablePipelines.get((i + 1) % numAvailablePipeline), pipeline); + selectedCountMap.compute(pipeline, (k, v) -> v == null ? 1 : v + 1); + } + // pipeline0 and pipeline1 are selected 5 times each + verifySelectedCountMap(selectedCountMap, new int[] {5, 5, 0, 0}); + + // Case 3. pipeline0, pipeline1 and pipeline2 are available + availablePipelines.add(allPipelines.get(2)); + numAvailablePipeline = availablePipelines.size(); + selectedCountMap = new HashMap<>(); + for (int i = 0; i < 10; i++) { + final Pipeline pipeline = policy.choosePipeline(availablePipelines, null); + assertEquals(availablePipelines.get((i + 1) % numAvailablePipeline), pipeline); + selectedCountMap.compute(pipeline, (k, v) -> v == null ? 1 : v + 1); + } + // pipeline0-2 are selected 3-4 times each + verifySelectedCountMap(selectedCountMap, new int[] {3, 4, 3, 0}); + + // Case 4. All 4 pipelines are available + availablePipelines.add(allPipelines.get(3)); + numAvailablePipeline = availablePipelines.size(); + selectedCountMap = new HashMap<>(); + for (int i = 0; i < 10; i++) { + final Pipeline pipeline = policy.choosePipeline(availablePipelines, null); + assertEquals(availablePipelines.get((i + 2) % numAvailablePipeline), pipeline); + selectedCountMap.compute(pipeline, (k, v) -> v == null ? 1 : v + 1); + } + // pipeline0-3 are selected 2-3 times each + verifySelectedCountMap(selectedCountMap, new int[] {2, 2, 3, 3}); + + // Case 5. Remove pipeline0 from the available pipeline list + availablePipelines.remove(allPipelines.get(0)); + numAvailablePipeline = availablePipelines.size(); + selectedCountMap = new HashMap<>(); + for (int i = 0; i < 10; i++) { + final Pipeline pipeline = policy.choosePipeline(availablePipelines, null); + assertEquals(availablePipelines.get((i + 1) % numAvailablePipeline), pipeline); + selectedCountMap.compute(pipeline, (k, v) -> v == null ? 1 : v + 1); + } + // pipeline1-3 are selected 3-4 times each + verifySelectedCountMap(selectedCountMap, new int[] {0, 3, 4, 3}); + } +} diff --git a/hadoop-ozone/client/pom.xml b/hadoop-ozone/client/pom.xml index a5a43643618..2a052584ada 100644 --- a/hadoop-ozone/client/pom.xml +++ b/hadoop-ozone/client/pom.xml @@ -40,6 +40,11 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> ozone-common + + com.github.stephenc.jcip + jcip-annotations + + org.apache.ozone diff --git a/hadoop-ozone/common/pom.xml b/hadoop-ozone/common/pom.xml index a925df56a9f..bc981f0bb34 100644 --- a/hadoop-ozone/common/pom.xml +++ b/hadoop-ozone/common/pom.xml @@ -70,6 +70,11 @@ https://maven.apache.org/xsd/maven-4.0.0.xsd"> hadoop-hdfs-client + + com.github.stephenc.jcip + jcip-annotations + + io.grpc diff --git a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java index 64aa9b1d240..6735c0a4f0a 100644 --- a/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java +++ b/hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java @@ -855,6 +855,12 @@ public static List getAllOMHAAddresses(OzoneConfiguration conf, try { OMNodeDetails omNodeDetails = OMNodeDetails.getOMNodeDetailsFromConf( conf, omServiceId, nodeId); + if (omNodeDetails == null) { + LOG.error( + "There is no OM configuration for node ID {} in ozone-site.xml.", + nodeId); + continue; + } if (decommissionedNodeIds.contains(omNodeDetails.getNodeId())) { omNodeDetails.setDecommissioningState(); } diff --git a/hadoop-ozone/csi/pom.xml b/hadoop-ozone/csi/pom.xml index d40a995ab92..4118af5b069 100644 --- a/hadoop-ozone/csi/pom.xml +++ b/hadoop-ozone/csi/pom.xml @@ -66,6 +66,10 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> + + org.apache.ozone + hdds-server-framework + com.google.code.findbugs jsr305 diff --git a/hadoop-ozone/csi/src/main/java/org/apache/hadoop/ozone/csi/CsiServer.java b/hadoop-ozone/csi/src/main/java/org/apache/hadoop/ozone/csi/CsiServer.java index 3b0c8f3c20f..dbafccf4fd2 100644 --- a/hadoop-ozone/csi/src/main/java/org/apache/hadoop/ozone/csi/CsiServer.java +++ b/hadoop-ozone/csi/src/main/java/org/apache/hadoop/ozone/csi/CsiServer.java @@ -19,13 +19,13 @@ import java.util.concurrent.Callable; -import org.apache.hadoop.hdds.StringUtils; import org.apache.hadoop.hdds.cli.GenericCli; import org.apache.hadoop.hdds.cli.HddsVersionProvider; import org.apache.hadoop.hdds.conf.Config; import org.apache.hadoop.hdds.conf.ConfigGroup; import org.apache.hadoop.hdds.conf.ConfigTag; import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.utils.HddsServerUtil; import org.apache.hadoop.ozone.client.OzoneClient; import org.apache.hadoop.ozone.client.OzoneClientFactory; import org.apache.hadoop.ozone.util.OzoneVersionInfo; @@ -55,7 +55,7 @@ public Void call() throws Exception { String[] originalArgs = getCmd().getParseResult().originalArgs() .toArray(new String[0]); OzoneConfiguration ozoneConfiguration = createOzoneConfiguration(); - StringUtils.startupShutdownMessage(OzoneVersionInfo.OZONE_VERSION_INFO, + HddsServerUtil.startupShutdownMessage(OzoneVersionInfo.OZONE_VERSION_INFO, CsiServer.class, originalArgs, LOG, ozoneConfiguration); CsiConfig csiConfig = ozoneConfiguration.getObject(CsiConfig.class); diff --git a/hadoop-ozone/dev-support/checks/junit.sh b/hadoop-ozone/dev-support/checks/junit.sh index d4936834b26..bb7088f0cd5 100755 --- a/hadoop-ozone/dev-support/checks/junit.sh +++ b/hadoop-ozone/dev-support/checks/junit.sh @@ -66,7 +66,7 @@ for i in $(seq 1 ${ITERATIONS}); do mkdir -p "${REPORT_DIR}" fi - mvn ${MAVEN_OPTIONS} -Dmaven-surefire-plugin.argLineAccessArgs="${OZONE_MODULE_ACCESS_ARGS}" "$@" test \ + mvn ${MAVEN_OPTIONS} -Dmaven-surefire-plugin.argLineAccessArgs="${OZONE_MODULE_ACCESS_ARGS}" "$@" verify \ | tee "${REPORT_DIR}/output.log" irc=$? diff --git a/hadoop-ozone/dist/src/main/compose/common/grafana/dashboards/Ozone - CreateKey Metrics.json b/hadoop-ozone/dist/src/main/compose/common/grafana/dashboards/Ozone - CreateKey Metrics.json new file mode 100644 index 00000000000..78b027afd2e --- /dev/null +++ b/hadoop-ozone/dist/src/main/compose/common/grafana/dashboards/Ozone - CreateKey Metrics.json @@ -0,0 +1,3730 @@ +{ + "annotations": { + "list": [ + { + "builtIn": 1, + "datasource": "-- Grafana --", + "enable": true, + "hide": true, + "iconColor": "rgba(0, 211, 255, 1)", + "name": "Annotations & Alerts", + "type": "dashboard" + } + ] + }, + "editable": true, + "fiscalYearStartMonth": 0, + "graphTooltip": 0, + "id": null, + "links": [], + "liveNow": false, + "panels": [ + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 0 + }, + "id": 52, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "time (ns)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "ns" + }, + "overrides": [ + { + "__systemRef": "hideSeriesFrom", + "matcher": { + "id": "byNames", + "options": { + "mode": "exclude", + "names": [ + "om_performance_metrics_check_access_latency_ns_avg_time" + ], + "prefix": "All except:", + "readOnly": true + } + }, + "properties": [ + { + "id": "custom.hideFrom", + "value": { + "legend": false, + "tooltip": false, + "viz": true + } + } + ] + } + ] + }, + "gridPos": { + "h": 7, + "w": 8, + "x": 0, + "y": 1 + }, + "id": 11, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "maxHeight": 600, + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "om_performance_metrics_check_access_latency_ns_avg_time", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Check access latency (time in ns)", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisBorderShow": false, + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "no. of ops", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green", + "value": null + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 8, + "x": 8, + "y": 1 + }, + "id": 12, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "maxHeight": 600, + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "om_performance_metrics_check_access_latency_ns_num_ops", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Check access latency (ops per sec)", + "type": "timeseries" + } + ], + "title": "OM Check Access Latency Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 1 + }, + "id": 51, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "no. of keys", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 8, + "x": 0, + "y": 2 + }, + "id": 13, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rocksdb_om_db_estimate_num_keys", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rocksdb_om_db_filetable_estimate_num_keys", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + } + ], + "title": "Estimate no. of keys metrics", + "type": "timeseries" + } + ], + "title": "OM Rocksdb Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 2 + }, + "id": 50, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "time (ns)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "ns" + }, + "overrides": [ + { + "__systemRef": "hideSeriesFrom", + "matcher": { + "id": "byNames", + "options": { + "mode": "exclude", + "names": [ + "ozone_manager_double_buffer_metrics_flush_time_avg_time" + ], + "prefix": "All except:", + "readOnly": true + } + }, + "properties": [ + { + "id": "custom.hideFrom", + "value": { + "legend": false, + "tooltip": false, + "viz": true + } + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 0, + "y": 3 + }, + "id": 15, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "ozone_manager_double_buffer_metrics_flush_time_avg_time", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "expr": "", + "hide": false, + "instant": false, + "range": true, + "refId": "B" + } + ], + "title": "Double buffer metrics (time in ns)", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "no. of ops", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 8, + "y": 3 + }, + "id": 14, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "ozone_manager_double_buffer_metrics_flush_time_num_ops", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "ozone_manager_double_buffer_metrics_queue_size_num_ops", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "ozone_manager_double_buffer_metrics_total_num_of_flush_operations", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "C", + "useBackend": false + } + ], + "title": "Double buffer metrics (no. of ops)", + "type": "timeseries" + } + ], + "title": "OM Double Buffer Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 3 + }, + "id": 8, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "description": "", + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [ + { + "__systemRef": "hideSeriesFrom", + "matcher": { + "id": "byNames", + "options": { + "mode": "exclude", + "names": [ + "table_cache_metrics_size, bucketTable" + ], + "prefix": "All except:", + "readOnly": true + } + }, + "properties": [ + { + "id": "custom.hideFrom", + "value": { + "legend": false, + "tooltip": false, + "viz": true + } + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 0, + "y": 4 + }, + "id": 16, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "table_cache_metrics_size", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}, {{tablename}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Table cache metrics (size)", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 8, + "y": 4 + }, + "id": 17, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "table_cache_metrics_miss_count", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}, {{tablename}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "table_cache_metrics_hit_count", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}, {{tablename}}", + "range": true, + "refId": "B", + "useBackend": false + } + ], + "title": "Table cache metrics (hit/miss count)", + "type": "timeseries" + } + ], + "title": "OM Table Cache Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 4 + }, + "id": 64, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "no. of ops", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [ + { + "__systemRef": "hideSeriesFrom", + "matcher": { + "id": "byNames", + "options": { + "mode": "exclude", + "names": [ + "om_metrics_num_get_service_lists" + ], + "prefix": "All except:", + "readOnly": true + } + }, + "properties": [ + { + "id": "custom.hideFrom", + "value": { + "legend": false, + "tooltip": false, + "viz": true + } + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 0, + "y": 13 + }, + "id": 10, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "om_metrics_num_get_service_lists", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "om_metrics_num_key_ops", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + } + ], + "title": "OM (no. of ops) Metrics", + "type": "timeseries" + } + ], + "title": "OM Service Lists Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 5 + }, + "id": 63, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "no. of keys", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [ + { + "__systemRef": "hideSeriesFrom", + "matcher": { + "id": "byNames", + "options": { + "mode": "exclude", + "names": [ + "om_metrics_num_keys, .*" + ], + "prefix": "All except:", + "readOnly": true + } + }, + "properties": [ + { + "id": "custom.hideFrom", + "value": { + "legend": false, + "tooltip": false, + "viz": true + } + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 0, + "y": 1 + }, + "id": 49, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "om_metrics_num_keys", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Num Key Metrics", + "type": "timeseries" + } + ], + "title": "OM No. of Key Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 6 + }, + "id": 53, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "no. of ops", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 8, + "x": 0, + "y": 20 + }, + "id": 9, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "om_lock_metrics_read_lock_held_time_num_ops", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "om_lock_metrics_write_lock_waiting_time_num_ops", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "om_lock_metrics_read_lock_waiting_time_num_ops", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "C", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "om_lock_metrics_write_lock_held_time_num_ops", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "D", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "om_lock_metrics_read_lock_held_time_i_num_ops", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "E", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "om_lock_metrics_read_lock_waiting_time_i_num_ops", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "F", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "om_lock_metrics_write_lock_held_time_i_num_ops", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "G", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "om_lock_metrics_write_lock_waiting_time_i_num_ops", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "H", + "useBackend": false + } + ], + "title": "Lock (read/write) metrics (no. of ops)", + "type": "timeseries" + } + ], + "title": "OM Lock Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 7 + }, + "id": 54, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 0, + "y": 21 + }, + "id": 48, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_log_worker_flushCount{instance=~\".*:9875\"}", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_log_worker_cacheHitCount{instance=~\".*:9875\"}", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "B", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_log_worker_appendEntryCount{instance=~\".*:9875\"}", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "C", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_server_retryCacheEntryCount{instance=~\".*:9875\"}", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "D", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_server_retryCacheMissCount{instance=~\".*:9875\"}", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "E", + "useBackend": false + } + ], + "title": "Ratis log metrics (count)", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [ + { + "__systemRef": "hideSeriesFrom", + "matcher": { + "id": "byNames", + "options": { + "mode": "exclude", + "names": [ + "ratis_log_worker_appendEntryLatency, .*:9875, group-0A2AA1204044" + ], + "prefix": "All except:", + "readOnly": true + } + }, + "properties": [ + { + "id": "custom.hideFrom", + "value": { + "legend": false, + "tooltip": false, + "viz": true + } + } + ] + } + ] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 8, + "y": 21 + }, + "id": 24, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_log_worker_appendEntryLatency{instance=~\".*:9875\"}", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_server_follower_entry_latency{instance=~\".*:9875\"}", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "B", + "useBackend": false + } + ], + "title": "Ratis log metrics (latency)", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "Time (ns)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "ns" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 16, + "y": 21 + }, + "id": 45, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "ratis_log_worker_flushTime{instance=~\".*:9875\"}", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_log_worker_writelogExecutionTime{instance=~\".*:9875\"}", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "B", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_log_worker_enqueuedTime{instance=~\".*:9875\"}", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "C", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_log_worker_syncTime{instance=~\".*:9875\"}", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "D", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_log_worker_queueingDelay{instance=~\".*:9875\"}", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "E", + "useBackend": false + } + ], + "title": "Ratis log metrics (time)", + "type": "timeseries" + } + ], + "title": "OM Ratis Log Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 8 + }, + "id": 56, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 0, + "y": 9 + }, + "id": 29, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "ugi_metrics_get_groups_avg_time{servername=\"ozoneManager\"}", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Ugi Metrics (avg. time)", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "no. of ops", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 8, + "y": 9 + }, + "id": 30, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "ugi_metrics_get_groups_num_ops{servername=\"ozoneManager\"}", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Ugi Metrics (no. of ops)", + "type": "timeseries" + } + ], + "title": "OM Ugi Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 9 + }, + "id": 7, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "no. of ops", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 0, + "y": 8 + }, + "id": 26, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_head_key_success_latency_ns_num_ops", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_init_multipart_upload_success_latency_ns_num_ops", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + } + ], + "title": "S3 Gateway multipart upload metrics (no. of ops)", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "Time (ns)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "ns" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 8, + "y": 8 + }, + "id": 25, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_head_key_success_latency_ns_avg_time", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_init_multipart_upload_success_latency_ns_avg_time", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + } + ], + "title": "S3 Gateway multipart upload metrics (avg time)", + "type": "timeseries" + } + ], + "title": "S3 Gateway Multipart Upload Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 10 + }, + "id": 57, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "time (ns)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 8, + "x": 0, + "y": 9 + }, + "id": 34, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_create_key_success_latency_ns_avg_time", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_head_key_success_latency_ns_avg_time", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + } + ], + "title": " Combined Latency (avg. time) Metrics ", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "no. of ops", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 8, + "x": 8, + "y": 9 + }, + "id": 32, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_head_key_success_latency_ns_num_ops", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_put_key_metadata_latency_ns_num_ops", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_create_key_success_latency_ns_num_ops", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "C", + "useBackend": false + } + ], + "title": "Combined Latency (no. of ops) Metrics", + "type": "timeseries" + } + ], + "title": "S3 Gateway Combined Latency Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 12 + }, + "id": 59, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 8, + "x": 0, + "y": 11 + }, + "id": 37, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_put_key_success_length", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Put key success length", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 8, + "x": 8, + "y": 11 + }, + "id": 36, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_head_key_success", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Head key success metrics", + "type": "timeseries" + } + ], + "title": "S3 Gateway Key Success Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 13 + }, + "id": 60, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 0, + "y": 12 + }, + "id": 33, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "container_client_metrics_total_write_chunk_calls", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Container client metrics", + "type": "timeseries" + } + ], + "title": "S3 Gateway Container Client Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 14 + }, + "id": 40, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "no. of ops", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 7, + "w": 8, + "x": 0, + "y": 13 + }, + "id": 44, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "storage_container_metrics_num_ops", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "storage_container_metrics_num_put_block", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "storage_container_metrics_num_write_chunk", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "C", + "useBackend": false + } + ], + "title": "SCM (no. of ops) Metrics", + "type": "timeseries" + } + ], + "title": "SCM Metrics (no. of ops)", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 15 + }, + "id": 61, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "time (ns)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "ns" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 0, + "y": 14 + }, + "id": 41, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "storage_container_metrics_latency_put_block_avg_time", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}},{{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "storage_container_metrics_latency_write_chunk_avg_time", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + } + ], + "title": "SCM Combined Latency Metrics (avg. time)", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "no. of ops", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 12, + "x": 8, + "y": 14 + }, + "id": 42, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "storage_container_metrics_latency_write_chunk_num_ops", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "SCM Combined Latency Metrics (no. of ops)", + "type": "timeseries" + } + ], + "title": "SCM Combined Latency Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 16 + }, + "id": 62, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "Time (ns)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "ns" + }, + "overrides": [] + }, + "gridPos": { + "h": 9, + "w": 8, + "x": 0, + "y": 15 + }, + "id": 46, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_log_worker_flushTime{instance=~\".*:9883\"}", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_log_worker_writelogExecutionTime{instance=~\".*:9883\"}", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "B", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_log_worker_enqueuedTime{instance=~\".*:9883\"}", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "C", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_log_worker_syncTime{instance=~\".*:9883\"}", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "D", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_log_worker_queueingDelay{instance=~\".*:9883\"}", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "E", + "useBackend": false + } + ], + "title": "Ratis log metrics (time)", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [ + { + "__systemRef": "hideSeriesFrom", + "matcher": { + "id": "byNames", + "options": { + "mode": "exclude", + "names": [ + "ratis_log_worker_appendEntryLatency, .*:9883, group-1E3FC9CE5915" + ], + "prefix": "All except:", + "readOnly": true + } + }, + "properties": [ + { + "id": "custom.hideFrom", + "value": { + "legend": false, + "tooltip": false, + "viz": true + } + } + ] + } + ] + }, + "gridPos": { + "h": 9, + "w": 8, + "x": 8, + "y": 15 + }, + "id": 47, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "ratis_log_worker_appendEntryLatency{instance=~\".*:9883\"}", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "ratis_server_follower_entry_latency{instance=~\".*:9883\"}", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "B", + "useBackend": false + } + ], + "title": "Ratis log metrics (latency)", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 9, + "w": 8, + "x": 16, + "y": 15 + }, + "id": 22, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_log_worker_flushCount{instance=~\".*:9883\"}", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_log_worker_cacheHitCount{instance=~\".*:9883\"}", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "B", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_log_worker_appendEntryCount{instance=~\".*:9883\"}", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "C", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_server_retryCacheEntryCount{instance=~\".*:9883\"}", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "D", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ratis_server_retryCacheMissCount{instance=~\".*:9883\"}", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{instance}}, {{group}}", + "range": true, + "refId": "E", + "useBackend": false + } + ], + "title": "Ratis log metrics (count)", + "type": "timeseries" + } + ], + "title": "Datanode Ratis Log Metrics", + "type": "row" + } + ], + "refresh": "", + "schemaVersion": 39, + "tags": [], + "templating": { + "list": [] + }, + "time": { + "from": "now-12h", + "to": "now" + }, + "timeRangeUpdatedDuringEditOrView": false, + "timepicker": {}, + "timezone": "", + "title": "Create key Dashboard", + "version": 36, + "weekStart": "" +} \ No newline at end of file diff --git a/hadoop-ozone/dist/src/main/compose/common/grafana/dashboards/Ozone - ReadKey Metrics.json b/hadoop-ozone/dist/src/main/compose/common/grafana/dashboards/Ozone - ReadKey Metrics.json new file mode 100644 index 00000000000..827e2f04e10 --- /dev/null +++ b/hadoop-ozone/dist/src/main/compose/common/grafana/dashboards/Ozone - ReadKey Metrics.json @@ -0,0 +1,3221 @@ +{ + "annotations": { + "list": [ + { + "builtIn": 1, + "datasource": "-- Grafana --", + "enable": true, + "hide": true, + "iconColor": "rgba(0, 211, 255, 1)", + "name": "Annotations & Alerts", + "type": "dashboard" + } + ] + }, + "editable": true, + "fiscalYearStartMonth": 0, + "graphTooltip": 0, + "id": null, + "links": [], + "liveNow": false, + "panels": [ + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 0 + }, + "id": 19, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "no. of keys", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 9, + "w": 8, + "x": 0, + "y": 1 + }, + "id": 8, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rocksdb_om_db_estimate_num_keys", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rocksdb_om_db_filetable_estimate_num_keys", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rocksdb_om_db_keytable_estimate_num_keys", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "C", + "useBackend": false + } + ], + "title": "Rocksdb metrics (no. of keys)", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "cache used", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 9, + "w": 8, + "x": 8, + "y": 1 + }, + "id": 7, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rocksdb_om_db_block_cache_usage", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rocksdb_om_db_filetable_block_cache_usage", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + } + ], + "title": "Rocksdb block cache usage metrics", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "no. of files", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 9, + "w": 8, + "x": 16, + "y": 1 + }, + "id": 13, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rocksdb_om_db_filetable_num_files_at_level0", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rocksdb_om_db_keytable_num_files_at_level0", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rocksdb_om_db_s3secrettable_num_files_at_level0", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "C", + "useBackend": false + } + ], + "title": "Rocksdb level0 metrics (num files)", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "no. of keys", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 9, + "w": 8, + "x": 0, + "y": 10 + }, + "id": 6, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "rdb_metrics_num_db_key_get_if_exist_checks{instance=~\".*:9875\"}", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "rdb_metrics_num_db_key_get_if_exist_gets{instance=~\".*:9875\"}", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + } + ], + "title": "Rocksdb no. of db key metrics", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 9, + "w": 8, + "x": 8, + "y": 10 + }, + "id": 10, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rocksdb_om_db_filetable_cur_size_active_mem_table", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rocksdb_om_db_filetable_cur_size_all_mem_tables", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rocksdb_om_db_filetable_size_all_mem_tables", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "C", + "useBackend": false + } + ], + "title": "Rocksdb mem table metrics (size)", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 9, + "w": 8, + "x": 16, + "y": 10 + }, + "id": 11, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rocksdb_om_db_estimate_table_readers_mem", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rocksdb_om_db_filetable_estimate_table_readers_mem", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rocksdb_om_db_keytable_estimate_table_readers_mem", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "C", + "useBackend": false + } + ], + "title": "Rocksdb om db table readers mem metrics", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 9, + "w": 8, + "x": 0, + "y": 19 + }, + "id": 12, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rocksdb_om_db_filetable_live_sst_files_size", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rocksdb_om_db_keytable_live_sst_files_size", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "rocksdb_om_db_s3secrettable_live_sst_files_size", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "C", + "useBackend": false + } + ], + "title": "Rocksdb live sst file size metrics", + "type": "timeseries" + } + ], + "title": "OM Rocksdb Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 1 + }, + "id": 20, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "no. of ops", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 0, + "y": 2 + }, + "id": 16, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "ugi_metrics_get_groups_num_ops{servername=\"ozoneManager\"}", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Ugi Metrics (no. of ops)", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "time (ns)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "ns" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 8, + "y": 2 + }, + "id": 15, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "ugi_metrics_get_groups_avg_time{servername=\"ozoneManager\"}", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Ugi Metrics (avg. time)", + "type": "timeseries" + } + ], + "title": "OM Ugi Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 2 + }, + "id": 2, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "no of keys", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 0, + "y": 163 + }, + "id": 4, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "om_metrics_num_keys", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "OM num key metrics", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "no. of ops", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 8, + "y": 163 + }, + "id": 5, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "om_metrics_num_key_ops", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "OM num key ops metrics", + "type": "timeseries" + } + ], + "title": "OM Num Key Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 3 + }, + "id": 21, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 0, + "y": 164 + }, + "id": 1, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "om_metrics_num_get_service_lists", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Get service lists metrics", + "type": "timeseries" + } + ], + "title": "OM Service Lists Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 4 + }, + "id": 22, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "time (ns)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "ns" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 0, + "y": 5 + }, + "id": 3, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "om_performance_metrics_get_key_info_read_key_info_latency_ns_avg_time", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Read key info (avg time) metrics", + "type": "timeseries" + } + ], + "title": "OM Read Key Info Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 5 + }, + "id": 23, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 0, + "y": 174 + }, + "id": 14, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "table_cache_metrics_hit_count{instance=~\".*:9875\"}", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}, {{tablename}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "table_cache_metrics_miss_count{instance=~\".*:9875\"}", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}, {{tablename}}", + "range": true, + "refId": "B", + "useBackend": false + } + ], + "title": "Table cache metrics (count)", + "type": "timeseries" + } + ], + "title": "OM Table Cache Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 6 + }, + "id": 9, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "time (ns)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "ns" + }, + "overrides": [] + }, + "gridPos": { + "h": 9, + "w": 8, + "x": 0, + "y": 47 + }, + "id": 17, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_get_key_metadata_latency_ns_avg_time", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_get_key_success_latency_ns_avg_time", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_head_key_failure_latency_ns_avg_time", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "C", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_head_key_success_latency_ns_avg_time", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "D", + "useBackend": false + } + ], + "title": "Combined key latency metrics (avg. time)", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "no. of ops", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 9, + "w": 8, + "x": 8, + "y": 47 + }, + "id": 18, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_get_key_metadata_latency_ns_num_ops", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_get_key_success_latency_ns_num_ops", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_head_key_failure_latency_ns_num_ops", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "C", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_head_key_success_latency_ns_num_ops", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "D", + "useBackend": false + } + ], + "title": "Combined key latency metrics (no. of ops)", + "type": "timeseries" + } + ], + "title": "S3 Gateway Combined Latency Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 7 + }, + "id": 24, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 9, + "w": 8, + "x": 0, + "y": 146 + }, + "id": 26, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_get_key_success", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_head_key_success", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + } + ], + "title": "Key success metrics", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 9, + "w": 8, + "x": 8, + "y": 146 + }, + "id": 27, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_get_key_success_length", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Key success length metrics", + "type": "timeseries" + } + ], + "title": "S3 Gateway Key Success Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 8 + }, + "id": 25, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 9, + "w": 8, + "x": 0, + "y": 138 + }, + "id": 30, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "s3_gateway_metrics_head_key_failure", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "head key failure metric", + "type": "timeseries" + } + ], + "title": "S3 Gateway Key Failure Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 10 + }, + "id": 29, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 0, + "y": 43 + }, + "id": 36, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "storage_container_metrics_bytes_get_block", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Bytes block metrics", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 8, + "y": 43 + }, + "id": 37, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "storage_container_metrics_bytes_read_chunk", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Bytes chunk metrics", + "type": "timeseries" + } + ], + "title": "SCM Bytes Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 11 + }, + "id": 38, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "time (ns)", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + }, + "unit": "ns" + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 0, + "y": 44 + }, + "id": 39, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "storage_container_metrics_latency_get_block_avg_time", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + }, + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "storage_container_metrics_latency_read_chunk_avg_time", + "fullMetaSearch": false, + "hide": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "B", + "useBackend": false + } + ], + "title": "Combined latency (avg. time) metrics", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "no. of ops", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 8, + "y": 44 + }, + "id": 40, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "storage_container_metrics_latency_get_block_num_ops", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "Combined Latency (no. of ops)", + "type": "timeseries" + } + ], + "title": "SCM Combined Latency Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 12 + }, + "id": 42, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 0, + "y": 21 + }, + "id": 41, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "storage_container_metrics_num_read_chunk", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "no. of read chunk metric", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 8, + "y": 21 + }, + "id": 43, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "storage_container_metrics_num_get_block", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "no. of block metrics", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "no. of ops", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 16, + "y": 21 + }, + "id": 44, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "builder", + "expr": "storage_container_metrics_num_ops", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "no. of ops metrics", + "type": "timeseries" + } + ], + "title": "SCM No. Of Ops Metrics", + "type": "row" + }, + { + "collapsed": true, + "gridPos": { + "h": 1, + "w": 24, + "x": 0, + "y": 13 + }, + "id": 45, + "panels": [ + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 0, + "y": 14 + }, + "id": 46, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ugi_metrics_login_success_avg_time{instance=~\".*:9877\"}", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "success avg time metric", + "type": "timeseries" + }, + { + "datasource": { + "type": "prometheus" + }, + "fieldConfig": { + "defaults": { + "color": { + "mode": "palette-classic" + }, + "custom": { + "axisCenteredZero": false, + "axisColorMode": "text", + "axisLabel": "", + "axisPlacement": "auto", + "barAlignment": 0, + "drawStyle": "line", + "fillOpacity": 0, + "gradientMode": "none", + "hideFrom": { + "legend": false, + "tooltip": false, + "viz": false + }, + "insertNulls": false, + "lineInterpolation": "linear", + "lineWidth": 1, + "pointSize": 5, + "scaleDistribution": { + "type": "linear" + }, + "showPoints": "auto", + "spanNulls": false, + "stacking": { + "group": "A", + "mode": "none" + }, + "thresholdsStyle": { + "mode": "off" + } + }, + "mappings": [], + "thresholds": { + "mode": "absolute", + "steps": [ + { + "color": "green" + }, + { + "color": "red", + "value": 80 + } + ] + } + }, + "overrides": [] + }, + "gridPos": { + "h": 8, + "w": 8, + "x": 8, + "y": 14 + }, + "id": 47, + "options": { + "legend": { + "calcs": [], + "displayMode": "list", + "placement": "bottom", + "showLegend": true + }, + "tooltip": { + "mode": "single", + "sort": "none" + } + }, + "targets": [ + { + "datasource": { + "type": "prometheus" + }, + "disableTextWrap": false, + "editorMode": "code", + "expr": "ugi_metrics_login_success_num_ops{instance=~\".*:9877\"}", + "fullMetaSearch": false, + "includeNullMetadata": true, + "instant": false, + "legendFormat": "{{__name__}}, {{hostname}}", + "range": true, + "refId": "A", + "useBackend": false + } + ], + "title": "success no. of ops metric", + "type": "timeseries" + } + ], + "title": "SCM Ugi Metrics", + "type": "row" + } + ], + "refresh": "", + "schemaVersion": 38, + "style": "dark", + "tags": [], + "templating": { + "list": [] + }, + "time": { + "from": "now-6h", + "to": "now" + }, + "timepicker": {}, + "timezone": "", + "title": "Read Key Dashboard", + "version": 21, + "weekStart": "" +} \ No newline at end of file diff --git a/hadoop-ozone/dist/src/main/compose/ozonescripts/test.sh b/hadoop-ozone/dist/src/main/compose/ozonescripts/disabled-test.sh similarity index 100% rename from hadoop-ozone/dist/src/main/compose/ozonescripts/test.sh rename to hadoop-ozone/dist/src/main/compose/ozonescripts/disabled-test.sh diff --git a/hadoop-ozone/dist/src/main/compose/ozonesecure-ha/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/ozonesecure-ha/docker-compose.yaml index af2b9c4ea6a..1ba764f2578 100644 --- a/hadoop-ozone/dist/src/main/compose/ozonesecure-ha/docker-compose.yaml +++ b/hadoop-ozone/dist/src/main/compose/ozonesecure-ha/docker-compose.yaml @@ -184,7 +184,7 @@ services: ozone_net: ipv4_address: 172.25.0.113 httpfs: - image: apache/ozone-runner:${OZONE_RUNNER_VERSION} + image: ${OZONE_RUNNER_IMAGE}:${OZONE_RUNNER_VERSION} hostname: httpfs volumes: - ../..:/opt/hadoop diff --git a/hadoop-ozone/dist/src/main/compose/ozonesecure/docker-compose.yaml b/hadoop-ozone/dist/src/main/compose/ozonesecure/docker-compose.yaml index bba79dea855..aabc0aafae2 100644 --- a/hadoop-ozone/dist/src/main/compose/ozonesecure/docker-compose.yaml +++ b/hadoop-ozone/dist/src/main/compose/ozonesecure/docker-compose.yaml @@ -66,7 +66,7 @@ services: - docker-config command: ["/opt/hadoop/bin/ozone","om"] httpfs: - image: apache/ozone-runner:${OZONE_RUNNER_VERSION} + image: ${OZONE_RUNNER_IMAGE}:${OZONE_RUNNER_VERSION} hostname: httpfs volumes: - ../..:/opt/hadoop diff --git a/hadoop-ozone/dist/src/main/compose/testlib.sh b/hadoop-ozone/dist/src/main/compose/testlib.sh index d4ad22e58ec..ff676dc2a56 100755 --- a/hadoop-ozone/dist/src/main/compose/testlib.sh +++ b/hadoop-ozone/dist/src/main/compose/testlib.sh @@ -212,7 +212,7 @@ execute_robot_test(){ "$SMOKETEST_DIR_INSIDE/$TEST" local -i rc=$? - FULL_CONTAINER_NAME=$(docker-compose ps | grep "[-_]${CONTAINER}[-_]" | head -n 1 | awk '{print $1}') + FULL_CONTAINER_NAME=$(docker-compose ps -a | grep "[-_]${CONTAINER}[-_]" | head -n 1 | awk '{print $1}') docker cp "$FULL_CONTAINER_NAME:$OUTPUT_PATH" "$RESULT_DIR/" if [[ ${rc} -gt 0 ]] && [[ ${rc} -le 250 ]]; then @@ -251,7 +251,7 @@ create_stack_dumps() { ## @description Copy any 'out' files for daemon processes to the result dir copy_daemon_logs() { local c f - for c in $(docker-compose ps | grep "^${COMPOSE_ENV_NAME}[_-]" | awk '{print $1}'); do + for c in $(docker-compose ps -a | grep "^${COMPOSE_ENV_NAME}[_-]" | awk '{print $1}'); do for f in $(docker exec "${c}" ls -1 /var/log/hadoop 2> /dev/null | grep -F -e '.out' -e audit); do docker cp "${c}:/var/log/hadoop/${f}" "$RESULT_DIR/" done @@ -305,7 +305,7 @@ get_output_name() { save_container_logs() { local output_name=$(get_output_name) local c - for c in $(docker-compose ps "$@" | cut -f1 -d' ' | tail -n +3); do + for c in $(docker-compose ps -a "$@" | cut -f1 -d' ' | tail -n +3); do docker logs "${c}" >> "$RESULT_DIR/docker-${output_name}${c}.log" 2>&1 done } diff --git a/hadoop-ozone/dist/src/main/license/bin/LICENSE.txt b/hadoop-ozone/dist/src/main/license/bin/LICENSE.txt index d118c92e29c..2ce8730586c 100644 --- a/hadoop-ozone/dist/src/main/license/bin/LICENSE.txt +++ b/hadoop-ozone/dist/src/main/license/bin/LICENSE.txt @@ -218,6 +218,7 @@ EDL 1.0 EPL 2.0 ===================== + com.github.jnr:jnr-posix jakarta.annotation:jakarta.annotation-api jakarta.ws.rs:jakarta.ws.rs-api org.aspectj:aspectjrt @@ -280,6 +281,10 @@ Apache License 2.0 com.fasterxml.jackson.datatype:jackson-datatype-jsr310 com.fasterxml.jackson.module:jackson-module-jaxb-annotations com.fasterxml.woodstox:woodstox-core + com.github.jnr:jnr-a64asm + com.github.jnr:jnr-constants + com.github.jnr:jnr-ffi + com.github.jnr:jffi com.github.stephenc.jcip:jcip-annotations com.google.android:annotations com.google.api.grpc:proto-google-common-protos @@ -446,6 +451,7 @@ MIT ===================== com.bettercloud:vault-java-driver + com.github.jnr:jnr-x86asm com.kstruct:gethostname4j org.bouncycastle:bcpkix-jdk18on org.bouncycastle:bcprov-jdk18on @@ -472,6 +478,11 @@ BSD 3-Clause com.google.re2j:re2j com.jcraft:jsch com.thoughtworks.paranamer:paranamer + org.ow2.asm:asm + org.ow2.asm:asm-analysis + org.ow2.asm:asm-commons + org.ow2.asm:asm-tree + org.ow2.asm:asm-util BSD 2-Clause diff --git a/hadoop-ozone/dist/src/main/license/jar-report.txt b/hadoop-ozone/dist/src/main/license/jar-report.txt index 3a5f5066df9..58c080bece4 100644 --- a/hadoop-ozone/dist/src/main/license/jar-report.txt +++ b/hadoop-ozone/dist/src/main/license/jar-report.txt @@ -3,6 +3,11 @@ share/ozone/lib/annotations.jar share/ozone/lib/annotations.jar share/ozone/lib/aopalliance.jar share/ozone/lib/aopalliance-repackaged.jar +share/ozone/lib/asm-analysis.jar +share/ozone/lib/asm-commons.jar +share/ozone/lib/asm.jar +share/ozone/lib/asm-tree.jar +share/ozone/lib/asm-util.jar share/ozone/lib/aspectjrt.jar share/ozone/lib/aspectjweaver.jar share/ozone/lib/aws-java-sdk-core.jar @@ -141,6 +146,8 @@ share/ozone/lib/jetty-util-ajax.jar share/ozone/lib/jetty-util.jar share/ozone/lib/jetty-webapp.jar share/ozone/lib/jetty-xml.jar +share/ozone/lib/jffi.jar +share/ozone/lib/jffi-native.jar share/ozone/lib/jgrapht-core.jar share/ozone/lib/jgrapht-ext.jar share/ozone/lib/jgraphx.jar @@ -148,6 +155,11 @@ share/ozone/lib/jheaps.jar share/ozone/lib/jmespath-java.jar share/ozone/lib/jna.jar share/ozone/lib/jna-platform.jar +share/ozone/lib/jnr-a64asm.jar +share/ozone/lib/jnr-constants.jar +share/ozone/lib/jnr-ffi.jar +share/ozone/lib/jnr-posix.jar +share/ozone/lib/jnr-x86asm.jar share/ozone/lib/joda-time.jar share/ozone/lib/jooq-codegen.jar share/ozone/lib/jooq.jar diff --git a/hadoop-ozone/dist/src/main/smoketest/s3/MultipartUpload.robot b/hadoop-ozone/dist/src/main/smoketest/s3/MultipartUpload.robot index 08fc692f711..dd06d55f75f 100644 --- a/hadoop-ozone/dist/src/main/smoketest/s3/MultipartUpload.robot +++ b/hadoop-ozone/dist/src/main/smoketest/s3/MultipartUpload.robot @@ -157,7 +157,7 @@ Test Multipart Upload Complete Compare files /tmp/part2 /tmp/${PREFIX}-multipartKey1-part2.result Test Multipart Upload with user defined metadata size larger than 2 KB - ${custom_metadata_value} = Execute printf 'v%.0s' {1..3000} + ${custom_metadata_value} = Generate Random String 3000 ${result} = Execute AWSS3APICli and checkrc create-multipart-upload --bucket ${BUCKET} --key ${PREFIX}/mpuWithLargeMetadata --metadata="custom-key1=${custom_metadata_value}" 255 Should contain ${result} MetadataTooLarge Should not contain ${result} custom-key1: ${custom_metadata_value} diff --git a/hadoop-ozone/dist/src/main/smoketest/s3/objectcopy.robot b/hadoop-ozone/dist/src/main/smoketest/s3/objectcopy.robot index b12199e300a..34fe7d1b3ea 100644 --- a/hadoop-ozone/dist/src/main/smoketest/s3/objectcopy.robot +++ b/hadoop-ozone/dist/src/main/smoketest/s3/objectcopy.robot @@ -112,6 +112,6 @@ Copy Object using an invalid copy directive Copy Object with user defined metadata size larger than 2 KB Execute echo "Randomtext" > /tmp/testfile2 - ${custom_metadata_value} = Execute printf 'v%.0s' {1..3000} + ${custom_metadata_value} = Generate Random String 3000 ${result} = Execute AWSS3ApiCli and checkrc copy-object --bucket ${DESTBUCKET} --key ${PREFIX}/copyobject/key=value/f1 --copy-source ${BUCKET}/${PREFIX}/copyobject/key=value/f1 --metadata="custom-key1=${custom_metadata_value}" --metadata-directive REPLACE 255 - Should contain ${result} MetadataTooLarge \ No newline at end of file + Should contain ${result} MetadataTooLarge diff --git a/hadoop-ozone/dist/src/main/smoketest/s3/objectputget.robot b/hadoop-ozone/dist/src/main/smoketest/s3/objectputget.robot index 4e725b036ef..12fb985348a 100644 --- a/hadoop-ozone/dist/src/main/smoketest/s3/objectputget.robot +++ b/hadoop-ozone/dist/src/main/smoketest/s3/objectputget.robot @@ -191,7 +191,7 @@ Create file with user defined metadata with gdpr enabled value in request Create file with user defined metadata size larger than 2 KB Execute echo "Randomtext" > /tmp/testfile2 - ${custom_metadata_value} = Execute printf 'v%.0s' {1..3000} + ${custom_metadata_value} = Generate Random String 3000 ${result} = Execute AWSS3APICli and checkrc put-object --bucket ${BUCKET} --key ${PREFIX}/putobject/custom-metadata/key2 --body /tmp/testfile2 --metadata="custom-key1=${custom_metadata_value}" 255 Should contain ${result} MetadataTooLarge Should not contain ${result} custom-key1: ${custom_metadata_value} @@ -199,10 +199,10 @@ Create file with user defined metadata size larger than 2 KB Create files invalid tags ${result} = Execute AWSS3APICli and checkrc put-object --bucket ${BUCKET} --key ${PREFIX}/putobject/custom-metadata/key2 --body /tmp/testfile2 --tagging="tag-key1=tag-value1&tag-key1=tag-value2" 255 Should contain ${result} InvalidTag - ${long_tag_key} = Execute printf 'v%.0s' {1..129} + ${long_tag_key} = Generate Random String 129 ${result} = Execute AWSS3APICli and checkrc put-object --bucket ${BUCKET} --key ${PREFIX}/putobject/custom-metadata/key2 --body /tmp/testfile2 --tagging="${long_tag_key}=tag-value1" 255 Should contain ${result} InvalidTag - ${long_tag_value} = Execute printf 'v%.0s' {1..257} + ${long_tag_value} = Generate Random String 257 ${result} = Execute AWSS3APICli and checkrc put-object --bucket ${BUCKET} --key ${PREFIX}/putobject/custom-metadata/key2 --body /tmp/testfile2 --tagging="tag-key1=${long_tag_value}" 255 Should contain ${result} InvalidTag diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStream.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStream.java index 817c43b1a5f..468eff9c86c 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStream.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestBlockOutputStream.java @@ -97,7 +97,7 @@ static MiniOzoneCluster createCluster() throws IOException, RatisClientConfig.RaftConfig raftClientConfig = conf.getObject(RatisClientConfig.RaftConfig.class); raftClientConfig.setRpcRequestTimeout(Duration.ofSeconds(3)); - raftClientConfig.setRpcWatchRequestTimeout(Duration.ofSeconds(3)); + raftClientConfig.setRpcWatchRequestTimeout(Duration.ofSeconds(5)); conf.setFromObject(raftClientConfig); RatisClientConfig ratisClientConfig = diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientRetriesOnExceptions.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientRetriesOnExceptions.java index 3e1667a38a6..051fef4b940 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientRetriesOnExceptions.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/client/rpc/TestOzoneClientRetriesOnExceptions.java @@ -36,6 +36,7 @@ import org.apache.hadoop.hdds.scm.container.ContainerInfo; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerNotOpenException; import org.apache.hadoop.hdds.scm.pipeline.Pipeline; +import org.apache.hadoop.hdds.scm.pipeline.choose.algorithms.RoundRobinPipelineChoosePolicy; import org.apache.hadoop.hdds.scm.storage.BlockOutputStream; import org.apache.hadoop.hdds.utils.IOUtils; import org.apache.hadoop.ozone.ClientConfigForTesting; @@ -107,6 +108,7 @@ public void init() throws Exception { conf.set(OzoneConfigKeys.OZONE_SCM_CLOSE_CONTAINER_WAIT_DURATION, "2s"); conf.set(ScmConfigKeys.OZONE_SCM_PIPELINE_SCRUB_INTERVAL, "2s"); conf.set(ScmConfigKeys.OZONE_SCM_PIPELINE_DESTROY_TIMEOUT, "5s"); + conf.set("hdds.scm.pipeline.choose.policy.impl", RoundRobinPipelineChoosePolicy.class.getName()); conf.setQuietMode(false); ClientConfigForTesting.newBuilder(StorageUnit.BYTES) @@ -216,7 +218,7 @@ void testMaxRetriesByOzoneClient() throws Exception { .getPipeline(container.getPipelineID()); XceiverClientSpi xceiverClient = xceiverClientManager.acquireClient(pipeline); - Assumptions.assumeFalse(containerList.contains(containerID)); + assertThat(containerList.contains(containerID)); containerList.add(containerID); xceiverClient.sendCommand(ContainerTestHelper .getCreateContainerRequest(containerID, pipeline)); diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMPerformanceMetrics.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMPerformanceMetrics.java index f2f11025158..a14adc80dde 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMPerformanceMetrics.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OMPerformanceMetrics.java @@ -19,6 +19,7 @@ import org.apache.hadoop.metrics2.MetricsSystem; import org.apache.hadoop.metrics2.annotation.Metric; import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem; +import org.apache.hadoop.metrics2.lib.MutableGaugeFloat; import org.apache.hadoop.metrics2.lib.MutableRate; /** @@ -113,12 +114,21 @@ public static void unregister() { @Metric(about = "Ratis local command execution latency in nano seconds") private MutableRate validateAndUpdateCacheLatencyNs; + @Metric(about = "average pagination for listKeys") + private MutableRate listKeysAveragePagination; + + @Metric(about = "ops per second for listKeys") + private MutableGaugeFloat listKeysOpsPerSec; + @Metric(about = "ACLs check latency in listKeys") private MutableRate listKeysAclCheckLatencyNs; @Metric(about = "resolveBucketLink latency in listKeys") private MutableRate listKeysResolveBucketLatencyNs; + @Metric(about = "readFromRockDb latency in listKeys") + private MutableRate listKeysReadFromRocksDbLatencyNs; + public void addLookupLatency(long latencyInNs) { lookupLatencyNs.add(latencyInNs); } @@ -216,6 +226,14 @@ public MutableRate getValidateAndUpdateCacheLatencyNs() { return validateAndUpdateCacheLatencyNs; } + public void setListKeysAveragePagination(long keyCount) { + listKeysAveragePagination.add(keyCount); + } + + public void setListKeysOpsPerSec(float opsPerSec) { + listKeysOpsPerSec.set(opsPerSec); + } + MutableRate getListKeysAclCheckLatencyNs() { return listKeysAclCheckLatencyNs; } @@ -223,4 +241,8 @@ MutableRate getListKeysAclCheckLatencyNs() { MutableRate getListKeysResolveBucketLatencyNs() { return listKeysResolveBucketLatencyNs; } + + public void addListKeysReadFromRocksDbLatencyNs(long latencyInNs) { + listKeysReadFromRocksDbLatencyNs.add(latencyInNs); + } } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java index 6e25dc1f7f0..9f90643a17c 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OmMetadataManagerImpl.java @@ -317,6 +317,7 @@ public class OmMetadataManagerImpl implements OMMetadataManager, private final Map tableCacheMetricsMap = new HashMap<>(); private SnapshotChainManager snapshotChainManager; + private final OMPerformanceMetrics perfMetrics; private final S3Batcher s3Batcher = new S3SecretBatcher(); /** @@ -328,7 +329,15 @@ public class OmMetadataManagerImpl implements OMMetadataManager, */ public OmMetadataManagerImpl(OzoneConfiguration conf, OzoneManager ozoneManager) throws IOException { + this(conf, ozoneManager, null); + } + + public OmMetadataManagerImpl(OzoneConfiguration conf, + OzoneManager ozoneManager, + OMPerformanceMetrics perfMetrics) + throws IOException { this.ozoneManager = ozoneManager; + this.perfMetrics = perfMetrics; this.lock = new OzoneManagerLock(conf); // TODO: This is a temporary check. Once fully implemented, all OM state // change should go through Ratis - be it standalone (for non-HA) or @@ -350,6 +359,7 @@ protected OmMetadataManagerImpl() { OzoneConfiguration conf = new OzoneConfiguration(); this.lock = new OzoneManagerLock(conf); this.omEpoch = 0; + perfMetrics = null; } public static OmMetadataManagerImpl createCheckpointMetadataManager( @@ -384,6 +394,7 @@ private OmMetadataManagerImpl(OzoneConfiguration conf, File dir, String name) setStore(loadDB(conf, dir, name, true, java.util.Optional.of(Boolean.TRUE), Optional.empty())); initializeOmTables(CacheType.PARTIAL_CACHE, false); + perfMetrics = null; } @@ -421,6 +432,7 @@ private OmMetadataManagerImpl(OzoneConfiguration conf, File dir, String name) stop(); throw e; } + perfMetrics = null; } @Override @@ -1163,7 +1175,7 @@ public List listBuckets(final String volumeName, public ListKeysResult listKeys(String volumeName, String bucketName, String startKey, String keyPrefix, int maxKeys) throws IOException { - + long startNanos = Time.monotonicNowNanos(); List result = new ArrayList<>(); if (maxKeys <= 0) { return new ListKeysResult(result, false); @@ -1232,11 +1244,11 @@ public ListKeysResult listKeys(String volumeName, String bucketName, cacheKeyMap.put(key, omKeyInfo); } } - + long readFromRDbStartNs, readFromRDbStopNs = 0; // Get maxKeys from DB if it has. - try (TableIterator> keyIter = getKeyTable(getBucketLayout()).iterator()) { + readFromRDbStartNs = Time.monotonicNowNanos(); KeyValue< String, OmKeyInfo > kv; keyIter.seek(seekKey); // we need to iterate maxKeys + 1 here because if skipStartKey is true, @@ -1259,10 +1271,24 @@ public ListKeysResult listKeys(String volumeName, String bucketName, break; } } + readFromRDbStopNs = Time.monotonicNowNanos(); } boolean isTruncated = cacheKeyMap.size() > maxKeys; + if (perfMetrics != null) { + long keyCount; + if (isTruncated) { + keyCount = maxKeys; + } else { + keyCount = cacheKeyMap.size(); + } + perfMetrics.setListKeysAveragePagination(keyCount); + float opsPerSec = + keyCount / ((Time.monotonicNowNanos() - startNanos) / 1000000000.0f); + perfMetrics.setListKeysOpsPerSec(opsPerSec); + perfMetrics.addListKeysReadFromRocksDbLatencyNs(readFromRDbStopNs - readFromRDbStartNs); + } // Finally DB entries and cache entries are merged, then return the count // of maxKeys from the sorted map. currentCount = 0; diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java index b3078739119..63617ee3637 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/OzoneManagerStarter.java @@ -17,11 +17,11 @@ package org.apache.hadoop.ozone.om; -import org.apache.hadoop.hdds.StringUtils; import org.apache.hadoop.hdds.cli.GenericCli; import org.apache.hadoop.hdds.cli.HddsVersionProvider; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.tracing.TracingUtil; +import org.apache.hadoop.hdds.utils.HddsServerUtil; import org.apache.hadoop.ozone.util.OzoneNetUtils; import org.apache.hadoop.ozone.util.OzoneVersionInfo; import org.apache.hadoop.ozone.util.ShutdownHookManager; @@ -172,7 +172,7 @@ private void commonInit() { String[] originalArgs = getCmd().getParseResult().originalArgs() .toArray(new String[0]); - StringUtils.startupShutdownMessage(OzoneVersionInfo.OZONE_VERSION_INFO, + HddsServerUtil.startupShutdownMessage(OzoneVersionInfo.OZONE_VERSION_INFO, OzoneManager.class, originalArgs, LOG, conf); } diff --git a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java index 29b2b319532..bb4fc076bde 100644 --- a/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java +++ b/hadoop-ozone/ozone-manager/src/main/java/org/apache/hadoop/ozone/om/service/SnapshotDeletingService.java @@ -100,6 +100,7 @@ public class SnapshotDeletingService extends AbstractKeyDeletingService { private final long snapshotDeletionPerTask; private final int keyLimitPerSnapshot; private final int ratisByteLimit; + private final boolean isSstFilteringServiceEnabled; public SnapshotDeletingService(long interval, long serviceTimeout, OzoneManager ozoneManager, ScmBlockLocationProtocol scmClient) @@ -127,6 +128,8 @@ public SnapshotDeletingService(long interval, long serviceTimeout, this.keyLimitPerSnapshot = conf.getInt( OZONE_SNAPSHOT_KEY_DELETING_LIMIT_PER_TASK, OZONE_SNAPSHOT_KEY_DELETING_LIMIT_PER_TASK_DEFAULT); + + this.isSstFilteringServiceEnabled = ((KeyManagerImpl) ozoneManager.getKeyManager()).isSstFilteringSvcEnabled(); } private class SnapshotDeletingTask implements BackgroundTask { @@ -153,12 +156,9 @@ public BackgroundTaskResult call() throws InterruptedException { while (iterator.hasNext() && snapshotLimit > 0) { SnapshotInfo snapInfo = iterator.next().getValue(); - boolean isSstFilteringServiceEnabled = - ((KeyManagerImpl) ozoneManager.getKeyManager()) - .isSstFilteringSvcEnabled(); // Only Iterate in deleted snapshot - if (shouldIgnoreSnapshot(snapInfo, isSstFilteringServiceEnabled)) { + if (shouldIgnoreSnapshot(snapInfo)) { continue; } @@ -591,10 +591,10 @@ public void submitRequest(OMRequest omRequest) { } } - public static boolean shouldIgnoreSnapshot(SnapshotInfo snapInfo, - boolean isSstFilteringServiceEnabled) { + @VisibleForTesting + boolean shouldIgnoreSnapshot(SnapshotInfo snapInfo) { SnapshotInfo.SnapshotStatus snapshotStatus = snapInfo.getSnapshotStatus(); - return !(snapshotStatus == SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED) + return snapshotStatus != SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED || (isSstFilteringServiceEnabled && !snapInfo.isSstFiltered()); } diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java new file mode 100644 index 00000000000..42da7377ea2 --- /dev/null +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/service/TestSnapshotDeletingService.java @@ -0,0 +1,100 @@ +/* + * 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.hadoop.ozone.om.service; + + +import org.apache.hadoop.hdds.conf.OzoneConfiguration; +import org.apache.hadoop.hdds.scm.protocol.ScmBlockLocationProtocol; +import org.apache.hadoop.ozone.om.KeyManagerImpl; +import org.apache.hadoop.ozone.om.OmMetadataManagerImpl; +import org.apache.hadoop.ozone.om.OmSnapshotManager; +import org.apache.hadoop.ozone.om.OzoneManager; +import org.apache.hadoop.ozone.om.SnapshotChainManager; +import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; +import org.junit.jupiter.api.extension.ExtendWith; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.jupiter.MockitoExtension; + +import java.io.IOException; +import java.time.Duration; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertEquals; + +/** + * Class to unit test SnapshotDeletingService. + */ +@ExtendWith(MockitoExtension.class) +public class TestSnapshotDeletingService { + @Mock + private OzoneManager ozoneManager; + @Mock + private KeyManagerImpl keyManager; + @Mock + private OmSnapshotManager omSnapshotManager; + @Mock + private SnapshotChainManager chainManager; + @Mock + private OmMetadataManagerImpl omMetadataManager; + @Mock + private ScmBlockLocationProtocol scmClient; + private final OzoneConfiguration conf = new OzoneConfiguration();; + private final long sdsRunInterval = Duration.ofMillis(1000).toMillis(); + private final long sdsServiceTimeout = Duration.ofSeconds(10).toMillis(); + + + private static Stream testCasesForIgnoreSnapshotGc() { + SnapshotInfo filteredSnapshot = SnapshotInfo.newBuilder().setSstFiltered(true).setName("snap1").build(); + SnapshotInfo unFilteredSnapshot = SnapshotInfo.newBuilder().setSstFiltered(false).setName("snap1").build(); + return Stream.of( + Arguments.of(filteredSnapshot, SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED, true, false), + Arguments.of(filteredSnapshot, SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE, true, true), + Arguments.of(unFilteredSnapshot, SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED, true, true), + Arguments.of(unFilteredSnapshot, SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE, true, true), + Arguments.of(filteredSnapshot, SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED, false, false), + Arguments.of(unFilteredSnapshot, SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED, false, false), + Arguments.of(unFilteredSnapshot, SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE, false, true), + Arguments.of(filteredSnapshot, SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE, false, true)); + } + + @ParameterizedTest + @MethodSource("testCasesForIgnoreSnapshotGc") + public void testProcessSnapshotLogicInSDS(SnapshotInfo snapshotInfo, + SnapshotInfo.SnapshotStatus status, + boolean sstFilteringServiceEnabled, + boolean expectedOutcome) + throws IOException { + Mockito.when(keyManager.isSstFilteringSvcEnabled()).thenReturn(sstFilteringServiceEnabled); + Mockito.when(omMetadataManager.getSnapshotChainManager()).thenReturn(chainManager); + Mockito.when(ozoneManager.getKeyManager()).thenReturn(keyManager); + Mockito.when(ozoneManager.getOmSnapshotManager()).thenReturn(omSnapshotManager); + Mockito.when(ozoneManager.getMetadataManager()).thenReturn(omMetadataManager); + Mockito.when(ozoneManager.getConfiguration()).thenReturn(conf); + + SnapshotDeletingService snapshotDeletingService = + new SnapshotDeletingService(sdsRunInterval, sdsServiceTimeout, ozoneManager, scmClient); + + snapshotInfo.setSnapshotStatus(status); + assertEquals(expectedOutcome, snapshotDeletingService.shouldIgnoreSnapshot(snapshotInfo)); + } +} diff --git a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestOmSnapshotUtils.java b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestOmSnapshotUtils.java index 190db469c19..8ab652612f5 100644 --- a/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestOmSnapshotUtils.java +++ b/hadoop-ozone/ozone-manager/src/test/java/org/apache/hadoop/ozone/om/snapshot/TestOmSnapshotUtils.java @@ -18,20 +18,14 @@ package org.apache.hadoop.ozone.om.snapshot; -import org.apache.hadoop.ozone.om.helpers.SnapshotInfo; -import org.apache.hadoop.ozone.om.service.SnapshotDeletingService; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.io.TempDir; -import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.MethodSource; import java.io.File; import java.nio.file.Files; import java.nio.file.Path; import java.util.Set; import java.util.stream.Collectors; -import java.util.stream.Stream; import static java.nio.charset.StandardCharsets.UTF_8; import static org.apache.hadoop.ozone.om.snapshot.OmSnapshotUtils.getINode; @@ -82,42 +76,4 @@ public void testLinkFiles(@TempDir File tempDir) throws Exception { assertEquals(tree1Files, tree2Files); } - - - private static Stream testCasesForIgnoreSnapshotGc() { - SnapshotInfo filteredSnapshot = - SnapshotInfo.newBuilder().setSstFiltered(true).setName("snap1").build(); - SnapshotInfo unFilteredSnapshot = - SnapshotInfo.newBuilder().setSstFiltered(false).setName("snap1") - .build(); - // {IsSnapshotFiltered,isSnapshotDeleted,IsSstServiceEnabled = ShouldIgnore} - return Stream.of(Arguments.of(filteredSnapshot, - SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED, true, false), - Arguments.of(filteredSnapshot, - SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE, true, true), - Arguments.of(unFilteredSnapshot, - SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED, true, true), - Arguments.of(unFilteredSnapshot, - SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE, true, true), - Arguments.of(filteredSnapshot, - SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED, false, false), - Arguments.of(unFilteredSnapshot, - SnapshotInfo.SnapshotStatus.SNAPSHOT_DELETED, false, false), - Arguments.of(unFilteredSnapshot, - SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE, false, true), - Arguments.of(filteredSnapshot, - SnapshotInfo.SnapshotStatus.SNAPSHOT_ACTIVE, false, true)); - } - - @ParameterizedTest - @MethodSource("testCasesForIgnoreSnapshotGc") - public void testProcessSnapshotLogicInSDS(SnapshotInfo snapshotInfo, - SnapshotInfo.SnapshotStatus status, boolean isSstFilteringSvcEnabled, - boolean expectedOutcome) { - snapshotInfo.setSnapshotStatus(status); - assertEquals(expectedOutcome, - SnapshotDeletingService.shouldIgnoreSnapshot(snapshotInfo, - isSstFilteringSvcEnabled)); - } - } diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServer.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServer.java index a079bfe508e..3295eb4524c 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServer.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconServer.java @@ -22,7 +22,6 @@ import com.google.inject.Guice; import com.google.inject.Injector; import org.apache.hadoop.hdds.HddsUtils; -import org.apache.hadoop.hdds.StringUtils; import org.apache.hadoop.hdds.cli.GenericCli; import org.apache.hadoop.hdds.conf.OzoneConfiguration; import org.apache.hadoop.hdds.protocolPB.SCMSecurityProtocolClientSideTranslatorPB; @@ -102,7 +101,7 @@ public Void call() throws Exception { .toArray(new String[0]); configuration = createOzoneConfiguration(); - StringUtils.startupShutdownMessage(OzoneVersionInfo.OZONE_VERSION_INFO, + HddsServerUtil.startupShutdownMessage(OzoneVersionInfo.OZONE_VERSION_INFO, ReconServer.class, originalArgs, LOG, configuration); ConfigurationProvider.setConfiguration(configuration); diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java index fe920b7098b..e346b4bc9e7 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/ReconUtils.java @@ -424,6 +424,32 @@ public static void upsertGlobalStatsTable(Configuration sqlConfiguration, } } + /** + * Converts Unix numeric permissions into a symbolic representation. + * @param numericPermissions The numeric string, e.g., "750". + * @return The symbolic representation, e.g., "rwxr-x---". + */ + public static String convertNumericToSymbolic(String numericPermissions) { + int owner = Character.getNumericValue(numericPermissions.charAt(0)); + int group = Character.getNumericValue(numericPermissions.charAt(1)); + int others = Character.getNumericValue(numericPermissions.charAt(2)); + + return String.format("%s%s%s", + convertToSymbolicPermission(owner), + convertToSymbolicPermission(group), + convertToSymbolicPermission(others)); + } + + /** + * Converts a single digit Unix permission into a symbolic representation. + * @param permission The permission digit. + * @return The symbolic representation for the digit. + */ + public static String convertToSymbolicPermission(int permission) { + String[] symbols = {"---", "--x", "-w-", "-wx", "r--", "r-x", "rw-", "rwx"}; + return symbols[permission]; + } + /** * Sorts a list of DiskUsage objects in descending order by size using parallel sorting and * returns the top N records as specified by the limit. diff --git a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java index 26b9bec9d6f..bde89eea1da 100644 --- a/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java +++ b/hadoop-ozone/recon/src/main/java/org/apache/hadoop/ozone/recon/spi/impl/OzoneManagerServiceProviderImpl.java @@ -23,16 +23,23 @@ import java.io.File; import java.io.IOException; import java.io.InputStream; +import java.nio.file.Files; import java.nio.file.Path; import java.nio.file.Paths; +import java.nio.file.attribute.PosixFilePermission; +import java.nio.file.attribute.PosixFilePermissions; +import java.util.Arrays; import java.util.List; +import java.util.Set; import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.ThreadFactory; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; import com.google.common.util.concurrent.ThreadFactoryBuilder; +import org.apache.hadoop.hdds.recon.ReconConfigKeys; import org.apache.hadoop.hdds.utils.db.RocksDatabase; import org.apache.hadoop.hdds.utils.db.managed.ManagedWriteBatch; import org.apache.hadoop.hdds.utils.db.managed.ManagedWriteOptions; @@ -50,6 +57,7 @@ import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.DBUpdatesRequest; import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos.ServicePort.Type; +import org.apache.hadoop.ozone.recon.ReconContext; import org.apache.hadoop.ozone.recon.ReconServerConfigKeys; import org.apache.hadoop.ozone.recon.ReconUtils; import org.apache.hadoop.ozone.recon.metrics.OzoneManagerSyncMetrics; @@ -63,6 +71,8 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.commons.io.FileUtils; + +import static org.apache.hadoop.hdds.recon.ReconConfigKeys.OZONE_RECON_DB_DIRS_PERMISSIONS_DEFAULT; import static org.apache.hadoop.ozone.OzoneConsts.OZONE_DB_CHECKPOINT_REQUEST_FLUSH; import static org.apache.hadoop.ozone.OzoneConsts.OZONE_DB_CHECKPOINT_HTTP_ENDPOINT; import static org.apache.hadoop.ozone.om.OMConfigKeys.OZONE_OM_HTTP_AUTH_TYPE; @@ -81,6 +91,7 @@ import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.RECON_OM_DELTA_UPDATE_LIMIT_DEFUALT; import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.RECON_OM_DELTA_UPDATE_LOOP_LIMIT; import static org.apache.hadoop.ozone.recon.ReconServerConfigKeys.RECON_OM_DELTA_UPDATE_LOOP_LIMIT_DEFUALT; +import static org.apache.hadoop.ozone.recon.ReconUtils.convertNumericToSymbolic; import static org.apache.ratis.proto.RaftProtos.RaftPeerRole.LEADER; import org.hadoop.ozone.recon.schema.tables.daos.ReconTaskStatusDao; @@ -101,6 +112,7 @@ public class OzoneManagerServiceProviderImpl private URLConnectionFactory connectionFactory; private File omSnapshotDBParentDir = null; + private File reconDbDir = null; private String omDBSnapshotUrl; private OzoneManagerProtocol ozoneManagerClient; @@ -119,6 +131,7 @@ public class OzoneManagerServiceProviderImpl private AtomicBoolean isSyncDataFromOMRunning; private final String threadNamePrefix; private ThreadFactory threadFactory; + private ReconContext reconContext; /** * OM Snapshot related task names. @@ -134,7 +147,8 @@ public OzoneManagerServiceProviderImpl( ReconOMMetadataManager omMetadataManager, ReconTaskController reconTaskController, ReconUtils reconUtils, - OzoneManagerProtocol ozoneManagerClient) { + OzoneManagerProtocol ozoneManagerClient, + ReconContext reconContext) { int connectionTimeout = (int) configuration.getTimeDuration( OZONE_RECON_OM_CONNECTION_TIMEOUT, @@ -168,6 +182,8 @@ public OzoneManagerServiceProviderImpl( omSnapshotDBParentDir = reconUtils.getReconDbDir(configuration, OZONE_RECON_OM_SNAPSHOT_DB_DIR); + reconDbDir = reconUtils.getReconDbDir(configuration, + ReconConfigKeys.OZONE_RECON_DB_DIR); HttpConfig.Policy policy = HttpConfig.getHttpPolicy(configuration); @@ -205,6 +221,7 @@ public OzoneManagerServiceProviderImpl( this.threadFactory = new ThreadFactoryBuilder().setNameFormat(threadNamePrefix + "SyncOM-%d") .build(); + this.reconContext = reconContext; } public void registerOMDBTasks() { @@ -242,7 +259,7 @@ public void start() { try { omMetadataManager.start(configuration); } catch (IOException ioEx) { - LOG.error("Error staring Recon OM Metadata Manager.", ioEx); + LOG.error("Error starting Recon OM Metadata Manager.", ioEx); } reconTaskController.start(); long initialDelay = configuration.getTimeDuration( @@ -264,10 +281,12 @@ private void startSyncDataFromOM(long initialDelay) { LOG.debug("Started the OM DB sync scheduler."); scheduler.scheduleWithFixedDelay(() -> { try { + LOG.info("Last known sequence number before sync: {}", getCurrentOMDBSequenceNumber()); boolean isSuccess = syncDataFromOM(); if (!isSuccess) { LOG.debug("OM DB sync is already running."); } + LOG.info("Sequence number after sync: {}", getCurrentOMDBSequenceNumber()); } catch (Throwable t) { LOG.error("Unexpected exception while syncing data from OM.", t); } @@ -361,16 +380,30 @@ connectionFactory, getOzoneManagerSnapshotUrl(), return null; }); // Untar the checkpoint file. - Path untarredDbDir = Paths.get(omSnapshotDBParentDir.getAbsolutePath(), - snapshotFileName); + Path untarredDbDir = Paths.get(omSnapshotDBParentDir.getAbsolutePath(), snapshotFileName); reconUtils.untarCheckpointFile(targetFile, untarredDbDir); FileUtils.deleteQuietly(targetFile); + // Validate the presence of required SST files + File[] sstFiles = untarredDbDir.toFile().listFiles((dir, name) -> name.endsWith(".sst")); + if (sstFiles == null || sstFiles.length == 0) { + LOG.warn("No SST files found in the OM snapshot directory: {}", untarredDbDir); + } + + List sstFileNames = Arrays.stream(sstFiles) + .map(File::getName) + .collect(Collectors.toList()); + LOG.debug("Valid SST files found: {}", sstFileNames); + // Currently, OM DB type is not configurable. Hence, defaulting to // RocksDB. + reconContext.updateHealthStatus(new AtomicBoolean(true)); + reconContext.getErrors().remove(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); return new RocksDBCheckpoint(untarredDbDir); } catch (IOException e) { LOG.error("Unable to obtain Ozone Manager DB Snapshot. ", e); + reconContext.updateHealthStatus(new AtomicBoolean(false)); + reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); } return null; } @@ -381,25 +414,36 @@ connectionFactory, getOzoneManagerSnapshotUrl(), */ @VisibleForTesting boolean updateReconOmDBWithNewSnapshot() throws IOException { + // Check permissions of the Recon DB directory + checkAndValidateReconDbPermissions(); // Obtain the current DB snapshot from OM and // update the in house OM metadata managed DB instance. long startTime = Time.monotonicNow(); DBCheckpoint dbSnapshot = getOzoneManagerDBSnapshot(); metrics.updateSnapshotRequestLatency(Time.monotonicNow() - startTime); - if (dbSnapshot != null && dbSnapshot.getCheckpointLocation() != null) { - LOG.info("Got new checkpoint from OM : " + - dbSnapshot.getCheckpointLocation()); - try { - omMetadataManager.updateOmDB( - dbSnapshot.getCheckpointLocation().toFile()); - return true; - } catch (IOException e) { - LOG.error("Unable to refresh Recon OM DB Snapshot. ", e); - } - } else { - LOG.error("Null snapshot location got from OM."); + + if (dbSnapshot == null) { + LOG.error("Failed to obtain a valid DB snapshot from Ozone Manager. This could be due to " + + "missing SST files or other fetch issues."); + return false; + } + + if (dbSnapshot.getCheckpointLocation() == null) { + LOG.error("Snapshot checkpoint location is null, indicating a failure to properly fetch or " + + "store the snapshot."); + return false; + } + + LOG.info("Attempting to update Recon OM DB with new snapshot located at: {}", + dbSnapshot.getCheckpointLocation()); + try { + omMetadataManager.updateOmDB(dbSnapshot.getCheckpointLocation().toFile()); + LOG.info("Successfully updated Recon OM DB with new snapshot."); + return true; + } catch (IOException e) { + LOG.error("Unable to refresh Recon OM DB Snapshot.", e); + return false; } - return false; } /** @@ -549,24 +593,69 @@ public boolean syncDataFromOM() { // Reinitialize tasks that are listening. LOG.info("Calling reprocess on Recon tasks."); reconTaskController.reInitializeTasks(omMetadataManager); + + // Update health status in ReconContext + reconContext.updateHealthStatus(new AtomicBoolean(true)); + reconContext.getErrors().remove(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); + } else { + metrics.incrNumSnapshotRequestsFailed(); + // Update health status in ReconContext + reconContext.updateHealthStatus(new AtomicBoolean(false)); + reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); } } catch (InterruptedException intEx) { Thread.currentThread().interrupt(); } catch (Exception e) { metrics.incrNumSnapshotRequestsFailed(); LOG.error("Unable to update Recon's metadata with new OM DB. ", e); + // Update health status in ReconContext + reconContext.updateHealthStatus(new AtomicBoolean(false)); + reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED); } } } finally { isSyncDataFromOMRunning.set(false); } } else { - LOG.debug("OM DB sync is already running."); + LOG.info("OM DB sync is already running."); return false; } return true; } + public void checkAndValidateReconDbPermissions() { + File dbDir = new File(reconDbDir.getPath()); + if (!dbDir.exists()) { + LOG.error("Recon DB directory does not exist: {}", dbDir.getAbsolutePath()); + return; + } + + try { + // Fetch expected minimum permissions from configuration + String expectedPermissions = + configuration.get(ReconConfigKeys.OZONE_RECON_DB_DIRS_PERMISSIONS, OZONE_RECON_DB_DIRS_PERMISSIONS_DEFAULT); + Set expectedPosixPermissions = + PosixFilePermissions.fromString(convertNumericToSymbolic(expectedPermissions)); + + // Get actual permissions + Set actualPermissions = Files.getPosixFilePermissions(dbDir.toPath()); + String actualPermissionsStr = PosixFilePermissions.toString(actualPermissions); + + // Check if actual permissions meet the minimum required permissions + if (actualPermissions.containsAll(expectedPosixPermissions)) { + LOG.info("Permissions for Recon DB directory '{}' meet the minimum required permissions '{}'", + dbDir.getAbsolutePath(), expectedPermissions); + } else { + LOG.warn("Permissions for Recon DB directory '{}' are '{}', which do not meet the minimum" + + " required permissions '{}'", dbDir.getAbsolutePath(), actualPermissionsStr, expectedPermissions); + } + } catch (IOException e) { + LOG.error("Failed to retrieve permissions for Recon DB directory: {}", dbDir.getAbsolutePath(), e); + } catch (IllegalArgumentException e) { + LOG.error("Configuration issue: {}", e.getMessage()); + } + } + /** * Get OM RocksDB's latest sequence number. * @return latest sequence number. diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/api/db.json b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/api/db.json index 47dd6ba0406..51f65119352 100644 --- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/api/db.json +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/api/db.json @@ -1398,217 +1398,42 @@ "root": { "status": "OK", "path": "/", - "size": 1709108, + "size": 15160, "sizeWithReplica": -1, - "subPathCount": 30, + "subPathCount": 5, "subPaths": [ - { - "key": false, - "path": "/vol-0-30461", - "size": 33096, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol-1-82078", - "size": 33096, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol-2-96509", - "size": 330496, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol-3-75124", - "size": 330496, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol-4-53147", - "size": 330240, - "sizeWithReplica": -1, - "isKey": false - }, { "key": false, "path": "/vol1", - "size": 12204, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol17", - "size": 8136, + "size": 10000, "sizeWithReplica": -1, "isKey": false }, { "key": false, - "path": "/vol21", - "size": 8136, + "path": "/vol-2", + "size": 10, "sizeWithReplica": -1, "isKey": false }, { "key": false, - "path": "/vol22", - "size": 8136, + "path": "/vol3", + "size": 100, "sizeWithReplica": -1, "isKey": false }, { "key": false, "path": "/vol4", - "size": 8136, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol18", - "size": 4068, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol19", - "size": 4068, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/s3v", - "size": 0, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol10", - "size": 0, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol11", - "size": 0, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol12", - "size": 0, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol13", - "size": 0, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol14", - "size": 0, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol15", - "size": 0, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol16", - "size": 0, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol2", - "size": 0, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol20", - "size": 0, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol3", - "size": 0, + "size": 50, "sizeWithReplica": -1, "isKey": false }, { "key": false, "path": "/vol5", - "size": 0, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol6", - "size": 0, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol7", - "size": 0, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol8", - "size": 0, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol9", - "size": 0, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol10", - "size": 10, - "sizeWithReplica": -1, - "isKey": false - }, - { - "key": false, - "path": "/vol11", - "size": 20, + "size": 5000, "sizeWithReplica": -1, "isKey": false } diff --git a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/diskUsage/diskUsage.tsx b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/diskUsage/diskUsage.tsx index e82d8f71fca..df280fe9fe9 100644 --- a/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/diskUsage/diskUsage.tsx +++ b/hadoop-ozone/recon/src/main/resources/webapps/recon/ozone-recon-web/src/views/diskUsage/diskUsage.tsx @@ -29,6 +29,7 @@ import { AxiosGetHelper, cancelRequests } from 'utils/axiosRequestHelper'; const DEFAULT_DISPLAY_LIMIT = 10; const OTHER_PATH_NAME = 'Other Objects'; const MAX_DISPLAY_LIMIT = 30; +const MIN_BLOCK_SIZE = 0.05; interface IDUSubpath { path: string; @@ -63,6 +64,7 @@ let cancelPieSignal: AbortController let cancelSummarySignal: AbortController let cancelQuotaSignal: AbortController; let cancelKeyMetadataSignal: AbortController; +let valuesWithMinBlockSize: number[] = []; export class DiskUsage extends React.Component, IDUState> { constructor(props = {}) { @@ -165,7 +167,7 @@ export class DiskUsage extends React.Component, IDUState> } } - let pathLabels, values, percentage, sizeStr, pieces, subpathName; + let pathLabels, values: number[] = [], percentage, sizeStr, pieces, subpathName; if (duResponse.subPathCount === 0 || subpaths === 0) { pieces = duResponse && duResponse.path != null && duResponse.path.split('/'); @@ -185,9 +187,17 @@ export class DiskUsage extends React.Component, IDUState> return (subpath.isKey || subpathName === OTHER_PATH_NAME) ? subpathName : subpathName + '/'; }); - values = subpaths.map(subpath => { - return subpath.size / dataSize; - }); + // To avoid NaN Condition NaN will get divide by Zero to avoid map iterations + if (dataSize > 0) { + values = subpaths.map(subpath => { + return subpath.size / dataSize; + }); + } + + // Adding a MIN_BLOCK_SIZE to non-zero size entities to ensure that even the smallest entities are visible on the pie chart. + // Note: The percentage and size string calculations remain unchanged. + const clonedValues = structuredClone(values); + valuesWithMinBlockSize = clonedValues && clonedValues.map((item: number) => item > 0 ? item + MIN_BLOCK_SIZE : item); percentage = values.map(value => { return (value * 100).toFixed(2); @@ -197,7 +207,6 @@ export class DiskUsage extends React.Component, IDUState> return byteToSize(subpath.size, 1); }); } - this.setState({ // Normalized path isLoading: false, @@ -209,7 +218,7 @@ export class DiskUsage extends React.Component, IDUState> plotData: [{ type: 'pie', hole: 0.2, - values: values, + values: valuesWithMinBlockSize, customdata: percentage, labels: pathLabels, text: sizeStr, @@ -524,10 +533,7 @@ export class DiskUsage extends React.Component, IDUState> return (

- Disk Usage   - - - + Disk Usage
{isLoading ? Loading... : ( @@ -551,6 +557,10 @@ export class DiskUsage extends React.Component, IDUState>
+      + + +