Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -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.container.common.utils;

import java.util.ArrayDeque;
import java.util.Deque;
import java.util.concurrent.TimeUnit;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* A time-based sliding window implementation that tracks only failed test results within a specified time duration.
* It determines failure based on a configured tolerance threshold.
*
* The queue saves one failure more than the configured tolerance threshold,
* so that the window can be considered failed.
*/
public class SlidingWindow {
private static final Logger LOG = LoggerFactory.getLogger(SlidingWindow.class);

private final long windowDuration;
private final TimeUnit timeUnit;
private final int failureTolerance;
private final Deque<Long> failureTimestamps;

/**
* @param failureTolerance the number of failures that can be tolerated before the window is considered failed
* @param windowDuration the duration of the sliding window
* @param timeUnit the time unit of the window duration
*/
public SlidingWindow(int failureTolerance, long windowDuration, TimeUnit timeUnit) {
this.windowDuration = windowDuration;
this.timeUnit = timeUnit;
this.failureTolerance = failureTolerance;
// If the failure tolerance is high, we limit the queue size to 100 as we want to control the memory usage
this.failureTimestamps = new ArrayDeque<>(Math.min(failureTolerance + 1, 100));
}

public synchronized void add(boolean result) {
LOG.debug("Received test result: {}", result);
if (!result) {
if (failureTolerance > 0 && failureTimestamps.size() > failureTolerance) {
failureTimestamps.remove();
}
long currentTime = System.currentTimeMillis();
failureTimestamps.addLast(currentTime);
}

removeExpiredFailures();
}

public synchronized boolean isFailed() {
removeExpiredFailures();
LOG.debug("Is failed: {} {}", failureTimestamps.size() > failureTolerance, failureTimestamps);
return failureTimestamps.size() > failureTolerance;
}

private void removeExpiredFailures() {
long currentTime = System.currentTimeMillis();
long expirationThreshold = currentTime - timeUnit.toMillis(windowDuration);

while (!failureTimestamps.isEmpty() && failureTimestamps.peek() < expirationThreshold) {
LOG.debug("Removing expired failure timestamp: {}", failureTimestamps.peek());
failureTimestamps.remove();
}

LOG.debug("Current failure count: {}", failureTimestamps.size());
}

public int getFailureTolerance() {
return failureTolerance;
}

public long getWindowDuration() {
return windowDuration;
}

public TimeUnit getTimeUnit() {
return timeUnit;
}

public int getFailureCount() {
return failureTimestamps.size();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -25,20 +25,15 @@
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;
Expand Down Expand Up @@ -102,11 +97,6 @@ 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<Boolean> volumeTestResultQueue;

/**
* Builder for HddsVolume.
*/
Expand Down Expand Up @@ -139,20 +129,13 @@ 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
// HddsVolume Object.
this.setState(VolumeState.FAILED);
volumeIOStats = null;
volumeInfoMetrics = new VolumeInfoMetrics(b.getVolumeRootStr(), this);
this.volumeTestCount = 0;
this.volumeTestFailureTolerance = 0;
}

LOG.info("HddsVolume: {}", getReport());
Expand Down Expand Up @@ -309,43 +292,6 @@ public synchronized VolumeCheckResult check(@Nullable Boolean unused)
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;
}

/**
* add "delta" bytes to committed space in the volume.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,15 +17,23 @@

package org.apache.hadoop.ozone.container.common.volume;

import static org.apache.hadoop.ozone.OzoneConsts.WITNESSED_CONTAINER_DB_NAME;

import jakarta.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import org.apache.hadoop.hdds.server.ServerUtils;
import org.apache.hadoop.hdfs.server.datanode.checker.VolumeCheckResult;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* MetadataVolume represents a volume in datanode for metadata(ratis).
* Datanode itself doesn't consume this volume, but only manages checks
* and volume info for it.
*/
public class MetadataVolume extends StorageVolume {

private static final Logger LOG = LoggerFactory.getLogger(MetadataVolume.class);
private final VolumeType type = VolumeType.META_VOLUME;

protected MetadataVolume(Builder b) throws IOException {
Expand Down Expand Up @@ -75,4 +83,23 @@ public MetadataVolume build() throws IOException {
public String getStorageID() {
return "";
}

@Override
public synchronized VolumeCheckResult check(@Nullable Boolean unused) throws Exception {
VolumeCheckResult result = super.check(unused);

if (result != VolumeCheckResult.HEALTHY) {
LOG.error("Volume failed health check.");
return result;
}

// Check that per-volume RocksDB is present.
File dbFile = new File(ServerUtils.getOzoneMetaDirPath(getConf()), WITNESSED_CONTAINER_DB_NAME);
if (!dbFile.exists() || !dbFile.canRead()) {
LOG.warn("Volume {} failed health check. Could not access RocksDB at {}", getStorageDir(), dbFile);
return VolumeCheckResult.FAILED;
}

return checkDbHealth(dbFile);
}
}
Loading