Skip to content

Commit c79f176

Browse files
isaacreathsmiklosovic
authored andcommitted
Fix disk usage guardrail disablement when failure threshold is reached
patch by Isaac Reath; reviewed by Jyothsna Konisa, Stefan Miklosovic for CASSANDRA-21057
1 parent 73648c8 commit c79f176

File tree

4 files changed

+98
-6
lines changed

4 files changed

+98
-6
lines changed

CHANGES.txt

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -1,4 +1,5 @@
11
4.1.11
2+
* Disk usage guardrail cannot be disabled when failure threshold is reached (CASSANDRA-21057)
23
* ReadCommandController should close fast to avoid deadlock when building secondary index (CASSANDRA-19564)
34
* Redact security-sensitive information in system_views.settings (CASSANDRA-20856)
45
Merged from 4.0:

src/java/org/apache/cassandra/db/guardrails/Guardrails.java

Lines changed: 1 addition & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -50,7 +50,7 @@ public final class Guardrails implements GuardrailsMBean
5050
private static final GuardrailsOptions DEFAULT_CONFIG = DatabaseDescriptor.getGuardrailsConfig();
5151

5252
@VisibleForTesting
53-
static final Guardrails instance = new Guardrails();
53+
public static final Guardrails instance = new Guardrails();
5454

5555
/**
5656
* Guardrail on the total number of user keyspaces.

src/java/org/apache/cassandra/service/disk/usage/DiskUsageMonitor.java

Lines changed: 19 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -59,6 +59,7 @@ public class DiskUsageMonitor
5959
private final Supplier<Multimap<FileStore, Directories.DataDirectory>> dataDirectoriesSupplier;
6060

6161
private volatile DiskUsageState localState = DiskUsageState.NOT_AVAILABLE;
62+
private volatile boolean enabled;
6263

6364
@VisibleForTesting
6465
public DiskUsageMonitor()
@@ -79,14 +80,27 @@ public void start(Consumer<DiskUsageState> notifier)
7980
{
8081
// start the scheduler regardless guardrail is enabled, so we can enable it later without a restart
8182
ScheduledExecutors.scheduledTasks.scheduleAtFixedRate(() -> {
82-
83-
if (!Guardrails.localDataDiskUsage.enabled(null))
84-
return;
85-
86-
updateLocalState(getDiskUsage(), notifier);
83+
boolean currentEnabled = Guardrails.localDataDiskUsage.enabled(null);
84+
boolean oldEnabled = enabled;
85+
enabled = currentEnabled;
86+
boolean isDisabled = !currentEnabled && oldEnabled;
87+
if (isDisabled)
88+
{
89+
onDiskUsageGuardrailDisabled(notifier);
90+
}
91+
if (currentEnabled)
92+
{
93+
updateLocalState(getDiskUsage(), notifier);
94+
}
8795
}, 0, CassandraRelevantProperties.DISK_USAGE_MONITOR_INTERVAL_MS.getLong(), TimeUnit.MILLISECONDS);
8896
}
8997

98+
private void onDiskUsageGuardrailDisabled(Consumer<DiskUsageState> notifier)
99+
{
100+
localState = DiskUsageState.NOT_AVAILABLE;
101+
notifier.accept(DiskUsageState.NOT_AVAILABLE);
102+
}
103+
90104
@VisibleForTesting
91105
public void updateLocalState(double usageRatio, Consumer<DiskUsageState> notifier)
92106
{

test/distributed/org/apache/cassandra/distributed/test/guardrails/GuardrailDiskUsageTest.java

Lines changed: 77 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -194,6 +194,83 @@ public void testDiskUsage() throws Throwable
194194
}
195195
}
196196

197+
@Test
198+
public void testDiskUsageNodetoolDisableWhenDiskIsFullShouldEnableWrites()
199+
{
200+
schemaChange("CREATE TABLE %s (k int PRIMARY KEY, v int)");
201+
String insert = format("INSERT INTO %s(k, v) VALUES (?, 0)");
202+
203+
// With both nodes in SPACIOUS state, we can write without warnings nor failures
204+
for (int i = 0; i < NUM_ROWS; i++)
205+
{
206+
ResultSet rs = driverSession.execute(insert, i);
207+
Assertions.assertThat(rs.getExecutionInfo().getWarnings()).isEmpty();
208+
}
209+
210+
// If the STUFFED node becomes FULL, the writes targeting that node will fail, while the writes targeting
211+
// the node that remains SPACIOUS will keep succeeding without warnings
212+
DiskStateInjection.setState(getCluster(), 2, DiskUsageState.FULL);
213+
int numFailures = 0;
214+
for (int i = 0; i < NUM_ROWS; i++)
215+
{
216+
try
217+
{
218+
ResultSet rs = driverSession.execute(insert, i);
219+
Assertions.assertThat(rs.getExecutionInfo().getWarnings()).isEmpty();
220+
}
221+
catch (InvalidQueryException e)
222+
{
223+
Assertions.assertThat(e).hasMessageContaining(FAIL_MESSAGE);
224+
numFailures++;
225+
}
226+
}
227+
Assertions.assertThat(numFailures).isGreaterThan(0).isLessThan(NUM_ROWS);
228+
229+
// After disabling the guardrail, we should be able to write again.
230+
cluster.get(2).runOnInstance(() -> Guardrails.instance.setDataDiskUsagePercentageThreshold(-1, -1));
231+
int stateDissemenationTimeoutSec = 2 * 60; // 2 minutes.
232+
Util.spinAssertEquals(true,
233+
() -> cluster.get(1).callOnInstance(() -> !DiskUsageBroadcaster.instance.hasStuffedOrFullNode()),
234+
stateDissemenationTimeoutSec
235+
);
236+
237+
for (int i = 0; i < NUM_ROWS; i++)
238+
{
239+
ResultSet rs = driverSession.execute(insert, i);
240+
Assertions.assertThat(rs.getExecutionInfo().getWarnings()).isEmpty();
241+
}
242+
243+
// Re-enabling the guardrail should again cause writes to fail
244+
cluster.get(2).runOnInstance(() -> Guardrails.instance.setDataDiskUsagePercentageThreshold(98, 99));
245+
Util.spinAssertEquals(true,
246+
() -> cluster.get(1).callOnInstance(() -> DiskUsageBroadcaster.instance.hasStuffedOrFullNode()),
247+
stateDissemenationTimeoutSec
248+
);
249+
numFailures = 0;
250+
for (int i = 0; i < NUM_ROWS; i++)
251+
{
252+
try
253+
{
254+
ResultSet rs = driverSession.execute(insert, i);
255+
Assertions.assertThat(rs.getExecutionInfo().getWarnings()).isEmpty();
256+
}
257+
catch (InvalidQueryException e)
258+
{
259+
Assertions.assertThat(e).hasMessageContaining(FAIL_MESSAGE);
260+
numFailures++;
261+
}
262+
}
263+
Assertions.assertThat(numFailures).isGreaterThan(0).isLessThan(NUM_ROWS);
264+
265+
// Finally, if both nodes go back to SPACIOUS, all queries will succeed again
266+
DiskStateInjection.setState(getCluster(), 2, DiskUsageState.SPACIOUS);
267+
for (int i = 0; i < NUM_ROWS; i++)
268+
{
269+
ResultSet rs = driverSession.execute(insert, i);
270+
Assertions.assertThat(rs.getExecutionInfo().getWarnings()).isEmpty();
271+
}
272+
}
273+
197274
/**
198275
* ByteBuddy rule to override the disk usage state of each node.
199276
*/

0 commit comments

Comments
 (0)