Skip to content

Commit e0d64d7

Browse files
committed
HDDS-11943. Fail storage volume after numerous reported IO errors.
1 parent 61a36f6 commit e0d64d7

File tree

6 files changed

+100
-17
lines changed

6 files changed

+100
-17
lines changed

hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/impl/HddsDispatcher.java

Lines changed: 3 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -103,6 +103,7 @@ public class HddsDispatcher implements ContainerDispatcher, Auditor {
103103
private final Map<ContainerType, Handler> handlers;
104104
private final ConfigurationSource conf;
105105
private final ContainerSet containerSet;
106+
private final VolumeSet volumeSet;
106107
private final StateContext context;
107108
private final float containerCloseThreshold;
108109
private final ProtocolMessageMetrics<ProtocolMessageEnum> protocolMetrics;
@@ -127,6 +128,7 @@ public HddsDispatcher(ConfigurationSource config, ContainerSet contSet,
127128
this.context = context;
128129
this.handlers = handlers;
129130
this.metrics = metrics;
131+
this.volumeSet = volumes;
130132
this.containerCloseThreshold = conf.getFloat(
131133
HddsConfigKeys.HDDS_CONTAINER_CLOSE_THRESHOLD,
132134
HddsConfigKeys.HDDS_CONTAINER_CLOSE_THRESHOLD_DEFAULT);
@@ -424,7 +426,7 @@ && getMissingContainerSet().contains(containerID)) {
424426
// Create a specific exception that signals for on demand scanning
425427
// and move this general scan to where it is more appropriate.
426428
// Add integration tests to test the full functionality.
427-
OnDemandContainerDataScanner.scanContainer(container);
429+
OnDemandContainerDataScanner.scanContainer(container, volumeSet);
428430
audit(action, eventType, msg, dispatcherContext, AuditEventStatus.FAILURE,
429431
new Exception(responseProto.getMessage()));
430432
}

hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/volume/StorageVolume.java

Lines changed: 6 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -136,6 +136,7 @@ public enum VolumeState {
136136
private AtomicInteger currentIOFailureCount;
137137
private Queue<Boolean> ioTestSlidingWindow;
138138
private int healthCheckFileSize;
139+
private AtomicInteger cumulativeIOFailureCount;
139140

140141
protected StorageVolume(Builder<?> b) throws IOException {
141142
if (!b.failedVolume) {
@@ -158,6 +159,7 @@ protected StorageVolume(Builder<?> b) throws IOException {
158159
this.ioFailureTolerance = dnConf.getVolumeIOFailureTolerance();
159160
this.ioTestSlidingWindow = new LinkedList<>();
160161
this.currentIOFailureCount = new AtomicInteger(0);
162+
this.cumulativeIOFailureCount = new AtomicInteger(0);
161163
this.healthCheckFileSize = dnConf.getVolumeHealthCheckFileSize();
162164
} else {
163165
storageDir = new File(b.volumeRootStr);
@@ -484,6 +486,10 @@ public void decrementUsedSpace(long reclaimedSpace) {
484486
.decrementUsedSpace(reclaimedSpace));
485487
}
486488

489+
public int incrementAndGetCumulativeIOFailureCount() {
490+
return cumulativeIOFailureCount.incrementAndGet();
491+
}
492+
487493
public VolumeSet getVolumeSet() {
488494
return this.volumeSet;
489495
}

hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/ContainerScannerConfiguration.java

Lines changed: 18 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -19,6 +19,7 @@
1919

2020
import static org.apache.hadoop.hdds.conf.ConfigTag.DATANODE;
2121

22+
import com.google.common.annotations.VisibleForTesting;
2223
import java.time.Duration;
2324
import org.apache.hadoop.hdds.conf.Config;
2425
import org.apache.hadoop.hdds.conf.ConfigGroup;
@@ -67,6 +68,7 @@ public class ContainerScannerConfiguration {
6768
public static final long BANDWIDTH_PER_VOLUME_DEFAULT = OzoneConsts.MB * 5L;
6869
public static final long ON_DEMAND_BANDWIDTH_PER_VOLUME_DEFAULT =
6970
OzoneConsts.MB * 5L;
71+
public static final int ON_DEMAND_IO_ERROR_TOLERANCE_DEFAULT = 1000;
7072

7173
@Config(key = "enabled",
7274
type = ConfigType.BOOLEAN,
@@ -137,6 +139,13 @@ public class ContainerScannerConfiguration {
137139
)
138140
private long containerScanMinGap = CONTAINER_SCAN_MIN_GAP_DEFAULT;
139141

142+
@Config(key = "on.demand.io.failure.tolerance.count",
143+
type = ConfigType.INT,
144+
defaultValue = "1000",
145+
tags = {ConfigTag.STORAGE},
146+
description = "Config parameter to set io failure tolerance count for volume")
147+
private int onDemandIOErrorToleranceCount = ON_DEMAND_IO_ERROR_TOLERANCE_DEFAULT;
148+
140149
@PostConstruct
141150
public void validate() {
142151
if (metadataScanInterval < 0) {
@@ -217,4 +226,13 @@ public long getOnDemandBandwidthPerVolume() {
217226
public long getContainerScanMinGap() {
218227
return containerScanMinGap;
219228
}
229+
230+
public int getOnDemandIOErrorToleranceCount() {
231+
return onDemandIOErrorToleranceCount;
232+
}
233+
234+
@VisibleForTesting
235+
public void setOnDemandIOErrorToleranceCount(int onDemandIOErrorToleranceCount) {
236+
this.onDemandIOErrorToleranceCount = onDemandIOErrorToleranceCount;
237+
}
220238
}

hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/ozoneimpl/OnDemandContainerDataScanner.java

Lines changed: 20 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,8 @@
3535
import org.apache.hadoop.ozone.container.common.interfaces.Container;
3636
import org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult;
3737
import org.apache.hadoop.ozone.container.common.volume.HddsVolume;
38+
import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
39+
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
3840
import org.slf4j.Logger;
3941
import org.slf4j.LoggerFactory;
4042

@@ -55,6 +57,7 @@ public final class OnDemandContainerDataScanner {
5557
.KeySetView<Long, Boolean> containerRescheduleCheckSet;
5658
private final OnDemandScannerMetrics metrics;
5759
private final long minScanGap;
60+
private final int onDemandIOErrorToleranceCount;
5861

5962
private OnDemandContainerDataScanner(
6063
ContainerScannerConfiguration conf, ContainerController controller) {
@@ -66,6 +69,7 @@ private OnDemandContainerDataScanner(
6669
scanExecutor = Executors.newSingleThreadExecutor();
6770
containerRescheduleCheckSet = ConcurrentHashMap.newKeySet();
6871
minScanGap = conf.getContainerScanMinGap();
72+
onDemandIOErrorToleranceCount = conf.getOnDemandIOErrorToleranceCount();
6973
}
7074

7175
public static synchronized void init(
@@ -100,7 +104,7 @@ private static boolean shouldScan(Container<?> container) {
100104
LOG) && container.shouldScanData();
101105
}
102106

103-
public static Optional<Future<?>> scanContainer(Container<?> container) {
107+
public static Optional<Future<?>> scanContainer(Container<?> container, VolumeSet volumeSet) {
104108
if (!shouldScan(container)) {
105109
return Optional.empty();
106110
}
@@ -109,7 +113,7 @@ public static Optional<Future<?>> scanContainer(Container<?> container) {
109113
long containerId = container.getContainerData().getContainerID();
110114
if (addContainerToScheduledContainers(containerId)) {
111115
resultFuture = instance.scanExecutor.submit(() -> {
112-
performOnDemandScan(container);
116+
performOnDemandScan(container, volumeSet);
113117
removeContainerFromScheduledContainers(containerId);
114118
});
115119
}
@@ -125,7 +129,7 @@ private static void removeContainerFromScheduledContainers(
125129
instance.containerRescheduleCheckSet.remove(containerId);
126130
}
127131

128-
private static void performOnDemandScan(Container<?> container) {
132+
private static void performOnDemandScan(Container<?> container, VolumeSet volumeSet) {
129133
if (!shouldScan(container)) {
130134
return;
131135
}
@@ -149,6 +153,15 @@ private static void performOnDemandScan(Container<?> container) {
149153
instance.metrics.incNumUnHealthyContainers();
150154
instance.containerController.markContainerUnhealthy(containerId,
151155
result);
156+
if (volumeSet instanceof MutableVolumeSet) {
157+
MutableVolumeSet mutableVolumeSet = (MutableVolumeSet) volumeSet;
158+
HddsVolume hddsVolume = container.getContainerData().getVolume();
159+
if (hddsVolume.incrementAndGetCumulativeIOFailureCount() > getIOErrorToleranceCount()
160+
&& !(getIOErrorToleranceCount() == 0)) {
161+
// If IOErrorToleranceCount is configured as 0, disable making fail volume.
162+
mutableVolumeSet.failVolume(container.getContainerData().getVolume().getVolumeRootDir());
163+
}
164+
}
152165
}
153166

154167
instance.metrics.incNumContainersScanned();
@@ -184,6 +197,10 @@ public static OnDemandScannerMetrics getMetrics() {
184197
return instance.metrics;
185198
}
186199

200+
public static int getIOErrorToleranceCount() {
201+
return instance.onDemandIOErrorToleranceCount;
202+
}
203+
187204
@VisibleForTesting
188205
public static DataTransferThrottler getThrottler() {
189206
return instance.throttler;

hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestContainerScannersAbstract.java

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -163,6 +163,7 @@ protected Container<?> mockKeyValueContainer() {
163163

164164
KeyValueContainerData data = mock(KeyValueContainerData.class);
165165
when(data.getContainerID()).thenReturn(CONTAINER_SEQ_ID.incrementAndGet());
166+
when(data.getVolume()).thenReturn(vol);
166167
when(unhealthy.getContainerData()).thenReturn(data);
167168
when(unhealthy.getContainerState()).thenReturn(CLOSED);
168169
// The above mocks should be enough for the scanners to call this method

hadoop-hdds/container-service/src/test/java/org/apache/hadoop/ozone/container/ozoneimpl/TestOnDemandContainerDataScanner.java

Lines changed: 52 additions & 13 deletions
Original file line numberDiff line numberDiff line change
@@ -40,11 +40,15 @@
4040
import java.util.concurrent.ExecutionException;
4141
import java.util.concurrent.Future;
4242
import org.apache.commons.compress.utils.Lists;
43+
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
4344
import org.apache.hadoop.hdfs.util.Canceler;
4445
import org.apache.hadoop.hdfs.util.DataTransferThrottler;
4546
import org.apache.hadoop.metrics2.lib.DefaultMetricsSystem;
4647
import org.apache.hadoop.ozone.container.common.interfaces.Container;
4748
import org.apache.hadoop.ozone.container.common.interfaces.Container.ScanResult;
49+
import org.apache.hadoop.ozone.container.common.volume.MutableVolumeSet;
50+
import org.apache.hadoop.ozone.container.common.volume.StorageVolume;
51+
import org.apache.hadoop.ozone.container.common.volume.VolumeSet;
4852
import org.junit.jupiter.api.AfterEach;
4953
import org.junit.jupiter.api.BeforeEach;
5054
import org.junit.jupiter.api.Test;
@@ -102,7 +106,7 @@ public void tearDown() {
102106
public void testScanTimestampUpdated() throws Exception {
103107
OnDemandContainerDataScanner.init(conf, controller);
104108
Optional<Future<?>> scanFuture =
105-
OnDemandContainerDataScanner.scanContainer(healthy);
109+
OnDemandContainerDataScanner.scanContainer(healthy, null);
106110
assertTrue(scanFuture.isPresent());
107111
scanFuture.get().get();
108112
verify(controller, atLeastOnce())
@@ -111,7 +115,7 @@ public void testScanTimestampUpdated() throws Exception {
111115

112116
// Metrics for deleted container should not be updated.
113117
scanFuture =
114-
OnDemandContainerDataScanner.scanContainer(healthy);
118+
OnDemandContainerDataScanner.scanContainer(healthy, null);
115119
assertTrue(scanFuture.isPresent());
116120
scanFuture.get().get();
117121
verify(controller, never())
@@ -126,7 +130,7 @@ public void testContainerScannerMultipleInitsAndShutdowns() throws Exception {
126130
OnDemandContainerDataScanner.shutdown();
127131
OnDemandContainerDataScanner.shutdown();
128132
//There shouldn't be an interaction after shutdown:
129-
OnDemandContainerDataScanner.scanContainer(corruptData);
133+
OnDemandContainerDataScanner.scanContainer(corruptData, null);
130134
verifyContainerMarkedUnhealthy(corruptData, never());
131135
}
132136

@@ -143,12 +147,12 @@ public void testSameContainerQueuedMultipleTimes() throws Exception {
143147
return getUnhealthyScanResult();
144148
});
145149
Optional<Future<?>> onGoingScan = OnDemandContainerDataScanner
146-
.scanContainer(corruptData);
150+
.scanContainer(corruptData, null);
147151
assertTrue(onGoingScan.isPresent());
148152
assertFalse(onGoingScan.get().isDone());
149153
//When scheduling the same container again
150154
Optional<Future<?>> secondScan = OnDemandContainerDataScanner
151-
.scanContainer(corruptData);
155+
.scanContainer(corruptData, null);
152156
//Then the second scan is not scheduled and the first scan can still finish
153157
assertFalse(secondScan.isPresent());
154158
latch.countDown();
@@ -163,12 +167,12 @@ public void testScannerMetrics() throws Exception {
163167
OnDemandContainerDataScanner.init(conf, controller);
164168
ArrayList<Optional<Future<?>>> resultFutureList = Lists.newArrayList();
165169
resultFutureList.add(OnDemandContainerDataScanner.scanContainer(
166-
corruptData));
170+
corruptData, null));
167171
resultFutureList.add(
168-
OnDemandContainerDataScanner.scanContainer(openContainer));
172+
OnDemandContainerDataScanner.scanContainer(openContainer, null));
169173
resultFutureList.add(
170-
OnDemandContainerDataScanner.scanContainer(openCorruptMetadata));
171-
resultFutureList.add(OnDemandContainerDataScanner.scanContainer(healthy));
174+
OnDemandContainerDataScanner.scanContainer(openCorruptMetadata, null));
175+
resultFutureList.add(OnDemandContainerDataScanner.scanContainer(healthy, null));
172176
waitOnScannerToFinish(resultFutureList);
173177
OnDemandScannerMetrics metrics = OnDemandContainerDataScanner.getMetrics();
174178
//Containers with shouldScanData = false shouldn't increase
@@ -184,7 +188,7 @@ public void testScannerMetricsUnregisters() {
184188
String metricsName = OnDemandContainerDataScanner.getMetrics().getName();
185189
assertNotNull(DefaultMetricsSystem.instance().getSource(metricsName));
186190
OnDemandContainerDataScanner.shutdown();
187-
OnDemandContainerDataScanner.scanContainer(healthy);
191+
OnDemandContainerDataScanner.scanContainer(healthy, null);
188192
assertNull(DefaultMetricsSystem.instance().getSource(metricsName));
189193
}
190194

@@ -193,7 +197,7 @@ public void testScannerMetricsUnregisters() {
193197
public void testUnhealthyContainersDetected() throws Exception {
194198
// Without initialization,
195199
// there shouldn't be interaction with containerController
196-
OnDemandContainerDataScanner.scanContainer(corruptData);
200+
OnDemandContainerDataScanner.scanContainer(corruptData, null);
197201
verifyNoInteractions(controller);
198202

199203
scanContainer(healthy);
@@ -246,7 +250,7 @@ public void testShutdownDuringScan() throws Exception {
246250

247251
// Start the blocking scan.
248252
OnDemandContainerDataScanner.init(conf, controller);
249-
OnDemandContainerDataScanner.scanContainer(healthy);
253+
OnDemandContainerDataScanner.scanContainer(healthy, null);
250254
// Shut down the on demand scanner. This will interrupt the blocked scan
251255
// on the healthy container.
252256
OnDemandContainerDataScanner.shutdown();
@@ -290,10 +294,45 @@ public void testUnhealthyContainerNotRescanned() throws Exception {
290294
assertEquals(0, metrics.getNumUnHealthyContainers());
291295
}
292296

297+
@Test
298+
public void testOnDemandVolumeFailure() throws Exception {
299+
Container<?> container = mockKeyValueContainer();
300+
when(container.scanMetaData()).thenReturn(ScanResult.healthy());
301+
when(container.scanData(
302+
any(DataTransferThrottler.class), any(Canceler.class)))
303+
.thenReturn(getUnhealthyScanResult());
304+
305+
// Mock IO failure count lesser than tolerance value
306+
when(vol.incrementAndGetCumulativeIOFailureCount()).thenReturn(conf.getOnDemandIOErrorToleranceCount() - 1);
307+
MutableVolumeSet volumeSet = new MutableVolumeSet("dnId123", new OzoneConfiguration(), null,
308+
StorageVolume.VolumeType.DATA_VOLUME, null);
309+
assertEquals(1, volumeSet.getVolumesList().size());
310+
when(vol.getVolumeRootDir()).thenReturn(volumeSet.getVolumesList().get(0).getStorageDir().getPath());
311+
scanContainer(container, volumeSet);
312+
// After scan container still volume remains healthy and present in volumeSet
313+
assertEquals(1, volumeSet.getVolumesList().size());
314+
315+
// Mock IO failure count higher than tolerance value
316+
when(vol.incrementAndGetCumulativeIOFailureCount()).thenReturn(conf.getOnDemandIOErrorToleranceCount() + 1);
317+
scanContainer(container, volumeSet);
318+
// After scan container volume has failed and moved out of volumeSet
319+
assertEquals(0, volumeSet.getVolumesList().size());
320+
when(vol.incrementAndGetCumulativeIOFailureCount()).thenReturn(0);
321+
}
322+
293323
private void scanContainer(Container<?> container) throws Exception {
294324
OnDemandContainerDataScanner.init(conf, controller);
295325
Optional<Future<?>> scanFuture =
296-
OnDemandContainerDataScanner.scanContainer(container);
326+
OnDemandContainerDataScanner.scanContainer(container, null);
327+
if (scanFuture.isPresent()) {
328+
scanFuture.get().get();
329+
}
330+
}
331+
332+
private void scanContainer(Container<?> container, VolumeSet volumeSet) throws Exception {
333+
OnDemandContainerDataScanner.init(conf, controller);
334+
Optional<Future<?>> scanFuture =
335+
OnDemandContainerDataScanner.scanContainer(container, volumeSet);
297336
if (scanFuture.isPresent()) {
298337
scanFuture.get().get();
299338
}

0 commit comments

Comments
 (0)