Skip to content

Commit f7fab8a

Browse files
committed
Merge branch 'cassandra-5.0' into trunk
2 parents 39019f0 + bbbf70a commit f7fab8a

File tree

3 files changed

+97
-5
lines changed

3 files changed

+97
-5
lines changed

CHANGES.txt

Lines changed: 1 addition & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -341,6 +341,7 @@ Merged from 5.0:
341341
* Prioritize built indexes in IndexStatusManager (CASSANDRA-19400)
342342
* Add java.base/java.lang.reflect among opens for jvm11-client.options (CASSANDRA-19780)
343343
Merged from 4.1:
344+
* Disk usage guardrail cannot be disabled when failure threshold is reached (CASSANDRA-21057)
344345
* ReadCommandController should close fast to avoid deadlock when building secondary index (CASSANDRA-19564)
345346
* Redact security-sensitive information in system_views.settings (CASSANDRA-20856)
346347
* Improve CommitLogSegmentReader to skip SyncBlocks correctly in case of CRC errors (CASSANDRA-20664)

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
@@ -202,6 +202,83 @@ public void testDiskUsage() throws Throwable
202202
}
203203
}
204204

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

0 commit comments

Comments
 (0)