17
17
*/
18
18
package org .apache .cassandra .db .commitlog ;
19
19
20
+ import java .io .FileOutputStream ;
20
21
import java .io .IOException ;
22
+ import java .nio .ByteBuffer ;
21
23
import java .util .ArrayList ;
22
24
import java .util .List ;
23
25
26
+ import org .apache .cassandra .distributed .shared .WithProperties ;
24
27
import org .apache .cassandra .io .util .File ;
28
+ import org .apache .cassandra .security .EncryptionContextGenerator ;
25
29
import org .junit .Assert ;
26
30
import org .junit .Before ;
27
31
import org .junit .BeforeClass ;
40
44
import org .apache .cassandra .db .rows .Row ;
41
45
import org .apache .cassandra .utils .JVMStabilityInspector ;
42
46
import org .apache .cassandra .utils .KillerForTests ;
47
+ import org .assertj .core .api .Assertions ;
48
+
49
+ import static org .apache .cassandra .db .commitlog .CommitLogReplayer .IGNORE_REPLAY_ERRORS_PROPERTY ;
43
50
44
51
public class CommitLogReaderTest extends CQLTester
45
52
{
53
+ private static final long CORRUPTED_COMMIT_LOG_FILE_ID = 111L ;
54
+ private static final String CORRUPTED_COMMIT_LOG_FILE_NAME = "CommitLog-7-1234567.log" ;
55
+
46
56
@ BeforeClass
47
57
public static void setUpClass ()
48
58
{
49
59
prePrepareServer ();
50
60
51
- DatabaseDescriptor .setCommitFailurePolicy (Config .CommitFailurePolicy .ignore );
52
61
JVMStabilityInspector .replaceKiller (new KillerForTests (false ));
53
62
54
63
DatabaseDescriptor .setCommitLogSync (Config .CommitLogSync .batch );
@@ -60,7 +69,12 @@ public static void setUpClass()
60
69
@ Before
61
70
public void before () throws IOException
62
71
{
72
+ clearCorruptedCommitLogFile ();
63
73
CommitLog .instance .resetUnsafe (true );
74
+
75
+ // always reset to what Cassandra's default is and let each test method
76
+ // handle its expected failure policy itself for better test encapsulation.
77
+ DatabaseDescriptor .setCommitFailurePolicy (Config .CommitFailurePolicy .stop );
64
78
}
65
79
66
80
@ Test
@@ -165,6 +179,58 @@ public void testReadCountFromMidpoint() throws Throwable
165
179
confirmReadOrder (testHandler , samples / 2 );
166
180
}
167
181
182
+ @ Test
183
+ public void testSyncMarkerChecksumReadFailed_ignoreReplayErrorsDisabled () throws Throwable
184
+ {
185
+ File corruptedSegmentFile = createAndWriteCorruptedCommitLogFile ();
186
+ CommitLogReader reader = new CommitLogReader ();
187
+ // use real CLR handler to test actual behavior
188
+ CommitLogReadHandler clrHandler =
189
+ new CommitLogReplayer (new CommitLog (null ), null , null , null );
190
+
191
+ // ignore.replay.errors disabled, so we expect the exception here
192
+ Assertions .assertThatThrownBy (() ->
193
+ reader .readCommitLogSegment (clrHandler ,
194
+ corruptedSegmentFile ,
195
+ CommitLogPosition .NONE ,
196
+ CommitLogReader .ALL_MUTATIONS ,
197
+ false )
198
+ ).isInstanceOf (CommitLogReplayer .CommitLogReplayException .class );
199
+ }
200
+
201
+ @ Test
202
+ public void testSyncMarkerChecksumReadFailed_ignoreReplayErrorsEnabled () throws Throwable
203
+ {
204
+ try (WithProperties properties = new WithProperties (IGNORE_REPLAY_ERRORS_PROPERTY , "true" ))
205
+ {
206
+ File corruptedSegmentFile = createAndWriteCorruptedCommitLogFile ();
207
+
208
+ CommitLogReader reader = new CommitLogReader ();
209
+ // use real CLR handler to test actual behavior
210
+ CommitLogReadHandler clrHandler =
211
+ new CommitLogReplayer (new CommitLog (null ), null , null , null );
212
+
213
+ // ignore.replay.errors enabled, so we don't expect any errors
214
+ reader .readCommitLogSegment (clrHandler , corruptedSegmentFile , CommitLogPosition .NONE , CommitLogReader .ALL_MUTATIONS , false );
215
+ }
216
+ }
217
+
218
+ @ Test
219
+ public void testSyncMarkerChecksumReadFailed_ignoreReplayErrorsDisabled_commitFailurePolicyIgnore () throws Throwable
220
+ {
221
+ DatabaseDescriptor .setCommitFailurePolicy (Config .CommitFailurePolicy .ignore );
222
+
223
+ File corruptedSegmentFile = createAndWriteCorruptedCommitLogFile ();
224
+
225
+ CommitLogReader reader = new CommitLogReader ();
226
+ // use real CLR handler to test actual behavior
227
+ CommitLogReadHandler clrHandler =
228
+ new CommitLogReplayer (new CommitLog (null ), null , null , null );
229
+
230
+ // commit.failure.policy=ignore, so we don't expect any errors
231
+ reader .readCommitLogSegment (clrHandler , corruptedSegmentFile , CommitLogPosition .NONE , CommitLogReader .ALL_MUTATIONS , false );
232
+ }
233
+
168
234
/**
169
235
* Since we have both table and non mixed into the CL, we ignore updates that aren't for the table the test handler
170
236
* is configured to check.
@@ -207,7 +273,7 @@ static ArrayList<File> getCommitLogs()
207
273
continue ;
208
274
results .add (f );
209
275
}
210
- Assert .assertTrue ("Didn't find any commit log files." , 0 != results .size ());
276
+ Assert .assertFalse ("Didn't find any commit log files." , results .isEmpty ());
211
277
return results ;
212
278
}
213
279
@@ -229,20 +295,20 @@ public TestCLRHandler(TableMetadata metadata)
229
295
this .metadata = metadata ;
230
296
}
231
297
232
- public boolean shouldSkipSegmentOnError (CommitLogReadException exception ) throws IOException
298
+ public boolean shouldSkipSegmentOnError (CommitLogReadException exception )
233
299
{
234
300
sawStopOnErrorCheck = true ;
235
301
return false ;
236
302
}
237
303
238
- public void handleUnrecoverableError (CommitLogReadException exception ) throws IOException
304
+ public void handleUnrecoverableError (CommitLogReadException exception )
239
305
{
240
306
sawStopOnErrorCheck = true ;
241
307
}
242
308
243
309
public void handleMutation (Mutation m , int size , int entryLocation , CommitLogDescriptor desc )
244
310
{
245
- if (( metadata == null ) || ( metadata != null && m .getPartitionUpdate (metadata ) != null ) ) {
311
+ if (metadata == null || m .getPartitionUpdate (metadata ) != null ) {
246
312
seenMutations .add (m );
247
313
}
248
314
}
@@ -274,4 +340,35 @@ CommitLogPosition populateData(int entryCount) throws Throwable
274
340
.forceBlockingFlush (ColumnFamilyStore .FlushReason .UNIT_TESTS );
275
341
return result ;
276
342
}
343
+
344
+ private static File createAndWriteCorruptedCommitLogFile () throws IOException
345
+ {
346
+ final ByteBuffer corruptedSegmentByteBuffer =
347
+ ByteBuffer .allocate (DatabaseDescriptor .getCommitLogSegmentSize ());
348
+
349
+ final CommitLogDescriptor commitLogDescriptor =
350
+ new CommitLogDescriptor (CORRUPTED_COMMIT_LOG_FILE_ID , null , EncryptionContextGenerator .createDisabledContext ());
351
+
352
+ CommitLogDescriptor .writeHeader (corruptedSegmentByteBuffer , commitLogDescriptor );
353
+
354
+ // write corrupted sync marker:
355
+ // put wrong offset
356
+ corruptedSegmentByteBuffer .putInt (42 );
357
+ // put wrong CRC
358
+ corruptedSegmentByteBuffer .putInt (42 );
359
+
360
+ final File corruptedLogFile = new File (DatabaseDescriptor .getCommitLogLocation (), CORRUPTED_COMMIT_LOG_FILE_NAME );
361
+ try (FileOutputStream fos = new FileOutputStream (corruptedLogFile .toJavaIOFile ()))
362
+ {
363
+ fos .write (corruptedSegmentByteBuffer .array ());
364
+ fos .flush ();
365
+ }
366
+
367
+ return corruptedLogFile ;
368
+ }
369
+
370
+ private static void clearCorruptedCommitLogFile ()
371
+ {
372
+ new File (DatabaseDescriptor .getCommitLogLocation (), CORRUPTED_COMMIT_LOG_FILE_NAME ).deleteIfExists ();
373
+ }
277
374
}
0 commit comments