github.com/segmentio/parquet-go@v0.0.0-20230712180008-5d42db8f0d47/buffer.go (about) 1 package parquet 2 3 import ( 4 "log" 5 "runtime" 6 "sort" 7 "sync" 8 "sync/atomic" 9 10 "github.com/segmentio/parquet-go/internal/debug" 11 ) 12 13 // Buffer represents an in-memory group of parquet rows. 14 // 15 // The main purpose of the Buffer type is to provide a way to sort rows before 16 // writing them to a parquet file. Buffer implements sort.Interface as a way 17 // to support reordering the rows that have been written to it. 18 type Buffer struct { 19 config *RowGroupConfig 20 schema *Schema 21 rowbuf []Row 22 colbuf [][]Value 23 chunks []ColumnChunk 24 columns []ColumnBuffer 25 sorted []ColumnBuffer 26 } 27 28 // NewBuffer constructs a new buffer, using the given list of buffer options 29 // to configure the buffer returned by the function. 30 // 31 // The function panics if the buffer configuration is invalid. Programs that 32 // cannot guarantee the validity of the options passed to NewBuffer should 33 // construct the buffer configuration independently prior to calling this 34 // function: 35 // 36 // config, err := parquet.NewRowGroupConfig(options...) 37 // if err != nil { 38 // // handle the configuration error 39 // ... 40 // } else { 41 // // this call to create a buffer is guaranteed not to panic 42 // buffer := parquet.NewBuffer(config) 43 // ... 44 // } 45 func NewBuffer(options ...RowGroupOption) *Buffer { 46 config, err := NewRowGroupConfig(options...) 47 if err != nil { 48 panic(err) 49 } 50 buf := &Buffer{ 51 config: config, 52 } 53 if config.Schema != nil { 54 buf.configure(config.Schema) 55 } 56 return buf 57 } 58 59 func (buf *Buffer) configure(schema *Schema) { 60 if schema == nil { 61 return 62 } 63 sortingColumns := buf.config.Sorting.SortingColumns 64 buf.sorted = make([]ColumnBuffer, len(sortingColumns)) 65 66 forEachLeafColumnOf(schema, func(leaf leafColumn) { 67 nullOrdering := nullsGoLast 68 columnIndex := int(leaf.columnIndex) 69 columnType := leaf.node.Type() 70 bufferCap := buf.config.ColumnBufferCapacity 71 dictionary := (Dictionary)(nil) 72 encoding := encodingOf(leaf.node) 73 74 if isDictionaryEncoding(encoding) { 75 estimatedDictBufferSize := columnType.EstimateSize(bufferCap) 76 dictBuffer := columnType.NewValues( 77 make([]byte, 0, estimatedDictBufferSize), 78 nil, 79 ) 80 dictionary = columnType.NewDictionary(columnIndex, 0, dictBuffer) 81 columnType = dictionary.Type() 82 } 83 84 sortingIndex := searchSortingColumn(sortingColumns, leaf.path) 85 if sortingIndex < len(sortingColumns) && sortingColumns[sortingIndex].NullsFirst() { 86 nullOrdering = nullsGoFirst 87 } 88 89 column := columnType.NewColumnBuffer(columnIndex, bufferCap) 90 switch { 91 case leaf.maxRepetitionLevel > 0: 92 column = newRepeatedColumnBuffer(column, leaf.maxRepetitionLevel, leaf.maxDefinitionLevel, nullOrdering) 93 case leaf.maxDefinitionLevel > 0: 94 column = newOptionalColumnBuffer(column, leaf.maxDefinitionLevel, nullOrdering) 95 } 96 buf.columns = append(buf.columns, column) 97 98 if sortingIndex < len(sortingColumns) { 99 if sortingColumns[sortingIndex].Descending() { 100 column = &reversedColumnBuffer{column} 101 } 102 buf.sorted[sortingIndex] = column 103 } 104 }) 105 106 buf.schema = schema 107 buf.rowbuf = make([]Row, 0, 1) 108 buf.colbuf = make([][]Value, len(buf.columns)) 109 buf.chunks = make([]ColumnChunk, len(buf.columns)) 110 111 for i, column := range buf.columns { 112 buf.chunks[i] = column 113 } 114 } 115 116 // Size returns the estimated size of the buffer in memory (in bytes). 117 func (buf *Buffer) Size() int64 { 118 size := int64(0) 119 for _, col := range buf.columns { 120 size += col.Size() 121 } 122 return size 123 } 124 125 // NumRows returns the number of rows written to the buffer. 126 func (buf *Buffer) NumRows() int64 { return int64(buf.Len()) } 127 128 // ColumnChunks returns the buffer columns. 129 func (buf *Buffer) ColumnChunks() []ColumnChunk { return buf.chunks } 130 131 // ColumnBuffer returns the buffer columns. 132 // 133 // This method is similar to ColumnChunks, but returns a list of ColumnBuffer 134 // instead of a ColumnChunk values (the latter being read-only); calling 135 // ColumnBuffers or ColumnChunks with the same index returns the same underlying 136 // objects, but with different types, which removes the need for making a type 137 // assertion if the program needed to write directly to the column buffers. 138 // The presence of the ColumnChunks method is still required to satisfy the 139 // RowGroup interface. 140 func (buf *Buffer) ColumnBuffers() []ColumnBuffer { return buf.columns } 141 142 // Schema returns the schema of the buffer. 143 // 144 // The schema is either configured by passing a Schema in the option list when 145 // constructing the buffer, or lazily discovered when the first row is written. 146 func (buf *Buffer) Schema() *Schema { return buf.schema } 147 148 // SortingColumns returns the list of columns by which the buffer will be 149 // sorted. 150 // 151 // The sorting order is configured by passing a SortingColumns option when 152 // constructing the buffer. 153 func (buf *Buffer) SortingColumns() []SortingColumn { return buf.config.Sorting.SortingColumns } 154 155 // Len returns the number of rows written to the buffer. 156 func (buf *Buffer) Len() int { 157 if len(buf.columns) == 0 { 158 return 0 159 } else { 160 // All columns have the same number of rows. 161 return buf.columns[0].Len() 162 } 163 } 164 165 // Less returns true if row[i] < row[j] in the buffer. 166 func (buf *Buffer) Less(i, j int) bool { 167 for _, col := range buf.sorted { 168 switch { 169 case col.Less(i, j): 170 return true 171 case col.Less(j, i): 172 return false 173 } 174 } 175 return false 176 } 177 178 // Swap exchanges the rows at indexes i and j. 179 func (buf *Buffer) Swap(i, j int) { 180 for _, col := range buf.columns { 181 col.Swap(i, j) 182 } 183 } 184 185 // Reset clears the content of the buffer, allowing it to be reused. 186 func (buf *Buffer) Reset() { 187 for _, col := range buf.columns { 188 col.Reset() 189 } 190 } 191 192 // Write writes a row held in a Go value to the buffer. 193 func (buf *Buffer) Write(row interface{}) error { 194 if buf.schema == nil { 195 buf.configure(SchemaOf(row)) 196 } 197 198 buf.rowbuf = buf.rowbuf[:1] 199 defer clearRows(buf.rowbuf) 200 201 buf.rowbuf[0] = buf.schema.Deconstruct(buf.rowbuf[0], row) 202 _, err := buf.WriteRows(buf.rowbuf) 203 return err 204 } 205 206 // WriteRows writes parquet rows to the buffer. 207 func (buf *Buffer) WriteRows(rows []Row) (int, error) { 208 defer func() { 209 for i, colbuf := range buf.colbuf { 210 clearValues(colbuf) 211 buf.colbuf[i] = colbuf[:0] 212 } 213 }() 214 215 if buf.schema == nil { 216 return 0, ErrRowGroupSchemaMissing 217 } 218 219 for _, row := range rows { 220 for _, value := range row { 221 columnIndex := value.Column() 222 buf.colbuf[columnIndex] = append(buf.colbuf[columnIndex], value) 223 } 224 } 225 226 for columnIndex, values := range buf.colbuf { 227 if _, err := buf.columns[columnIndex].WriteValues(values); err != nil { 228 // TODO: an error at this stage will leave the buffer in an invalid 229 // state since the row was partially written. Applications are not 230 // expected to continue using the buffer after getting an error, 231 // maybe we can enforce it? 232 return 0, err 233 } 234 } 235 236 return len(rows), nil 237 } 238 239 // WriteRowGroup satisfies the RowGroupWriter interface. 240 func (buf *Buffer) WriteRowGroup(rowGroup RowGroup) (int64, error) { 241 rowGroupSchema := rowGroup.Schema() 242 switch { 243 case rowGroupSchema == nil: 244 return 0, ErrRowGroupSchemaMissing 245 case buf.schema == nil: 246 buf.configure(rowGroupSchema) 247 case !nodesAreEqual(buf.schema, rowGroupSchema): 248 return 0, ErrRowGroupSchemaMismatch 249 } 250 if !sortingColumnsHavePrefix(rowGroup.SortingColumns(), buf.SortingColumns()) { 251 return 0, ErrRowGroupSortingColumnsMismatch 252 } 253 n := buf.NumRows() 254 r := rowGroup.Rows() 255 defer r.Close() 256 _, err := CopyRows(bufferWriter{buf}, r) 257 return buf.NumRows() - n, err 258 } 259 260 // Rows returns a reader exposing the current content of the buffer. 261 // 262 // The buffer and the returned reader share memory. Mutating the buffer 263 // concurrently to reading rows may result in non-deterministic behavior. 264 func (buf *Buffer) Rows() Rows { return newRowGroupRows(buf, ReadModeSync) } 265 266 // bufferWriter is an adapter for Buffer which implements both RowWriter and 267 // PageWriter to enable optimizations in CopyRows for types that support writing 268 // rows by copying whole pages instead of calling WriteRow repeatedly. 269 type bufferWriter struct{ buf *Buffer } 270 271 func (w bufferWriter) WriteRows(rows []Row) (int, error) { 272 return w.buf.WriteRows(rows) 273 } 274 275 func (w bufferWriter) WriteValues(values []Value) (int, error) { 276 return w.buf.columns[values[0].Column()].WriteValues(values) 277 } 278 279 func (w bufferWriter) WritePage(page Page) (int64, error) { 280 return CopyValues(w.buf.columns[page.Column()], page.Values()) 281 } 282 283 var ( 284 _ RowGroup = (*Buffer)(nil) 285 _ RowGroupWriter = (*Buffer)(nil) 286 _ sort.Interface = (*Buffer)(nil) 287 288 _ RowWriter = (*bufferWriter)(nil) 289 _ PageWriter = (*bufferWriter)(nil) 290 _ ValueWriter = (*bufferWriter)(nil) 291 ) 292 293 type buffer struct { 294 data []byte 295 refc uintptr 296 pool *bufferPool 297 stack []byte 298 } 299 300 func (b *buffer) refCount() int { 301 return int(atomic.LoadUintptr(&b.refc)) 302 } 303 304 func (b *buffer) ref() { 305 atomic.AddUintptr(&b.refc, +1) 306 } 307 308 func (b *buffer) unref() { 309 if atomic.AddUintptr(&b.refc, ^uintptr(0)) == 0 { 310 if b.pool != nil { 311 b.pool.put(b) 312 } 313 } 314 } 315 316 func monitorBufferRelease(b *buffer) { 317 if rc := b.refCount(); rc != 0 { 318 log.Printf("PARQUETGODEBUG: buffer garbage collected with non-zero reference count\n%s", string(b.stack)) 319 } 320 } 321 322 type bufferPool struct { 323 // Buckets are split in two groups for short and large buffers. In the short 324 // buffer group (below 256KB), the growth rate between each bucket is 2. The 325 // growth rate changes to 1.5 in the larger buffer group. 326 // 327 // Short buffer buckets: 328 // --------------------- 329 // 4K, 8K, 16K, 32K, 64K, 128K, 256K 330 // 331 // Large buffer buckets: 332 // --------------------- 333 // 364K, 546K, 819K ... 334 // 335 buckets [bufferPoolBucketCount]sync.Pool 336 } 337 338 func (p *bufferPool) newBuffer(bufferSize, bucketSize int) *buffer { 339 b := &buffer{ 340 data: make([]byte, bufferSize, bucketSize), 341 refc: 1, 342 pool: p, 343 } 344 if debug.TRACEBUF > 0 { 345 b.stack = make([]byte, 4096) 346 runtime.SetFinalizer(b, monitorBufferRelease) 347 } 348 return b 349 } 350 351 // get returns a buffer from the levelled buffer pool. size is used to choose 352 // the appropriate pool. 353 func (p *bufferPool) get(bufferSize int) *buffer { 354 bucketIndex, bucketSize := bufferPoolBucketIndexAndSizeOfGet(bufferSize) 355 356 b := (*buffer)(nil) 357 if bucketIndex >= 0 { 358 b, _ = p.buckets[bucketIndex].Get().(*buffer) 359 } 360 361 if b == nil { 362 b = p.newBuffer(bufferSize, bucketSize) 363 } else { 364 b.data = b.data[:bufferSize] 365 b.ref() 366 } 367 368 if debug.TRACEBUF > 0 { 369 b.stack = b.stack[:runtime.Stack(b.stack[:cap(b.stack)], false)] 370 } 371 return b 372 } 373 374 func (p *bufferPool) put(b *buffer) { 375 if b.pool != p { 376 panic("BUG: buffer returned to a different pool than the one it was allocated from") 377 } 378 if b.refCount() != 0 { 379 panic("BUG: buffer returned to pool with a non-zero reference count") 380 } 381 if bucketIndex, _ := bufferPoolBucketIndexAndSizeOfPut(cap(b.data)); bucketIndex >= 0 { 382 p.buckets[bucketIndex].Put(b) 383 } 384 } 385 386 const ( 387 bufferPoolBucketCount = 32 388 bufferPoolMinSize = 4096 389 bufferPoolLastShortBucketSize = 262144 390 ) 391 392 func bufferPoolNextSize(size int) int { 393 if size < bufferPoolLastShortBucketSize { 394 return size * 2 395 } else { 396 return size + (size / 2) 397 } 398 } 399 400 func bufferPoolBucketIndexAndSizeOfGet(size int) (int, int) { 401 limit := bufferPoolMinSize 402 403 for i := 0; i < bufferPoolBucketCount; i++ { 404 if size <= limit { 405 return i, limit 406 } 407 limit = bufferPoolNextSize(limit) 408 } 409 410 return -1, size 411 } 412 413 func bufferPoolBucketIndexAndSizeOfPut(size int) (int, int) { 414 // When releasing buffers, some may have a capacity that is not one of the 415 // bucket sizes (due to the use of append for example). In this case, we 416 // have to put the buffer is the highest bucket with a size less or equal 417 // to the buffer capacity. 418 if limit := bufferPoolMinSize; size >= limit { 419 for i := 0; i < bufferPoolBucketCount; i++ { 420 n := bufferPoolNextSize(limit) 421 if size < n { 422 return i, limit 423 } 424 limit = n 425 } 426 } 427 return -1, size 428 } 429 430 var ( 431 buffers bufferPool 432 ) 433 434 type bufferedPage struct { 435 Page 436 values *buffer 437 offsets *buffer 438 repetitionLevels *buffer 439 definitionLevels *buffer 440 } 441 442 func newBufferedPage(page Page, values, offsets, definitionLevels, repetitionLevels *buffer) *bufferedPage { 443 p := &bufferedPage{ 444 Page: page, 445 values: values, 446 offsets: offsets, 447 definitionLevels: definitionLevels, 448 repetitionLevels: repetitionLevels, 449 } 450 bufferRef(values) 451 bufferRef(offsets) 452 bufferRef(definitionLevels) 453 bufferRef(repetitionLevels) 454 return p 455 } 456 457 func (p *bufferedPage) Slice(i, j int64) Page { 458 return newBufferedPage( 459 p.Page.Slice(i, j), 460 p.values, 461 p.offsets, 462 p.definitionLevels, 463 p.repetitionLevels, 464 ) 465 } 466 467 func (p *bufferedPage) Retain() { 468 bufferRef(p.values) 469 bufferRef(p.offsets) 470 bufferRef(p.definitionLevels) 471 bufferRef(p.repetitionLevels) 472 } 473 474 func (p *bufferedPage) Release() { 475 bufferUnref(p.values) 476 bufferUnref(p.offsets) 477 bufferUnref(p.definitionLevels) 478 bufferUnref(p.repetitionLevels) 479 } 480 481 func bufferRef(buf *buffer) { 482 if buf != nil { 483 buf.ref() 484 } 485 } 486 487 func bufferUnref(buf *buffer) { 488 if buf != nil { 489 buf.unref() 490 } 491 } 492 493 // Retain is a helper function to increment the reference counter of pages 494 // backed by memory which can be granularly managed by the application. 495 // 496 // Usage of this function is optional and with Release, is intended to allow 497 // finer grain memory management in the application. Most programs should be 498 // able to rely on automated memory management provided by the Go garbage 499 // collector instead. 500 // 501 // The function should be called when a page lifetime is about to be shared 502 // between multiple goroutines or layers of an application, and the program 503 // wants to express "sharing ownership" of the page. 504 // 505 // Calling this function on pages that do not embed a reference counter does 506 // nothing. 507 func Retain(page Page) { 508 if p, _ := page.(retainable); p != nil { 509 p.Retain() 510 } 511 } 512 513 // Release is a helper function to decrement the reference counter of pages 514 // backed by memory which can be granularly managed by the application. 515 // 516 // Usage of this is optional and with Retain, is intended to allow finer grained 517 // memory management in the application, at the expense of potentially causing 518 // panics if the page is used after its reference count has reached zero. Most 519 // programs should be able to rely on automated memory management provided by 520 // the Go garbage collector instead. 521 // 522 // The function should be called to return a page to the internal buffer pool, 523 // when a goroutine "releases ownership" it acquired either by being the single 524 // owner (e.g. capturing the return value from a ReadPage call) or having gotten 525 // shared ownership by calling Retain. 526 // 527 // Calling this function on pages that do not embed a reference counter does 528 // nothing. 529 func Release(page Page) { 530 if p, _ := page.(releasable); p != nil { 531 p.Release() 532 } 533 } 534 535 type retainable interface { 536 Retain() 537 } 538 539 type releasable interface { 540 Release() 541 } 542 543 var ( 544 _ retainable = (*bufferedPage)(nil) 545 _ releasable = (*bufferedPage)(nil) 546 )