17
17
package org .apache .kafka .storage .internals .log ;
18
18
19
19
import org .apache .kafka .common .utils .ByteBufferUnmapper ;
20
- import org .apache .kafka .common .utils .OperatingSystem ;
21
20
import org .apache .kafka .common .utils .Utils ;
21
+ import org .apache .kafka .server .util .LockUtils ;
22
22
23
23
import org .slf4j .Logger ;
24
24
import org .slf4j .LoggerFactory ;
33
33
import java .nio .file .Files ;
34
34
import java .util .Objects ;
35
35
import java .util .OptionalInt ;
36
- import java .util .concurrent .locks .Lock ;
37
36
import java .util .concurrent .locks .ReentrantLock ;
37
+ import java .util .concurrent .locks .ReentrantReadWriteLock ;
38
+ import java .util .function .Supplier ;
38
39
39
40
/**
40
41
* The abstract index class which holds entry format agnostic methods.
@@ -47,7 +48,10 @@ private enum SearchResultType {
47
48
48
49
private static final Logger log = LoggerFactory .getLogger (AbstractIndex .class );
49
50
50
- protected final ReentrantLock lock = new ReentrantLock ();
51
+ // Serializes all index operations that mutate internal state
52
+ private final ReentrantLock lock = new ReentrantLock ();
53
+ // Allows concurrent read operations while ensuring exclusive access if the underlying mmap is changed
54
+ private final ReentrantReadWriteLock remapLock = new ReentrantReadWriteLock ();
51
55
52
56
private final long baseOffset ;
53
57
private final int maxIndexSize ;
@@ -187,36 +191,32 @@ public void updateParentDir(File parentDir) {
187
191
* @return a boolean indicating whether the size of the memory map and the underneath file is changed or not.
188
192
*/
189
193
public boolean resize (int newSize ) throws IOException {
190
- lock .lock ();
191
- try {
192
- int roundedNewSize = roundDownToExactMultiple (newSize , entrySize ());
193
-
194
- if (length == roundedNewSize ) {
195
- log .debug ("Index {} was not resized because it already has size {}" , file .getAbsolutePath (), roundedNewSize );
196
- return false ;
197
- } else {
198
- RandomAccessFile raf = new RandomAccessFile (file , "rw" );
199
- try {
200
- int position = mmap .position ();
201
-
202
- /* Windows or z/OS won't let us modify the file length while the file is mmapped :-( */
203
- if (OperatingSystem .IS_WINDOWS || OperatingSystem .IS_ZOS )
204
- safeForceUnmap ();
205
- raf .setLength (roundedNewSize );
206
- this .length = roundedNewSize ;
207
- mmap = raf .getChannel ().map (FileChannel .MapMode .READ_WRITE , 0 , roundedNewSize );
208
- this .maxEntries = mmap .limit () / entrySize ();
209
- mmap .position (position );
210
- log .debug ("Resized {} to {}, position is {} and limit is {}" , file .getAbsolutePath (), roundedNewSize ,
211
- mmap .position (), mmap .limit ());
212
- return true ;
213
- } finally {
214
- Utils .closeQuietly (raf , "index file " + file .getName ());
215
- }
216
- }
217
- } finally {
218
- lock .unlock ();
219
- }
194
+ return inLockThrows (() ->
195
+ inRemapWriteLockThrows (() -> {
196
+ int roundedNewSize = roundDownToExactMultiple (newSize , entrySize ());
197
+
198
+ if (length == roundedNewSize ) {
199
+ log .debug ("Index {} was not resized because it already has size {}" , file .getAbsolutePath (), roundedNewSize );
200
+ return false ;
201
+ } else {
202
+ RandomAccessFile raf = new RandomAccessFile (file , "rw" );
203
+ try {
204
+ int position = mmap .position ();
205
+
206
+ safeForceUnmap ();
207
+ raf .setLength (roundedNewSize );
208
+ this .length = roundedNewSize ;
209
+ mmap = raf .getChannel ().map (FileChannel .MapMode .READ_WRITE , 0 , roundedNewSize );
210
+ this .maxEntries = mmap .limit () / entrySize ();
211
+ mmap .position (position );
212
+ log .debug ("Resized {} to {}, position is {} and limit is {}" , file .getAbsolutePath (), roundedNewSize ,
213
+ mmap .position (), mmap .limit ());
214
+ return true ;
215
+ } finally {
216
+ Utils .closeQuietly (raf , "index file " + file .getName ());
217
+ }
218
+ }
219
+ }));
220
220
}
221
221
222
222
/**
@@ -236,12 +236,9 @@ public void renameTo(File f) throws IOException {
236
236
* Flush the data in the index to disk
237
237
*/
238
238
public void flush () {
239
- lock .lock ();
240
- try {
239
+ inLock (() -> {
241
240
mmap .force ();
242
- } finally {
243
- lock .unlock ();
244
- }
241
+ });
245
242
}
246
243
247
244
/**
@@ -261,14 +258,11 @@ public boolean deleteIfExists() throws IOException {
261
258
* the file.
262
259
*/
263
260
public void trimToValidSize () throws IOException {
264
- lock .lock ();
265
- try {
261
+ inLockThrows (() -> {
266
262
if (mmap != null ) {
267
263
resize (entrySize () * entries );
268
264
}
269
- } finally {
270
- lock .unlock ();
271
- }
265
+ });
272
266
}
273
267
274
268
/**
@@ -288,12 +282,10 @@ public void closeHandler() {
288
282
// However, in some cases it can pause application threads(STW) for a long moment reading metadata from a physical disk.
289
283
// To prevent this, we forcefully cleanup memory mapping within proper execution which never affects API responsiveness.
290
284
// See https://issues.apache.org/jira/browse/KAFKA-4614 for the details.
291
- lock .lock ();
292
- try {
293
- safeForceUnmap ();
294
- } finally {
295
- lock .unlock ();
296
- }
285
+ inLockThrows (() ->
286
+ inRemapWriteLockThrows (() -> {
287
+ safeForceUnmap ();
288
+ }));
297
289
}
298
290
299
291
/**
@@ -420,20 +412,36 @@ protected void truncateToEntries0(int entries) {
420
412
mmap .position (entries * entrySize ());
421
413
}
422
414
423
- /**
424
- * Execute the given function in a lock only if we are running on windows or z/OS. We do this
425
- * because Windows or z/OS won't let us resize a file while it is mmapped. As a result we have to force unmap it
426
- * and this requires synchronizing reads.
427
- */
428
- protected final <T , E extends Exception > T maybeLock (Lock lock , StorageAction <T , E > action ) throws E {
429
- if (OperatingSystem .IS_WINDOWS || OperatingSystem .IS_ZOS )
430
- lock .lock ();
431
- try {
432
- return action .execute ();
433
- } finally {
434
- if (OperatingSystem .IS_WINDOWS || OperatingSystem .IS_ZOS )
435
- lock .unlock ();
436
- }
415
+ protected final <T > T inLock (Supplier <T > action ) {
416
+ return LockUtils .inLock (lock , action );
417
+ }
418
+
419
+ protected final void inLock (Runnable action ) {
420
+ LockUtils .inLock (lock , action );
421
+ }
422
+
423
+ protected final <T , E extends Exception > T inLockThrows (LockUtils .ThrowingSupplier <T , E > action ) throws E {
424
+ return LockUtils .inLockThrows (lock , action );
425
+ }
426
+
427
+ protected final <E extends Exception > void inLockThrows (LockUtils .ThrowingRunnable <E > action ) throws E {
428
+ LockUtils .inLockThrows (lock , action );
429
+ }
430
+
431
+ protected final <T > T inRemapReadLock (Supplier <T > action ) {
432
+ return LockUtils .inLock (remapLock .readLock (), action );
433
+ }
434
+
435
+ protected final void inRemapReadLock (Runnable action ) {
436
+ LockUtils .inLock (remapLock .readLock (), action );
437
+ }
438
+
439
+ protected final <T , E extends Exception > T inRemapWriteLockThrows (LockUtils .ThrowingSupplier <T , E > action ) throws E {
440
+ return LockUtils .inLockThrows (remapLock .writeLock (), action );
441
+ }
442
+
443
+ protected final <E extends Exception > void inRemapWriteLockThrows (LockUtils .ThrowingRunnable <E > action ) throws E {
444
+ LockUtils .inLockThrows (remapLock .writeLock (), action );
437
445
}
438
446
439
447
/**
0 commit comments