16
16
* specific language governing permissions and limitations
17
17
* under the License.
18
18
*/
19
+
19
20
package org .apache .hadoop .fs .impl ;
20
21
21
22
import java .nio .ByteBuffer ;
22
23
import java .util .HashMap ;
23
24
import java .util .Map ;
24
25
import java .util .Objects ;
26
+ import java .util .concurrent .atomic .AtomicInteger ;
25
27
26
28
import org .slf4j .Logger ;
27
29
import org .slf4j .LoggerFactory ;
28
30
29
31
import org .apache .hadoop .io .ByteBufferPool ;
30
32
31
33
/**
32
- * A wrapper {@link ByteBufferPool} implementation that tracks whether all allocated buffers are released. It
33
- * throws the related exception at {@link #close()} if any buffer remains un-released. It also clears the buffers at
34
- * release so if they continued being used it'll generate errors.
34
+ * A wrapper {@link ByteBufferPool} implementation that tracks whether all allocated buffers
35
+ * are released.
36
+ * <p>
37
+ * It throws the related exception at {@link #close()} if any buffer remains un-released.
38
+ * It also clears the buffers at release so if they continued being used it'll generate errors.
35
39
* <p>
36
40
* To be used for testing only.
37
41
* <p>
38
- * The stacktraces of the allocation are not stored by default because it significantly decreases the unit test
39
- * execution performance. Configuring this class to log at DEBUG will trigger their collection.
42
+ * The stacktraces of the allocation are not stored by default because
43
+ * it can significantly decreases the unit test performance.
44
+ * Configuring this class to log at DEBUG will trigger their collection.
40
45
* @see ByteBufferAllocationStacktraceException
41
46
* <p>
42
47
* Adapted from Parquet class {@code org.apache.parquet.bytes.TrackingByteBufferAllocator}.
43
48
*/
44
49
public final class TrackingByteBufferPool implements ByteBufferPool , AutoCloseable {
45
50
46
- /**
47
-
48
- */
49
- private static final boolean DEBUG = true ;
50
51
private static final Logger LOG = LoggerFactory .getLogger (TrackingByteBufferPool .class );
51
52
52
53
/**
@@ -60,10 +61,13 @@ public static TrackingByteBufferPool wrap(ByteBufferPool allocator) {
60
61
61
62
/**
62
63
* Key for the tracker map.
64
+ * This uses the identity hash code of the buffer as the hash code
65
+ * for the map.
63
66
*/
64
67
private static class Key {
65
68
66
69
private final int hashCode ;
70
+
67
71
private final ByteBuffer buffer ;
68
72
69
73
Key (ByteBuffer buffer ) {
@@ -105,7 +109,10 @@ private LeakDetectorHeapByteBufferPoolException(String msg, Throwable cause) {
105
109
}
106
110
107
111
private LeakDetectorHeapByteBufferPoolException (
108
- String message , Throwable cause , boolean enableSuppression , boolean writableStackTrace ) {
112
+ String message ,
113
+ Throwable cause ,
114
+ boolean enableSuppression ,
115
+ boolean writableStackTrace ) {
109
116
super (message , cause , enableSuppression , writableStackTrace );
110
117
}
111
118
}
@@ -116,6 +123,9 @@ private LeakDetectorHeapByteBufferPoolException(
116
123
public static final class ByteBufferAllocationStacktraceException
117
124
extends LeakDetectorHeapByteBufferPoolException {
118
125
126
+ /**
127
+ * Single stack trace instance to use when DEBUG is not enabled.
128
+ */
119
129
private static final ByteBufferAllocationStacktraceException WITHOUT_STACKTRACE =
120
130
new ByteBufferAllocationStacktraceException (false );
121
131
@@ -134,9 +144,12 @@ private ByteBufferAllocationStacktraceException() {
134
144
super ("Allocation stacktrace of the first ByteBuffer:" );
135
145
}
136
146
147
+ /**
148
+ * Private constructor to for the singleton {@link #WITHOUT_STACKTRACE},
149
+ * telling develoers how to see a trace per buffer.
150
+ */
137
151
private ByteBufferAllocationStacktraceException (boolean unused ) {
138
- super (
139
- "Log org.apache.hadoop.fs.impl.TrackingByteBufferPool at DEBUG for full stack traces" ,
152
+ super ("Log org.apache.hadoop.fs.impl.TrackingByteBufferPool at DEBUG for stack traces" ,
140
153
null ,
141
154
false ,
142
155
false );
@@ -147,20 +160,36 @@ private ByteBufferAllocationStacktraceException(boolean unused) {
147
160
* Exception raised in {@link TrackingByteBufferPool#putBuffer(ByteBuffer)} if the
148
161
* buffer to release was not in the hash map.
149
162
*/
150
- public static final class ReleasingUnallocatedByteBufferException extends LeakDetectorHeapByteBufferPoolException {
163
+ public static final class ReleasingUnallocatedByteBufferException
164
+ extends LeakDetectorHeapByteBufferPoolException {
151
165
152
- private ReleasingUnallocatedByteBufferException () {
153
- super ("Releasing a ByteBuffer instance that is not allocated by this buffer pool or already been released" );
166
+ private ReleasingUnallocatedByteBufferException (final ByteBuffer b ) {
167
+ super (String .format ("Releasing a ByteBuffer instance that is not allocated"
168
+ + " by this buffer pool or already been released: %s size %d" , b , b .capacity ()));
154
169
}
155
170
}
156
171
157
172
/**
158
- * Exception raised in {@link TrackingByteBufferPool#close()} if there was an unreleased buffer.
173
+ * Exception raised in {@link TrackingByteBufferPool#close()} if there
174
+ * was an unreleased buffer.
159
175
*/
160
- public static class LeakedByteBufferException extends LeakDetectorHeapByteBufferPoolException {
176
+ public static final class LeakedByteBufferException
177
+ extends LeakDetectorHeapByteBufferPoolException {
178
+
179
+ private final int count ;
161
180
162
181
private LeakedByteBufferException (int count , ByteBufferAllocationStacktraceException e ) {
163
- super (count + " ByteBuffer object(s) is/are remained unreleased after closing this buffer pool." , e );
182
+ super (count + " ByteBuffer object(s) is/are remained unreleased"
183
+ + " after closing this buffer pool." , e );
184
+ this .count = count ;
185
+ }
186
+
187
+ /**
188
+ * Get the number of unreleased buffers.
189
+ * @return number of unreleased buffers
190
+ */
191
+ public int getCount () {
192
+ return count ;
164
193
}
165
194
}
166
195
@@ -170,13 +199,28 @@ private LeakedByteBufferException(int count, ByteBufferAllocationStacktraceExcep
170
199
* The key maps by the object id of the buffer, and refers to either a common stack trace
171
200
* or one dynamically created for each allocation.
172
201
*/
173
- private final Map <Key , ByteBufferAllocationStacktraceException > allocated = new HashMap <>();
202
+ private final Map <Key , ByteBufferAllocationStacktraceException > allocated =
203
+ new HashMap <>();
174
204
175
205
/**
176
206
* Wrapped buffer pool.
177
207
*/
178
208
private final ByteBufferPool allocator ;
179
209
210
+ /**
211
+ * Number of buffer allocations.
212
+ * <p>
213
+ * This is incremented in {@link #getBuffer(boolean, int)}.
214
+ */
215
+ private final AtomicInteger bufferAllocations = new AtomicInteger ();
216
+
217
+ /**
218
+ * Number of buffer releases.
219
+ * <p>
220
+ * This is incremented in {@link #putBuffer(ByteBuffer)}.
221
+ */
222
+ private final AtomicInteger bufferReleases = new AtomicInteger ();
223
+
180
224
/**
181
225
* private constructor.
182
226
* @param allocator pool allocator.
@@ -185,34 +229,87 @@ private TrackingByteBufferPool(ByteBufferPool allocator) {
185
229
this .allocator = allocator ;
186
230
}
187
231
232
+ public int getBufferAllocations () {
233
+ return bufferAllocations .get ();
234
+ }
235
+
236
+ public int getBufferReleases () {
237
+ return bufferReleases .get ();
238
+ }
239
+
240
+ /**
241
+ * Get a buffer from the pool.
242
+ * <p>
243
+ * This increments the {@link #bufferAllocations} counter and stores the
244
+ * singleron or local allocation stack trace in the {@link #allocated} map.
245
+ * @param direct whether to allocate a direct buffer or not
246
+ * @param size size of the buffer to allocate
247
+ * @return a ByteBuffer instance
248
+ */
188
249
@ Override
189
- public ByteBuffer getBuffer (final boolean direct , final int size ) {
250
+ public synchronized ByteBuffer getBuffer (final boolean direct , final int size ) {
251
+ bufferAllocations .incrementAndGet ();
190
252
ByteBuffer buffer = allocator .getBuffer (direct , size );
191
- final ByteBufferAllocationStacktraceException ex = ByteBufferAllocationStacktraceException .create ();
253
+ final ByteBufferAllocationStacktraceException ex =
254
+ ByteBufferAllocationStacktraceException .create ();
192
255
final Key key = new Key (buffer );
193
256
allocated .put (key , ex );
194
257
LOG .debug ("Creating ByteBuffer:{} size {} {}" , key .hashCode (), size , buffer , ex );
195
258
return buffer ;
196
259
}
197
260
261
+ /**
262
+ * Release a buffer back to the pool.
263
+ * <p>
264
+ * This increments the {@link #bufferReleases} counter and removes the
265
+ * buffer from the {@link #allocated} map.
266
+ * <p>
267
+ * If the buffer was not allocated by this pool, it throws
268
+ * {@link ReleasingUnallocatedByteBufferException}.
269
+ *
270
+ * @param b buffer to release
271
+ * @throws ReleasingUnallocatedByteBufferException if the buffer was not allocated by this pool
272
+ */
198
273
@ Override
199
- public void putBuffer (ByteBuffer b ) throws ReleasingUnallocatedByteBufferException {
274
+ public synchronized void putBuffer (ByteBuffer b )
275
+ throws ReleasingUnallocatedByteBufferException {
276
+
277
+ bufferReleases .incrementAndGet ();
200
278
Objects .requireNonNull (b );
201
279
final Key key = new Key (b );
202
280
LOG .debug ("Releasing ByteBuffer: {}: {}" , key .hashCode (), b );
203
281
if (allocated .remove (key ) == null ) {
204
- throw new ReleasingUnallocatedByteBufferException ();
282
+ throw new ReleasingUnallocatedByteBufferException (b );
205
283
}
206
284
allocator .putBuffer (b );
207
285
// Clearing the buffer so subsequent access would probably generate errors
208
286
b .clear ();
209
287
}
210
288
289
+ /**
290
+ * Check if the buffer is in the pool.
291
+ * @param b buffer
292
+ * @return true if the buffer is in the pool
293
+ */
294
+ public boolean containsBuffer (ByteBuffer b ) {
295
+ Objects .requireNonNull (b );
296
+ final Key key = new Key (b );
297
+ return allocated .containsKey (key );
298
+ }
299
+
300
+ /**
301
+ * Get the number of allocated buffers.
302
+ * @return number of allocated buffers
303
+ */
304
+ public int size () {
305
+ return allocated .size ();
306
+ }
307
+
211
308
/**
212
309
* Expect all buffers to be released -if not, log unreleased ones
213
310
* and then raise an exception with the stack trace of the first
214
311
* unreleased buffer.
215
- * @throws LeakedByteBufferException if at least one was unsued.
312
+ * @throws LeakedByteBufferException if at least one buffer was not released
216
313
*/
217
314
@ Override
218
315
public void close () throws LeakedByteBufferException {
0 commit comments