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>
Display Limit: {displayLimit}
+
+
+
+
this.showMetadataDetails(e, returnPath)}>
@@ -567,8 +577,8 @@ export class DiskUsage extends React.Component, IDUState>
data={plotData}
layout={
{
- width: 1200,
- height: 900,
+ width: 1000,
+ height: 850,
font: {
family: 'Roboto, sans-serif',
size: 15
diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestTriggerDBSyncEndpoint.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestTriggerDBSyncEndpoint.java
index 9bf824e4eea..7ad5dcc58cd 100644
--- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestTriggerDBSyncEndpoint.java
+++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/api/TestTriggerDBSyncEndpoint.java
@@ -27,6 +27,7 @@
import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
import org.apache.hadoop.ozone.recon.MetricsServiceProviderFactory;
+import org.apache.hadoop.ozone.recon.ReconContext;
import org.apache.hadoop.ozone.recon.ReconTestInjector;
import org.apache.hadoop.ozone.recon.ReconUtils;
import org.apache.hadoop.ozone.recon.common.CommonUtils;
@@ -121,7 +122,7 @@ public void setUp() throws IOException, AuthenticationException {
OzoneManagerServiceProviderImpl ozoneManagerServiceProvider =
new OzoneManagerServiceProviderImpl(configuration,
reconOMMetadataManager, reconTaskController, reconUtilsMock,
- ozoneManagerProtocol);
+ ozoneManagerProtocol, new ReconContext(configuration, reconUtilsMock));
ozoneManagerServiceProvider.start();
reconTestInjector =
diff --git a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestOzoneManagerServiceProviderImpl.java b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestOzoneManagerServiceProviderImpl.java
index 032bff80ade..2700034aaed 100644
--- a/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestOzoneManagerServiceProviderImpl.java
+++ b/hadoop-ozone/recon/src/test/java/org/apache/hadoop/ozone/recon/spi/impl/TestOzoneManagerServiceProviderImpl.java
@@ -66,6 +66,7 @@
import org.apache.hadoop.ozone.om.helpers.DBUpdates;
import org.apache.hadoop.ozone.om.protocol.OzoneManagerProtocol;
import org.apache.hadoop.ozone.protocol.proto.OzoneManagerProtocolProtos;
+import org.apache.hadoop.ozone.recon.ReconContext;
import org.apache.hadoop.ozone.recon.ReconUtils;
import org.apache.hadoop.ozone.recon.common.CommonUtils;
import org.apache.hadoop.ozone.recon.metrics.OzoneManagerSyncMetrics;
@@ -91,6 +92,7 @@ public class TestOzoneManagerServiceProviderImpl {
private OzoneConfiguration configuration;
private OzoneManagerProtocol ozoneManagerProtocol;
private CommonUtils commonUtils;
+ private ReconContext reconContext;
@BeforeEach
public void setUp(@TempDir File dirReconSnapDB, @TempDir File dirReconDB)
@@ -103,6 +105,7 @@ public void setUp(@TempDir File dirReconSnapDB, @TempDir File dirReconDB)
configuration.set("ozone.om.address", "localhost:9862");
ozoneManagerProtocol = getMockOzoneManagerClient(new DBUpdates());
commonUtils = new CommonUtils();
+ reconContext = new ReconContext(configuration, new ReconUtils());
}
@Test
@@ -136,7 +139,7 @@ public void testUpdateReconOmDBWithNewSnapshot(
OzoneManagerServiceProviderImpl ozoneManagerServiceProvider =
new OzoneManagerServiceProviderImpl(configuration,
reconOMMetadataManager, reconTaskController, reconUtilsMock,
- ozoneManagerProtocol);
+ ozoneManagerProtocol, reconContext);
assertNull(reconOMMetadataManager.getKeyTable(getBucketLayout())
.get("/sampleVol/bucketOne/key_one"));
@@ -145,6 +148,81 @@ public void testUpdateReconOmDBWithNewSnapshot(
assertTrue(ozoneManagerServiceProvider.updateReconOmDBWithNewSnapshot());
+ assertNotNull(reconOMMetadataManager.getKeyTable(getBucketLayout())
+ .get("/sampleVol/bucketOne/key_one"));
+ assertNotNull(reconOMMetadataManager.getKeyTable(getBucketLayout())
+ .get("/sampleVol/bucketOne/key_two"));
+
+ // Verifying if context error GET_OM_DB_SNAPSHOT_FAILED is removed
+ assertFalse(reconContext.getErrors().contains(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED));
+ }
+
+ @Test
+ public void testUpdateReconOmDBWithNewSnapshotFailure(
+ @TempDir File dirOmMetadata, @TempDir File dirReconMetadata)
+ throws Exception {
+
+ OMMetadataManager omMetadataManager =
+ initializeNewOmMetadataManager(dirOmMetadata);
+ ReconOMMetadataManager reconOMMetadataManager =
+ getTestReconOmMetadataManager(omMetadataManager,
+ dirReconMetadata);
+
+ ReconUtils reconUtilsMock = getMockReconUtils();
+
+ when(reconUtilsMock.makeHttpCall(any(), anyString(), anyBoolean()))
+ .thenThrow(new IOException("Mocked IOException"));
+ when(reconUtilsMock.getReconNodeDetails(
+ any(OzoneConfiguration.class))).thenReturn(
+ commonUtils.getReconNodeDetails());
+ ReconTaskController reconTaskController = getMockTaskController();
+
+ OzoneManagerServiceProviderImpl ozoneManagerServiceProvider =
+ new OzoneManagerServiceProviderImpl(configuration,
+ reconOMMetadataManager, reconTaskController, reconUtilsMock,
+ ozoneManagerProtocol, reconContext);
+
+ assertFalse(ozoneManagerServiceProvider.updateReconOmDBWithNewSnapshot());
+
+ // Verifying if context error GET_OM_DB_SNAPSHOT_FAILED is added
+ assertTrue(reconContext.getErrors().contains(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED));
+ }
+
+ @Test
+ public void testUpdateReconOmDBWithNewSnapshotSuccess(
+ @TempDir File dirOmMetadata, @TempDir File dirReconMetadata) throws Exception {
+
+ OMMetadataManager omMetadataManager =
+ initializeNewOmMetadataManager(dirOmMetadata);
+ ReconOMMetadataManager reconOMMetadataManager =
+ getTestReconOmMetadataManager(omMetadataManager, dirReconMetadata);
+
+ writeDataToOm(omMetadataManager, "key_one");
+ writeDataToOm(omMetadataManager, "key_two");
+
+ DBCheckpoint checkpoint = omMetadataManager.getStore().getCheckpoint(true);
+ File tarFile = createTarFile(checkpoint.getCheckpointLocation());
+ InputStream inputStream = new FileInputStream(tarFile);
+ ReconUtils reconUtilsMock = getMockReconUtils();
+ HttpURLConnection httpURLConnectionMock = mock(HttpURLConnection.class);
+ when(httpURLConnectionMock.getInputStream()).thenReturn(inputStream);
+ when(reconUtilsMock.makeHttpCall(any(), anyString(), anyBoolean()))
+ .thenReturn(httpURLConnectionMock);
+ when(reconUtilsMock.getReconNodeDetails(any(OzoneConfiguration.class)))
+ .thenReturn(commonUtils.getReconNodeDetails());
+ ReconTaskController reconTaskController = getMockTaskController();
+
+ reconContext.updateErrors(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED);
+
+ OzoneManagerServiceProviderImpl ozoneManagerServiceProvider =
+ new OzoneManagerServiceProviderImpl(configuration,
+ reconOMMetadataManager, reconTaskController, reconUtilsMock,
+ ozoneManagerProtocol, reconContext);
+
+ assertTrue(reconContext.getErrors().contains(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED));
+ assertTrue(ozoneManagerServiceProvider.updateReconOmDBWithNewSnapshot());
+ assertFalse(reconContext.getErrors().contains(ReconContext.ErrorCode.GET_OM_DB_SNAPSHOT_FAILED));
+
assertNotNull(reconOMMetadataManager.getKeyTable(getBucketLayout())
.get("/sampleVol/bucketOne/key_one"));
assertNotNull(reconOMMetadataManager.getKeyTable(getBucketLayout())
@@ -182,7 +260,7 @@ public void testReconOmDBCloseAndOpenNewSnapshotDb(
OzoneManagerServiceProviderImpl ozoneManagerServiceProvider1 =
new OzoneManagerServiceProviderImpl(configuration,
reconOMMetadataManager, reconTaskController, reconUtilsMock,
- ozoneManagerProtocol);
+ ozoneManagerProtocol, reconContext);
assertTrue(ozoneManagerServiceProvider1.updateReconOmDBWithNewSnapshot());
HttpURLConnection httpURLConnectionMock2 = mock(HttpURLConnection.class);
@@ -192,7 +270,7 @@ public void testReconOmDBCloseAndOpenNewSnapshotDb(
OzoneManagerServiceProviderImpl ozoneManagerServiceProvider2 =
new OzoneManagerServiceProviderImpl(configuration,
reconOMMetadataManager, reconTaskController, reconUtilsMock,
- ozoneManagerProtocol);
+ ozoneManagerProtocol, reconContext);
assertTrue(ozoneManagerServiceProvider2.updateReconOmDBWithNewSnapshot());
}
@@ -238,7 +316,7 @@ public void testGetOzoneManagerDBSnapshot(@TempDir File dirReconMetadata)
OzoneManagerServiceProviderImpl ozoneManagerServiceProvider =
new OzoneManagerServiceProviderImpl(configuration,
reconOMMetadataManager, reconTaskController, reconUtilsMock,
- ozoneManagerProtocol);
+ ozoneManagerProtocol, reconContext);
DBCheckpoint checkpoint = ozoneManagerServiceProvider
.getOzoneManagerDBSnapshot();
@@ -288,7 +366,7 @@ public void testGetAndApplyDeltaUpdatesFromOM(
new OzoneManagerServiceProviderImpl(configuration,
getTestReconOmMetadataManager(omMetadataManager, dirReconMetadata),
getMockTaskController(), new ReconUtils(),
- getMockOzoneManagerClient(dbUpdatesWrapper));
+ getMockOzoneManagerClient(dbUpdatesWrapper), reconContext);
OMDBUpdatesHandler updatesHandler =
new OMDBUpdatesHandler(omMetadataManager);
@@ -358,7 +436,7 @@ public void testGetAndApplyDeltaUpdatesFromOMWithLimit(
getTestReconOmMetadataManager(omMetadataManager, dirReconMetadata),
getMockTaskController(), new ReconUtils(),
getMockOzoneManagerClientWith4Updates(dbUpdatesWrapper[0],
- dbUpdatesWrapper[1], dbUpdatesWrapper[2], dbUpdatesWrapper[3]));
+ dbUpdatesWrapper[1], dbUpdatesWrapper[2], dbUpdatesWrapper[3]), reconContext);
assertTrue(dbUpdatesWrapper[0].isDBUpdateSuccess());
assertTrue(dbUpdatesWrapper[1].isDBUpdateSuccess());
@@ -414,7 +492,7 @@ public void testSyncDataFromOMFullSnapshot(
OzoneManagerServiceProviderImpl ozoneManagerServiceProvider =
new MockOzoneServiceProvider(configuration, omMetadataManager,
- reconTaskControllerMock, new ReconUtils(), ozoneManagerProtocol);
+ reconTaskControllerMock, new ReconUtils(), ozoneManagerProtocol, reconContext);
OzoneManagerSyncMetrics metrics = ozoneManagerServiceProvider.getMetrics();
assertEquals(0, metrics.getNumSnapshotRequests());
@@ -454,7 +532,7 @@ public void testSyncDataFromOMDeltaUpdates(
OzoneManagerServiceProviderImpl ozoneManagerServiceProvider =
new OzoneManagerServiceProviderImpl(configuration, omMetadataManager,
- reconTaskControllerMock, new ReconUtils(), ozoneManagerProtocol);
+ reconTaskControllerMock, new ReconUtils(), ozoneManagerProtocol, reconContext);
OzoneManagerSyncMetrics metrics = ozoneManagerServiceProvider.getMetrics();
@@ -495,7 +573,7 @@ public void testSyncDataFromOMFullSnapshotForSNNFE(
OzoneManagerProtocol protocol = getMockOzoneManagerClientWithThrow();
OzoneManagerServiceProviderImpl ozoneManagerServiceProvider =
new MockOzoneServiceProvider(configuration, omMetadataManager,
- reconTaskControllerMock, new ReconUtils(), protocol);
+ reconTaskControllerMock, new ReconUtils(), protocol, reconContext);
OzoneManagerSyncMetrics metrics = ozoneManagerServiceProvider.getMetrics();
@@ -569,9 +647,10 @@ class MockOzoneServiceProvider extends OzoneManagerServiceProviderImpl {
ReconOMMetadataManager omMetadataManager,
ReconTaskController reconTaskController,
ReconUtils reconUtils,
- OzoneManagerProtocol ozoneManagerClient) {
+ OzoneManagerProtocol ozoneManagerClient,
+ ReconContext reconContext) {
super(configuration, omMetadataManager, reconTaskController, reconUtils,
- ozoneManagerClient);
+ ozoneManagerClient, reconContext);
}
@Override
diff --git a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/Gateway.java b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/Gateway.java
index cfbcb51d266..86d25d19417 100644
--- a/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/Gateway.java
+++ b/hadoop-ozone/s3gateway/src/main/java/org/apache/hadoop/ozone/s3/Gateway.java
@@ -38,7 +38,6 @@
import org.slf4j.LoggerFactory;
import picocli.CommandLine.Command;
-import static org.apache.hadoop.hdds.StringUtils.startupShutdownMessage;
import static org.apache.hadoop.hdds.ratis.RatisHelper.newJvmPauseMonitor;
import static org.apache.hadoop.hdds.server.http.HttpServer2.setHttpBaseDir;
import static org.apache.hadoop.ozone.conf.OzoneServiceConfig.DEFAULT_SHUTDOWN_HOOK_PRIORITY;
@@ -95,7 +94,7 @@ public Void call() throws Exception {
public void start() throws IOException {
String[] originalArgs = getCmd().getParseResult().originalArgs()
.toArray(new String[0]);
- startupShutdownMessage(OzoneVersionInfo.OZONE_VERSION_INFO,
+ HddsServerUtil.startupShutdownMessage(OzoneVersionInfo.OZONE_VERSION_INFO,
Gateway.class, originalArgs, LOG, ozoneConfiguration);
LOG.info("Starting Ozone S3 gateway");
diff --git a/pom.xml b/pom.xml
index 5c4799dbe89..ec059503dcc 100644
--- a/pom.xml
+++ b/pom.xml
@@ -75,10 +75,10 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
${hdds.version}
- 3.0.1
+ 3.1.0
- 1.0.5
+ 1.0.6
2.3.0
@@ -100,8 +100,9 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
false
false
true
- 9.4.54.v20240208
+ 9.4.55.v20240627
5.2.0
+ 1.0-1
4.2.0
@@ -121,7 +122,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
1.1
3.6.1
3.10.0
- 1.11.0
+ 1.12.0
1.6
1.5
1.7.1
@@ -144,6 +145,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
1.2.2
2.3.3
2.3.9
+ 3.1.19
0.1.55
2.0
3.1.0
@@ -210,7 +212,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
3.6.0
4.11.0
2.2
- 5.10.2
+ 5.10.3
3.8.4
@@ -228,7 +230,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
3.1.9.Final
- 1.8
+ 8
- -Xmx4096m -XX:+HeapDumpOnOutOfMemoryError
+ -Xmx8192m -XX:+HeapDumpOnOutOfMemoryError
native | unhealthy
flaky | native | slow | unhealthy
@@ -251,24 +253,24 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
3.1.2
3.1.0
3.6.0
- 3.4.1
+ 3.4.2
3.4.0
3.3.1
1.6.1
1.7.0
3.5.0
- 3.0.1
+ 3.7.0
3.7.1
0.16.1
3.1.2
- 3.5.0
- 3.6.1
+ 3.6.0
+ 3.7.1
4.2.2
0.44.0
3.1.1
- 2.3.0
+ 2.4.0
1.0-beta-1
- 1.0-alpha-11
+ 1.0-M1
3.4.0
3.12.1
3.1.0
@@ -302,7 +304,7 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
3.9.6
1.1.10.5
1.2.0
- 9.37.2
+ 9.40
@@ -478,11 +480,21 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
commons-validator
${commons-validator.version}
+
+ com.github.jnr
+ jnr-posix
+ ${jnr-posix.version}
+
com.github.luben
zstd-jni
${zstd-jni.version}
+
+ com.github.stephenc.jcip
+ jcip-annotations
+ ${jcip-annotations.version}
+
javax.annotation
javax.annotation-api
@@ -1310,8 +1322,6 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
maven-compiler-plugin
${maven-compiler-plugin.version}
- ${javac.version}
- ${javac.version}
false
@@ -1956,6 +1966,28 @@ xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xs
+
+
+
+ java8
+
+ [,8]
+
+
+ ${javac.version}
+ ${javac.version}
+
+
+
+ java9-or-later
+
+ [9,]
+
+
+ ${javac.version}
+
+
+
go-offline