diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java index 55bd55021093..cc9be3892bed 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java @@ -25,15 +25,20 @@ import jakarta.annotation.Nullable; import java.io.File; import java.io.IOException; +import java.util.LinkedList; import java.util.List; +import java.util.Queue; import java.util.concurrent.ThreadLocalRandom; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import org.apache.commons.io.FileUtils; import org.apache.hadoop.hdds.annotation.InterfaceAudience; import org.apache.hadoop.hdds.annotation.InterfaceStability; import org.apache.hadoop.hdds.conf.ConfigurationSource; import org.apache.hadoop.hdds.upgrade.HDDSLayoutFeature; +import org.apache.hadoop.hdds.utils.db.managed.ManagedOptions; +import org.apache.hadoop.hdds.utils.db.managed.ManagedRocksDB; import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult; import org.apache.hadoop.ozone.container.common.impl.StorageLocationReport; import org.apache.hadoop.ozone.container.common.utils.DatanodeStoreCache; @@ -97,6 +102,11 @@ public class HddsVolume extends StorageVolume { private AtomicBoolean dbLoaded = new AtomicBoolean(false); private final AtomicBoolean dbLoadFailure = new AtomicBoolean(false); + private final int volumeTestCount; + private final int volumeTestFailureTolerance; + private AtomicInteger volumeTestFailureCount; + private Queue volumeTestResultQueue; + /** * Builder for HddsVolume. */ @@ -129,6 +139,11 @@ private HddsVolume(Builder b) throws IOException { this.volumeInfoMetrics = new VolumeInfoMetrics(b.getVolumeRootStr(), this); + this.volumeTestCount = getDatanodeConfig().getVolumeIOTestCount(); + this.volumeTestFailureTolerance = getDatanodeConfig().getVolumeIOFailureTolerance(); + this.volumeTestFailureCount = new AtomicInteger(0); + this.volumeTestResultQueue = new LinkedList<>(); + initialize(); } else { // Builder is called with failedVolume set, so create a failed volume @@ -136,6 +151,8 @@ private HddsVolume(Builder b) throws IOException { this.setState(VolumeState.FAILED); volumeIOStats = null; volumeInfoMetrics = new VolumeInfoMetrics(b.getVolumeRootStr(), this); + this.volumeTestCount = 0; + this.volumeTestFailureTolerance = 0; } LOG.info("HddsVolume: {}", getReport()); @@ -275,6 +292,7 @@ public synchronized VolumeCheckResult check(@Nullable Boolean unused) "the volume might not have been loaded properly.", getStorageDir()); return VolumeCheckResult.FAILED; } + if (result != VolumeCheckResult.HEALTHY || !getDatanodeConfig().getContainerSchemaV3Enabled() || !isDbLoaded()) { return result; @@ -288,6 +306,43 @@ public synchronized VolumeCheckResult check(@Nullable Boolean unused) return VolumeCheckResult.FAILED; } + return checkDbHealth(dbFile); + } + + @VisibleForTesting + public VolumeCheckResult checkDbHealth(File dbFile) throws InterruptedException { + if (volumeTestCount == 0) { + return VolumeCheckResult.HEALTHY; + } + + final boolean isVolumeTestResultHealthy = true; + try (ManagedOptions managedOptions = new ManagedOptions(); + ManagedRocksDB readOnlyDb = ManagedRocksDB.openReadOnly(managedOptions, dbFile.toString())) { + volumeTestResultQueue.add(isVolumeTestResultHealthy); + } catch (Exception e) { + if (Thread.currentThread().isInterrupted()) { + throw new InterruptedException("Check of database for volume " + this + " interrupted."); + } + LOG.warn("Could not open Volume DB located at {}", dbFile, e); + volumeTestResultQueue.add(!isVolumeTestResultHealthy); + volumeTestFailureCount.incrementAndGet(); + } + + if (volumeTestResultQueue.size() > volumeTestCount + && (Boolean.TRUE.equals(volumeTestResultQueue.poll()) != isVolumeTestResultHealthy)) { + volumeTestFailureCount.decrementAndGet(); + } + + if (volumeTestFailureCount.get() > volumeTestFailureTolerance) { + LOG.error("Failed to open the database at \"{}\" for HDDS volume {}: " + + "the last {} runs encountered {} out of {} tolerated failures.", + dbFile, this, volumeTestResultQueue.size(), volumeTestFailureCount.get(), volumeTestFailureTolerance); + return VolumeCheckResult.FAILED; + } + + LOG.debug("Successfully opened the database at \"{}\" for HDDS volume {}: " + + "the last {} runs encountered {} out of {} tolerated failures", + dbFile, this, volumeTestResultQueue.size(), volumeTestFailureTolerance, volumeTestFailureTolerance); return VolumeCheckResult.HEALTHY; } diff --git a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolume.java b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolume.java index fe0fa896921b..2c02c02f59e2 100644 --- a/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolume.java +++ b/hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolume.java @@ -693,7 +693,7 @@ public synchronized VolumeCheckResult check(@Nullable Boolean unused) // Once the volume is failed, it will not be checked anymore. // The failure counts can be left as is. if (currentIOFailureCount.get() > ioFailureTolerance) { - LOG.info("Failed IO test for volume {}: the last {} runs " + + LOG.error("Failed IO test for volume {}: the last {} runs " + "encountered {} out of {} tolerated failures.", this, ioTestSlidingWindow.size(), currentIOFailureCount, ioFailureTolerance); diff --git a/hadoop-hdds/managed-rocksdb/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedRocksDB.java b/hadoop-hdds/managed-rocksdb/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedRocksDB.java index cf16ac5fc99c..bd8f86a6efbb 100644 --- a/hadoop-hdds/managed-rocksdb/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedRocksDB.java +++ b/hadoop-hdds/managed-rocksdb/src/main/java/org/apache/hadoop/hdds/utils/db/managed/ManagedRocksDB.java @@ -46,6 +46,13 @@ public class ManagedRocksDB extends ManagedObject { super(original); } + public static ManagedRocksDB openReadOnly( + final ManagedOptions options, + final String path) + throws RocksDBException { + return new ManagedRocksDB(RocksDB.openReadOnly(options, path)); + } + public static ManagedRocksDB openReadOnly( final ManagedDBOptions options, final String path, final List columnFamilyDescriptors, diff --git a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/dn/volume/TestDatanodeHddsVolumeFailureDetection.java b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/dn/volume/TestDatanodeHddsVolumeFailureDetection.java index fdea9054ce7a..af6e5b490974 100644 --- a/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/dn/volume/TestDatanodeHddsVolumeFailureDetection.java +++ b/hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/dn/volume/TestDatanodeHddsVolumeFailureDetection.java @@ -46,6 +46,7 @@ import org.apache.hadoop.hdds.scm.cli.ContainerOperationClient; import org.apache.hadoop.hdds.scm.client.ScmClient; import org.apache.hadoop.hdds.scm.container.common.helpers.ContainerWithPipeline; +import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult; import org.apache.hadoop.ozone.HddsDatanodeService; import org.apache.hadoop.ozone.MiniOzoneCluster; import org.apache.hadoop.ozone.OzoneConsts; @@ -73,7 +74,6 @@ */ @Timeout(300) class TestDatanodeHddsVolumeFailureDetection { - private static final int KEY_SIZE = 128; @ParameterizedTest @@ -241,6 +241,74 @@ void corruptDbFile(boolean schemaV3) throws Exception { } } + /** + * {@link HddsVolume#check(Boolean)} will capture the failures injected by this test and not allow the + * test to reach the helper method {@link HddsVolume#checkDbHealth}. + * As a workaround, we test the helper method directly. + * As we test the helper method directly, we cannot test for schemas older than V3. + * + * @param schemaV3 + * @throws Exception + */ + @ParameterizedTest + @ValueSource(booleans = {true}) + void corruptDbFileWithoutDbHandleCacheInvalidation(boolean schemaV3) throws Exception { + try (MiniOzoneCluster cluster = newCluster(schemaV3)) { + try (OzoneClient client = cluster.newClient()) { + OzoneBucket bucket = TestDataUtil.createVolumeAndBucket(client); + + // write a file, will create container1 + String keyName = UUID.randomUUID().toString(); + long containerId = createKey(bucket, keyName); + + // close container1 + HddsDatanodeService dn = cluster.getHddsDatanodes().get(0); + OzoneContainer oc = dn.getDatanodeStateMachine().getContainer(); + Container c1 = oc.getContainerSet().getContainer(containerId); + c1.close(); + + // create container2, and container1 is kicked out of cache + OzoneConfiguration conf = cluster.getConf(); + try (ScmClient scmClient = new ContainerOperationClient(conf)) { + ContainerWithPipeline c2 = scmClient.createContainer( + ReplicationType.STAND_ALONE, ReplicationFactor.ONE, + OzoneConsts.OZONE); + assertEquals(c2.getContainerInfo().getState(), LifeCycleState.OPEN); + } + + // corrupt db by rename dir->file + File dbDir; + if (schemaV3) { + dbDir = new File(((KeyValueContainerData) (c1.getContainerData())) + .getDbFile().getAbsolutePath()); + } else { + File metadataDir = new File(c1.getContainerFile().getParent()); + dbDir = new File(metadataDir, "1" + OzoneConsts.DN_CONTAINER_DB); + } + + MutableVolumeSet volSet = oc.getVolumeSet(); + HddsVolume vol0 = (HddsVolume) volSet.getVolumesList().get(0); + + try { + DatanodeTestUtils.injectDataDirFailure(dbDir); + // simulate bad volume by removing write permission on root dir + // refer to HddsVolume.check() + DatanodeTestUtils.simulateBadVolume(vol0); + + // one volume health check got automatically executed when the cluster started + // the second health should log the rocksdb failure but return a healthy-volume status + assertEquals(VolumeCheckResult.HEALTHY, vol0.checkDbHealth(dbDir)); + // the third health check should log the rocksdb failure and return a failed-volume status + assertEquals(VolumeCheckResult.FAILED, vol0.checkDbHealth(dbDir)); + } finally { + // restore all + DatanodeTestUtils.restoreBadVolume(vol0); + DatanodeTestUtils.restoreDataDirFromFailure(dbDir); + } + } + } + } + private static void readKeyToTriggerCheckVolumeAsync(OzoneBucket bucket, String key) throws IOException { try (InputStream is = bucket.readKey(key)) {