2222import java .util .Collections ;
2323import java .util .List ;
2424
25- import org .apache .arrow .memory .BaseAllocator ;
2625import org .apache .arrow .memory .BufferAllocator ;
2726import org .apache .arrow .memory .OutOfMemoryException ;
2827import org .apache .arrow .vector .ipc .message .ArrowFieldNode ;
@@ -43,8 +42,7 @@ public abstract class BaseFixedWidthVector extends BaseValueVector
4342 implements FixedWidthVector , FieldVector , VectorDefinitionSetter {
4443 private final int typeWidth ;
4544
46- protected int valueAllocationSizeInBytes ;
47- protected int validityAllocationSizeInBytes ;
45+ protected int initialValueAllocation ;
4846
4947 protected final Field field ;
5048 private int allocationMonitor ;
@@ -61,14 +59,7 @@ public BaseFixedWidthVector(final String name, final BufferAllocator allocator,
6159 allocationMonitor = 0 ;
6260 validityBuffer = allocator .getEmpty ();
6361 valueBuffer = allocator .getEmpty ();
64- if (typeWidth > 0 ) {
65- valueAllocationSizeInBytes = INITIAL_VALUE_ALLOCATION * typeWidth ;
66- validityAllocationSizeInBytes = getValidityBufferSizeFromCount (INITIAL_VALUE_ALLOCATION );
67- } else {
68- /* specialized handling for BitVector */
69- valueAllocationSizeInBytes = getValidityBufferSizeFromCount (INITIAL_VALUE_ALLOCATION );
70- validityAllocationSizeInBytes = valueAllocationSizeInBytes ;
71- }
62+ initialValueAllocation = INITIAL_VALUE_ALLOCATION ;
7263 }
7364
7465
@@ -159,12 +150,8 @@ public ArrowBuf getOffsetBuffer() {
159150 */
160151 @ Override
161152 public void setInitialCapacity (int valueCount ) {
162- final long size = (long ) valueCount * typeWidth ;
163- if (size > MAX_ALLOCATION_SIZE ) {
164- throw new OversizedAllocationException ("Requested amount of memory is more than max allowed" );
165- }
166- valueAllocationSizeInBytes = (int ) size ;
167- validityAllocationSizeInBytes = getValidityBufferSizeFromCount (valueCount );
153+ computeAndCheckBufferSize (valueCount );
154+ initialValueAllocation = valueCount ;
168155 }
169156
170157 /**
@@ -267,18 +254,13 @@ public void allocateNew() {
267254 */
268255 @ Override
269256 public boolean allocateNewSafe () {
270- long curAllocationSizeValue = valueAllocationSizeInBytes ;
271- long curAllocationSizeValidity = validityAllocationSizeInBytes ;
272-
273- if (align (curAllocationSizeValue ) + curAllocationSizeValidity > MAX_ALLOCATION_SIZE ) {
274- throw new OversizedAllocationException ("Requested amount of memory exceeds limit" );
275- }
257+ computeAndCheckBufferSize (initialValueAllocation );
276258
277259 /* we are doing a new allocation -- release the current buffers */
278260 clear ();
279261
280262 try {
281- allocateBytes (curAllocationSizeValue , curAllocationSizeValidity );
263+ allocateBytes (initialValueAllocation );
282264 } catch (Exception e ) {
283265 clear ();
284266 return false ;
@@ -295,33 +277,30 @@ public boolean allocateNewSafe() {
295277 * @throws org.apache.arrow.memory.OutOfMemoryException on error
296278 */
297279 public void allocateNew (int valueCount ) {
298- long valueBufferSize = valueCount * typeWidth ;
299- long validityBufferSize = getValidityBufferSizeFromCount (valueCount );
300- if (typeWidth == 0 ) {
301- /* specialized handling for BitVector */
302- valueBufferSize = validityBufferSize ;
303- }
304-
305- if (align (valueBufferSize ) + validityBufferSize > MAX_ALLOCATION_SIZE ) {
306- throw new OversizedAllocationException ("Requested amount of memory is more than max allowed" );
307- }
280+ computeAndCheckBufferSize (valueCount );
308281
309282 /* we are doing a new allocation -- release the current buffers */
310283 clear ();
311284
312285 try {
313- allocateBytes (valueBufferSize , validityBufferSize );
286+ allocateBytes (valueCount );
314287 } catch (Exception e ) {
315288 clear ();
316289 throw e ;
317290 }
318291 }
319292
320293 /*
321- * align to a 8-byte value .
294+ * Compute the buffer size required for 'valueCount', and check if it's within bounds .
322295 */
323- private long align (long size ) {
324- return ((size + 7 ) / 8 ) * 8 ;
296+ private long computeAndCheckBufferSize (int valueCount ) {
297+ final long size = computeCombinedBufferSize (valueCount , typeWidth );
298+ if (size > MAX_ALLOCATION_SIZE ) {
299+ throw new OversizedAllocationException ("Memory required for vector capacity " +
300+ valueCount +
301+ " is (" + size + "), which is more than max allowed (" + MAX_ALLOCATION_SIZE + ")" );
302+ }
303+ return size ;
325304 }
326305
327306 /**
@@ -333,25 +312,11 @@ private long align(long size) {
333312 * within the bounds of max allocation allowed and any other error
334313 * conditions.
335314 */
336- private void allocateBytes (final long valueBufferSize , final long validityBufferSize ) {
337- int valueBufferSlice = (int )align (valueBufferSize );
338- int validityBufferSlice = (int )validityBufferSize ;
339-
340- /* allocate combined buffer */
341- ArrowBuf buffer = allocator .buffer (valueBufferSlice + validityBufferSlice );
342-
343- valueAllocationSizeInBytes = valueBufferSlice ;
344- valueBuffer = buffer .slice (0 , valueBufferSlice );
345- valueBuffer .retain ();
346- valueBuffer .readerIndex (0 );
347-
348- validityAllocationSizeInBytes = validityBufferSlice ;
349- validityBuffer = buffer .slice (valueBufferSlice , validityBufferSlice );
350- validityBuffer .retain ();
351- validityBuffer .readerIndex (0 );
315+ private void allocateBytes (int valueCount ) {
316+ DataAndValidityBuffers buffers = allocFixedDataAndValidityBufs (valueCount , typeWidth );
317+ valueBuffer = buffers .getDataBuf ();
318+ validityBuffer = buffers .getValidityBuf ();
352319 zeroVector ();
353-
354- buffer .release ();
355320 }
356321
357322 /**
@@ -363,7 +328,6 @@ private void allocateBytes(final long valueBufferSize, final long validityBuffer
363328 private void allocateValidityBuffer (final int validityBufferSize ) {
364329 validityBuffer = allocator .buffer (validityBufferSize );
365330 validityBuffer .readerIndex (0 );
366- validityAllocationSizeInBytes = validityBufferSize ;
367331 }
368332
369333 /**
@@ -439,50 +403,28 @@ public ArrowBuf[] getBuffers(boolean clear) {
439403 */
440404 @ Override
441405 public void reAlloc () {
442- int valueBaseSize = Integer .max (valueBuffer .capacity (), valueAllocationSizeInBytes );
443- long newValueBufferSlice = align (valueBaseSize * 2L );
444- long newValidityBufferSlice ;
445- if (typeWidth > 0 ) {
446- long targetValueBufferSize = align (BaseAllocator .nextPowerOfTwo (newValueBufferSlice ));
447- long targetValueCount = targetValueBufferSize / typeWidth ;
448- targetValueBufferSize -= getValidityBufferSizeFromCount ((int ) targetValueCount );
449- if (newValueBufferSlice < targetValueBufferSize ) {
450- newValueBufferSlice = targetValueBufferSize ;
406+ int targetValueCount = getValueCapacity () * 2 ;
407+ if (targetValueCount == 0 ) {
408+ if (initialValueAllocation > 0 ) {
409+ targetValueCount = initialValueAllocation * 2 ;
410+ } else {
411+ targetValueCount = INITIAL_VALUE_ALLOCATION * 2 ;
451412 }
452-
453- newValidityBufferSlice = getValidityBufferSizeFromCount ((int )(newValueBufferSlice / typeWidth ));
454- } else {
455- newValidityBufferSlice = newValueBufferSlice ;
456- }
457-
458- long newAllocationSize = newValueBufferSlice + newValidityBufferSlice ;
459- assert newAllocationSize >= 1 ;
460-
461- if (newAllocationSize > MAX_ALLOCATION_SIZE ) {
462- throw new OversizedAllocationException ("Unable to expand the buffer" );
463413 }
414+ computeAndCheckBufferSize (targetValueCount );
464415
465- final ArrowBuf newBuffer = allocator . buffer (( int ) newAllocationSize );
466- final ArrowBuf newValueBuffer = newBuffer . slice ( 0 , ( int ) newValueBufferSlice );
416+ DataAndValidityBuffers buffers = allocFixedDataAndValidityBufs ( targetValueCount , typeWidth );
417+ final ArrowBuf newValueBuffer = buffers . getDataBuf ( );
467418 newValueBuffer .setBytes (0 , valueBuffer , 0 , valueBuffer .capacity ());
468- newValueBuffer .setZero (valueBuffer .capacity (), (int )newValueBufferSlice - valueBuffer .capacity ());
469- newValueBuffer .retain ();
470- newValueBuffer .readerIndex (0 );
419+ newValueBuffer .setZero (valueBuffer .capacity (), newValueBuffer .capacity () - valueBuffer .capacity ());
471420 valueBuffer .release ();
472421 valueBuffer = newValueBuffer ;
473- valueAllocationSizeInBytes = (int )newValueBufferSlice ;
474422
475- final ArrowBuf newValidityBuffer = newBuffer .slice ((int )newValueBufferSlice ,
476- (int )newValidityBufferSlice );
423+ final ArrowBuf newValidityBuffer = buffers .getValidityBuf ();
477424 newValidityBuffer .setBytes (0 , validityBuffer , 0 , validityBuffer .capacity ());
478- newValidityBuffer .setZero (validityBuffer .capacity (), (int )newValidityBufferSlice - validityBuffer .capacity ());
479- newValidityBuffer .retain ();
480- newValidityBuffer .readerIndex (0 );
425+ newValidityBuffer .setZero (validityBuffer .capacity (), newValidityBuffer .capacity () - validityBuffer .capacity ());
481426 validityBuffer .release ();
482427 validityBuffer = newValidityBuffer ;
483- validityAllocationSizeInBytes = (int )newValidityBufferSlice ;
484-
485- newBuffer .release ();
486428 }
487429
488430 @ Override
@@ -535,9 +477,6 @@ public void loadFieldBuffers(ArrowFieldNode fieldNode, List<ArrowBuf> ownBuffers
535477 valueBuffer = dataBuffer .retain (allocator );
536478
537479 valueCount = fieldNode .getLength ();
538-
539- valueAllocationSizeInBytes = valueBuffer .capacity ();
540- validityAllocationSizeInBytes = validityBuffer .capacity ();
541480 }
542481
543482 /**
0 commit comments