1919package org .apache .paimon .format .vortex ;
2020
2121import org .apache .paimon .arrow .ArrowBundleRecords ;
22+ import org .apache .paimon .arrow .ArrowUtils ;
23+ import org .apache .paimon .arrow .vector .ArrowCStruct ;
2224import org .apache .paimon .arrow .vector .ArrowFormatWriter ;
2325import org .apache .paimon .data .InternalRow ;
2426import org .apache .paimon .format .BundleFormatWriter ;
2830
2931import dev .vortex .api .DType ;
3032import dev .vortex .api .VortexWriter ;
33+ import org .apache .arrow .c .ArrowArray ;
34+ import org .apache .arrow .c .ArrowSchema ;
35+ import org .apache .arrow .memory .BufferAllocator ;
3136import org .apache .arrow .vector .VectorSchemaRoot ;
3237import org .slf4j .Logger ;
3338import org .slf4j .LoggerFactory ;
3439
3540import java .io .IOException ;
3641import java .util .Map ;
3742
38- import static org .apache .paimon .arrow .ArrowUtils .serializeToIpc ;
39-
4043/** Vortex records writer. */
4144public class VortexRecordsWriter implements BundleFormatWriter {
4245
@@ -45,7 +48,9 @@ public class VortexRecordsWriter implements BundleFormatWriter {
4548 private final ArrowFormatWriter arrowFormatWriter ;
4649 private final VortexWriter nativeWriter ;
4750 private final String path ;
48- private long bytesWritten = 0 ;
51+ private final ArrowArray arrowArray ;
52+ private final ArrowSchema arrowSchema ;
53+ private final BufferAllocator allocator ;
4954 private long jniCost = 0 ;
5055
5156 public VortexRecordsWriter (
@@ -56,6 +61,9 @@ public VortexRecordsWriter(
5661 throws IOException {
5762 this .arrowFormatWriter = arrowFormatWriter ;
5863 this .path = path .toUri ().toString ();
64+ this .allocator = arrowFormatWriter .getAllocator ();
65+ this .arrowArray = ArrowArray .allocateNew (allocator );
66+ this .arrowSchema = ArrowSchema .allocateNew (allocator );
5967
6068 DType dtype = VortexTypeUtils .toDType (rowType );
6169 this .nativeWriter = VortexWriter .create (this .path , dtype , storageOptions );
@@ -84,9 +92,9 @@ public void writeBundle(BundleRecords bundleRecords) throws IOException {
8492
8593 @ Override
8694 public boolean reachTargetSize (boolean suggestedCheck , long targetSize ) {
87- // Note: bytesWritten tracks Arrow IPC serialized bytes, not the actual Vortex file size
88- // (which may differ due to Vortex's own compression/encoding) .
89- return suggestedCheck && ( bytesWritten > targetSize ) ;
95+ // Vortex applies its own compression/encoding, so in-memory Arrow size is much larger
96+ // than the actual file size on disk. Always return false to avoid rolling into small files .
97+ return false ;
9098 }
9199
92100 @ Override
@@ -95,6 +103,8 @@ public void close() throws IOException {
95103 LOG .info ("Jni cost: {}ms for file: {}" , jniCost , path );
96104 long t1 = System .currentTimeMillis ();
97105 nativeWriter .close ();
106+ arrowArray .close ();
107+ arrowSchema .close ();
98108 arrowFormatWriter .close ();
99109 long closeCost = (System .currentTimeMillis () - t1 );
100110 LOG .info ("Close cost: {}ms for file: {}" , closeCost , path );
@@ -109,10 +119,10 @@ private void flush() throws IOException {
109119 }
110120
111121 private void writeVsr (VectorSchemaRoot vsr ) throws IOException {
112- byte [] arrowData = serializeToIpc ( vsr );
113- bytesWritten += arrowData . length ;
122+ ArrowCStruct cStruct =
123+ ArrowUtils . serializeToCStruct ( vsr , arrowArray , arrowSchema , allocator ) ;
114124 long t1 = System .currentTimeMillis ();
115- nativeWriter .writeBatch ( arrowData );
125+ nativeWriter .writeBatchFfi ( cStruct . arrayAddress (), cStruct . schemaAddress () );
116126 jniCost += (System .currentTimeMillis () - t1 );
117127 }
118128}
0 commit comments