errose28 commented on code in PR #8256: URL: https://github.com/apache/ozone/pull/8256#discussion_r2049610694
########## hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java: ########## @@ -288,6 +306,40 @@ public synchronized VolumeCheckResult check(@Nullable Boolean unused) return VolumeCheckResult.FAILED; } + return checkDbHealth(dbFile); + } + + @VisibleForTesting + public VolumeCheckResult checkDbHealth(File dbFile) { + if (volumeTestCount == 0) { + return VolumeCheckResult.HEALTHY; + } + + boolean isVolumeTestResultHealthy = true; Review Comment: We should either make this `final` or just add `true` and `false` literally to the queue. If this value gets changed for some reason the checks here will break. ########## hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java: ########## @@ -288,6 +306,40 @@ public synchronized VolumeCheckResult check(@Nullable Boolean unused) return VolumeCheckResult.FAILED; } + return checkDbHealth(dbFile); + } + + @VisibleForTesting + public VolumeCheckResult checkDbHealth(File dbFile) { + if (volumeTestCount == 0) { + return VolumeCheckResult.HEALTHY; + } + + boolean isVolumeTestResultHealthy = true; + try (ManagedOptions managedOptions = new ManagedOptions(); + ManagedRocksDB readOnlyDb = ManagedRocksDB.openReadOnly(managedOptions, dbFile.toString())) { + volumeTestResultQueue.add(isVolumeTestResultHealthy); + } catch (Exception e) { + LOG.warn("Could not open Volume DB located at {}", dbFile, e); + volumeTestResultQueue.add(!isVolumeTestResultHealthy); + volumeTestFailureCount.incrementAndGet(); + } + + if (volumeTestResultQueue.size() > volumeTestCount) { + if (!volumeTestResultQueue.isEmpty() && + volumeTestResultQueue.poll() != isVolumeTestResultHealthy) { + volumeTestFailureCount.decrementAndGet(); + } + } + + if (volumeTestFailureCount.get() > volumeTestFailureTolerance) { + LOG.error("Failed volume test for volume {}: the last {} runs encountered {} out of {} tolerated failures.", Review Comment: Let's specify in this log message that this check is specifically for DB open errors in an HDDS volume. We can log the path to the DB too. Also let's use WARN for the log level to be consistent with the other volume failure messages. I'm not sure why the parent class logs at INFO for the same type of message, we can fix that in this change too. ########## hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java: ########## @@ -288,6 +306,40 @@ public synchronized VolumeCheckResult check(@Nullable Boolean unused) return VolumeCheckResult.FAILED; } + return checkDbHealth(dbFile); + } + + @VisibleForTesting + public VolumeCheckResult checkDbHealth(File dbFile) { + if (volumeTestCount == 0) { + return VolumeCheckResult.HEALTHY; + } + + boolean isVolumeTestResultHealthy = true; + try (ManagedOptions managedOptions = new ManagedOptions(); + ManagedRocksDB readOnlyDb = ManagedRocksDB.openReadOnly(managedOptions, dbFile.toString())) { + volumeTestResultQueue.add(isVolumeTestResultHealthy); + } catch (Exception e) { + LOG.warn("Could not open Volume DB located at {}", dbFile, e); + volumeTestResultQueue.add(!isVolumeTestResultHealthy); + volumeTestFailureCount.incrementAndGet(); + } + + if (volumeTestResultQueue.size() > volumeTestCount) { + if (!volumeTestResultQueue.isEmpty() && + volumeTestResultQueue.poll() != isVolumeTestResultHealthy) { + volumeTestFailureCount.decrementAndGet(); + } + } + + if (volumeTestFailureCount.get() > volumeTestFailureTolerance) { + LOG.error("Failed volume test for volume {}: the last {} runs encountered {} out of {} tolerated failures.", + this, volumeTestResultQueue.size(), volumeTestFailureCount.get(), volumeTestFailureTolerance); + return VolumeCheckResult.FAILED; + } + + LOG.info("IO test results for volume {}: the last {} runs encountered {} out of {} tolerated failures", Review Comment: Also like the previous message let's specify that this pertains to the DB check on an HDDS volume. ########## hadoop-ozone/integration-test/src/test/java/org/apache/hadoop/ozone/dn/volume/TestDatanodeHddsVolumeFailureDetection.java: ########## @@ -241,6 +241,65 @@ void corruptDbFile(boolean schemaV3) throws Exception { } } + @ParameterizedTest + @ValueSource(booleans = {true}) + void corruptDbFileWithoutDbHandleCacheInvalidation(boolean schemaV3) throws Exception { Review Comment: This is almost identical to `corruptDbFile` except what is being checked at the very end. Can we consolidate the tests? ########## hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java: ########## @@ -288,6 +306,40 @@ public synchronized VolumeCheckResult check(@Nullable Boolean unused) return VolumeCheckResult.FAILED; } + return checkDbHealth(dbFile); + } + + @VisibleForTesting + public VolumeCheckResult checkDbHealth(File dbFile) { + if (volumeTestCount == 0) { + return VolumeCheckResult.HEALTHY; + } + + boolean isVolumeTestResultHealthy = true; + try (ManagedOptions managedOptions = new ManagedOptions(); + ManagedRocksDB readOnlyDb = ManagedRocksDB.openReadOnly(managedOptions, dbFile.toString())) { + volumeTestResultQueue.add(isVolumeTestResultHealthy); + } catch (Exception e) { + LOG.warn("Could not open Volume DB located at {}", dbFile, e); + volumeTestResultQueue.add(!isVolumeTestResultHealthy); + volumeTestFailureCount.incrementAndGet(); + } + + if (volumeTestResultQueue.size() > volumeTestCount) { + if (!volumeTestResultQueue.isEmpty() && Review Comment: Technically we don't need the `isEmpty` check here because at the top of the method we've made sure `volumeTestCount != 0` so the size check above already handles this case. This is how the check in the parent class is done. ########## hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java: ########## @@ -288,6 +306,40 @@ public synchronized VolumeCheckResult check(@Nullable Boolean unused) return VolumeCheckResult.FAILED; } + return checkDbHealth(dbFile); + } + + @VisibleForTesting + public VolumeCheckResult checkDbHealth(File dbFile) { + if (volumeTestCount == 0) { + return VolumeCheckResult.HEALTHY; + } + + boolean isVolumeTestResultHealthy = true; + try (ManagedOptions managedOptions = new ManagedOptions(); + ManagedRocksDB readOnlyDb = ManagedRocksDB.openReadOnly(managedOptions, dbFile.toString())) { + volumeTestResultQueue.add(isVolumeTestResultHealthy); + } catch (Exception e) { + LOG.warn("Could not open Volume DB located at {}", dbFile, e); + volumeTestResultQueue.add(!isVolumeTestResultHealthy); + volumeTestFailureCount.incrementAndGet(); + } + + if (volumeTestResultQueue.size() > volumeTestCount) { + if (!volumeTestResultQueue.isEmpty() && + volumeTestResultQueue.poll() != isVolumeTestResultHealthy) { + volumeTestFailureCount.decrementAndGet(); + } + } + + if (volumeTestFailureCount.get() > volumeTestFailureTolerance) { + LOG.error("Failed volume test for volume {}: the last {} runs encountered {} out of {} tolerated failures.", + this, volumeTestResultQueue.size(), volumeTestFailureCount.get(), volumeTestFailureTolerance); + return VolumeCheckResult.FAILED; + } + + LOG.info("IO test results for volume {}: the last {} runs encountered {} out of {} tolerated failures", Review Comment: Let's keep the parent class behavior where this type of message is only logged at debug level. No need to fill the logs with info about healthy tests since that info is not actionable. If this information is needed we can easily switch the log4j config to debug for this class and we will start seeing this as the volume scanner is triggered again. ########## hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/HddsVolume.java: ########## @@ -288,6 +306,40 @@ public synchronized VolumeCheckResult check(@Nullable Boolean unused) return VolumeCheckResult.FAILED; } + return checkDbHealth(dbFile); + } + + @VisibleForTesting + public VolumeCheckResult checkDbHealth(File dbFile) { + if (volumeTestCount == 0) { + return VolumeCheckResult.HEALTHY; + } + + boolean isVolumeTestResultHealthy = true; + try (ManagedOptions managedOptions = new ManagedOptions(); + ManagedRocksDB readOnlyDb = ManagedRocksDB.openReadOnly(managedOptions, dbFile.toString())) { + volumeTestResultQueue.add(isVolumeTestResultHealthy); + } catch (Exception e) { Review Comment: Like the parent class, let's check the interrupt flag and throw `InterruptedException` if it is set. This prevents any sort of thread interrupt during IO from incorrectly reporting a failure. -- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. To unsubscribe, e-mail: issues-unsubscr...@ozone.apache.org For queries about this service, please contact Infrastructure at: us...@infra.apache.org --------------------------------------------------------------------- To unsubscribe, e-mail: issues-unsubscr...@ozone.apache.org For additional commands, e-mail: issues-h...@ozone.apache.org