-
Notifications
You must be signed in to change notification settings - Fork 3.3k
/
PooledByteBufAllocatorL.java
280 lines (230 loc) · 9.11 KB
/
PooledByteBufAllocatorL.java
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package io.netty.buffer;
import static org.apache.arrow.memory.util.AssertionUtil.ASSERT_ENABLED;
import java.lang.reflect.Field;
import java.nio.ByteBuffer;
import java.util.concurrent.atomic.AtomicLong;
import org.apache.arrow.memory.OutOfMemoryException;
import org.apache.arrow.memory.util.LargeMemoryUtil;
import io.netty.util.internal.OutOfDirectMemoryError;
import io.netty.util.internal.StringUtil;
/**
* The base allocator that we use for all of Arrow's memory management. Returns
* UnsafeDirectLittleEndian buffers.
*/
public class PooledByteBufAllocatorL {
private static final org.slf4j.Logger memoryLogger = org.slf4j.LoggerFactory.getLogger("arrow.allocator");
private static final int MEMORY_LOGGER_FREQUENCY_SECONDS = 60;
public final UnsafeDirectLittleEndian empty;
private final AtomicLong hugeBufferSize = new AtomicLong(0);
private final AtomicLong hugeBufferCount = new AtomicLong(0);
private final AtomicLong normalBufferSize = new AtomicLong(0);
private final AtomicLong normalBufferCount = new AtomicLong(0);
private final InnerAllocator allocator;
public PooledByteBufAllocatorL() {
allocator = new InnerAllocator();
empty = new UnsafeDirectLittleEndian(new DuplicatedByteBuf(Unpooled.EMPTY_BUFFER));
}
/**
* Returns a {@linkplain io.netty.buffer.UnsafeDirectLittleEndian} of the given size.
*/
public UnsafeDirectLittleEndian allocate(long size) {
try {
return allocator.directBuffer(LargeMemoryUtil.checkedCastToInt(size), Integer.MAX_VALUE);
} catch (OutOfMemoryError e) {
/*
* OutOfDirectMemoryError is thrown by Netty when we exceed the direct memory limit defined by
* -XX:MaxDirectMemorySize. OutOfMemoryError with "Direct buffer memory" message is thrown by
* java.nio.Bits when we exceed the direct memory limit. This should never be hit in practice
* as Netty is expected to throw an OutOfDirectMemoryError first.
*/
if (e instanceof OutOfDirectMemoryError || "Direct buffer memory".equals(e.getMessage())) {
throw new OutOfMemoryException("Failure allocating buffer.", e);
}
throw e;
}
}
public int getChunkSize() {
return allocator.chunkSize;
}
public long getHugeBufferSize() {
return hugeBufferSize.get();
}
public long getHugeBufferCount() {
return hugeBufferCount.get();
}
public long getNormalBufferSize() {
return normalBufferSize.get();
}
public long getNormalBufferCount() {
return normalBufferSize.get();
}
private static class AccountedUnsafeDirectLittleEndian extends UnsafeDirectLittleEndian {
private final long initialCapacity;
private final AtomicLong count;
private final AtomicLong size;
private AccountedUnsafeDirectLittleEndian(LargeBuffer buf, AtomicLong count, AtomicLong size) {
super(buf);
this.initialCapacity = buf.capacity();
this.count = count;
this.size = size;
}
private AccountedUnsafeDirectLittleEndian(PooledUnsafeDirectByteBuf buf, AtomicLong count,
AtomicLong size) {
super(buf);
this.initialCapacity = buf.capacity();
this.count = count;
this.size = size;
}
@Override
public ByteBuf copy() {
throw new UnsupportedOperationException("copy method is not supported");
}
@Override
public ByteBuf copy(int index, int length) {
throw new UnsupportedOperationException("copy method is not supported");
}
@Override
public boolean release(int decrement) {
boolean released = super.release(decrement);
if (released) {
count.decrementAndGet();
size.addAndGet(-initialCapacity);
}
return released;
}
}
private class InnerAllocator extends PooledByteBufAllocator {
private final PoolArena<ByteBuffer>[] directArenas;
private final MemoryStatusThread statusThread;
private final int chunkSize;
public InnerAllocator() {
super(true);
try {
Field f = PooledByteBufAllocator.class.getDeclaredField("directArenas");
f.setAccessible(true);
this.directArenas = (PoolArena<ByteBuffer>[]) f.get(this);
} catch (Exception e) {
throw new RuntimeException("Failure while initializing allocator. Unable to retrieve direct arenas field.", e);
}
this.chunkSize = directArenas[0].chunkSize;
if (memoryLogger.isTraceEnabled()) {
statusThread = new MemoryStatusThread();
statusThread.start();
} else {
statusThread = null;
}
}
private UnsafeDirectLittleEndian newDirectBufferL(int initialCapacity, int maxCapacity) {
PoolThreadCache cache = threadCache();
PoolArena<ByteBuffer> directArena = cache.directArena;
if (directArena != null) {
if (initialCapacity > directArena.chunkSize) {
// This is beyond chunk size so we'll allocate separately.
ByteBuf buf = UnpooledByteBufAllocator.DEFAULT.directBuffer(initialCapacity, maxCapacity);
hugeBufferSize.addAndGet(buf.capacity());
hugeBufferCount.incrementAndGet();
// logger.debug("Allocating huge buffer of size {}", initialCapacity, new Exception());
return new AccountedUnsafeDirectLittleEndian(new LargeBuffer(buf), hugeBufferCount,
hugeBufferSize);
} else {
// within chunk, use arena.
ByteBuf buf = directArena.allocate(cache, initialCapacity, maxCapacity);
if (!(buf instanceof PooledUnsafeDirectByteBuf)) {
fail();
}
if (!ASSERT_ENABLED) {
return new UnsafeDirectLittleEndian((PooledUnsafeDirectByteBuf) buf);
}
normalBufferSize.addAndGet(buf.capacity());
normalBufferCount.incrementAndGet();
return new AccountedUnsafeDirectLittleEndian((PooledUnsafeDirectByteBuf) buf,
normalBufferCount, normalBufferSize);
}
} else {
throw fail();
}
}
private UnsupportedOperationException fail() {
return new UnsupportedOperationException(
"Arrow requires that the JVM used supports access sun.misc.Unsafe. This platform " +
"didn't provide that functionality.");
}
@Override
public UnsafeDirectLittleEndian directBuffer(int initialCapacity, int maxCapacity) {
if (initialCapacity == 0 && maxCapacity == 0) {
newDirectBuffer(initialCapacity, maxCapacity);
}
validate(initialCapacity, maxCapacity);
return newDirectBufferL(initialCapacity, maxCapacity);
}
@Override
public ByteBuf heapBuffer(int initialCapacity, int maxCapacity) {
throw new UnsupportedOperationException("Arrow doesn't support using heap buffers.");
}
private void validate(int initialCapacity, int maxCapacity) {
if (initialCapacity < 0) {
throw new IllegalArgumentException("initialCapacity: " + initialCapacity + " (expected: 0+)");
}
if (initialCapacity > maxCapacity) {
throw new IllegalArgumentException(String.format(
"initialCapacity: %d (expected: not greater than maxCapacity(%d)",
initialCapacity, maxCapacity));
}
}
@Override
public String toString() {
StringBuilder buf = new StringBuilder();
buf.append(directArenas.length);
buf.append(" direct arena(s):");
buf.append(StringUtil.NEWLINE);
for (PoolArena<ByteBuffer> a : directArenas) {
buf.append(a);
}
buf.append("Large buffers outstanding: ");
buf.append(hugeBufferCount.get());
buf.append(" totaling ");
buf.append(hugeBufferSize.get());
buf.append(" bytes.");
buf.append('\n');
buf.append("Normal buffers outstanding: ");
buf.append(normalBufferCount.get());
buf.append(" totaling ");
buf.append(normalBufferSize.get());
buf.append(" bytes.");
return buf.toString();
}
private class MemoryStatusThread extends Thread {
public MemoryStatusThread() {
super("allocation.logger");
this.setDaemon(true);
}
@Override
public void run() {
while (true) {
memoryLogger.trace("Memory Usage: \n{}", PooledByteBufAllocatorL.this.toString());
try {
Thread.sleep(MEMORY_LOGGER_FREQUENCY_SECONDS * 1000);
} catch (InterruptedException e) {
return;
}
}
}
}
}
}