github.com/vc42/parquet-go@v0.0.0-20240320194221-1a9adb5f23f5/writer.go (about) 1 package parquet 2 3 import ( 4 "bufio" 5 "bytes" 6 "encoding/binary" 7 "fmt" 8 "hash/crc32" 9 "io" 10 "math" 11 "math/bits" 12 "sort" 13 14 "github.com/segmentio/encoding/thrift" 15 "github.com/vc42/parquet-go/compress" 16 "github.com/vc42/parquet-go/encoding" 17 "github.com/vc42/parquet-go/encoding/plain" 18 "github.com/vc42/parquet-go/format" 19 ) 20 21 // Deprecated: A Writer uses a parquet schema and sequence of Go values to 22 // produce a parquet file to an io.Writer. 23 // 24 // This example showcases a typical use of parquet writers: 25 // 26 // writer := parquet.NewWriter(output) 27 // 28 // for _, row := range rows { 29 // if err := writer.Write(row); err != nil { 30 // ... 31 // } 32 // } 33 // 34 // if err := writer.Close(); err != nil { 35 // ... 36 // } 37 // 38 // The Writer type optimizes for minimal memory usage, each page is written as 39 // soon as it has been filled so only a single page per column needs to be held 40 // in memory and as a result, there are no opportunities to sort rows within an 41 // entire row group. Programs that need to produce parquet files with sorted 42 // row groups should use the Buffer type to buffer and sort the rows prior to 43 // writing them to a Writer. 44 // 45 // For programs building with Go 1.18 or later, the GenericWriter[T] type 46 // supersedes this one. 47 type Writer struct { 48 output io.Writer 49 config *WriterConfig 50 schema *Schema 51 writer *writer 52 rowbuf []Row 53 } 54 55 // NewWriter constructs a parquet writer writing a file to the given io.Writer. 56 // 57 // The function panics if the writer configuration is invalid. Programs that 58 // cannot guarantee the validity of the options passed to NewWriter should 59 // construct the writer configuration independently prior to calling this 60 // function: 61 // 62 // config, err := parquet.NewWriterConfig(options...) 63 // if err != nil { 64 // // handle the configuration error 65 // ... 66 // } else { 67 // // this call to create a writer is guaranteed not to panic 68 // writer := parquet.NewWriter(output, config) 69 // ... 70 // } 71 // 72 func NewWriter(output io.Writer, options ...WriterOption) *Writer { 73 config, err := NewWriterConfig(options...) 74 if err != nil { 75 panic(err) 76 } 77 w := &Writer{ 78 output: output, 79 config: config, 80 } 81 if config.Schema != nil { 82 w.configure(config.Schema) 83 } 84 return w 85 } 86 87 func (w *Writer) configure(schema *Schema) { 88 if schema != nil { 89 w.config.Schema = schema 90 w.schema = schema 91 w.writer = newWriter(w.output, w.config) 92 } 93 } 94 95 // Close must be called after all values were produced to the writer in order to 96 // flush all buffers and write the parquet footer. 97 func (w *Writer) Close() error { 98 if w.writer != nil { 99 return w.writer.close() 100 } 101 return nil 102 } 103 104 // Flush flushes all buffers into a row group to the underlying io.Writer. 105 // 106 // Flush is called automatically on Close, it is only useful to call explicitly 107 // if the application needs to limit the size of row groups or wants to produce 108 // multiple row groups per file. 109 func (w *Writer) Flush() error { 110 if w.writer != nil { 111 return w.writer.flush() 112 } 113 return nil 114 } 115 116 // Reset clears the state of the writer without flushing any of the buffers, 117 // and setting the output to the io.Writer passed as argument, allowing the 118 // writer to be reused to produce another parquet file. 119 // 120 // Reset may be called at any time, including after a writer was closed. 121 func (w *Writer) Reset(output io.Writer) { 122 if w.output = output; w.writer != nil { 123 w.writer.reset(w.output) 124 } 125 } 126 127 // Write is called to write another row to the parquet file. 128 // 129 // The method uses the parquet schema configured on w to traverse the Go value 130 // and decompose it into a set of columns and values. If no schema were passed 131 // to NewWriter, it is deducted from the Go type of the row, which then have to 132 // be a struct or pointer to struct. 133 func (w *Writer) Write(row interface{}) error { 134 if w.schema == nil { 135 w.configure(SchemaOf(row)) 136 } 137 if cap(w.rowbuf) == 0 { 138 w.rowbuf = make([]Row, 1) 139 } else { 140 w.rowbuf = w.rowbuf[:1] 141 } 142 defer clearRows(w.rowbuf) 143 w.rowbuf[0] = w.schema.Deconstruct(w.rowbuf[0][:0], row) 144 _, err := w.WriteRows(w.rowbuf) 145 return err 146 } 147 148 // WriteRows is called to write rows to the parquet file. 149 // 150 // The Writer must have been given a schema when NewWriter was called, otherwise 151 // the structure of the parquet file cannot be determined from the row only. 152 // 153 // The row is expected to contain values for each column of the writer's schema, 154 // in the order produced by the parquet.(*Schema).Deconstruct method. 155 func (w *Writer) WriteRows(rows []Row) (int, error) { 156 return w.writer.WriteRows(rows) 157 } 158 159 // WriteRowGroup writes a row group to the parquet file. 160 // 161 // Buffered rows will be flushed prior to writing rows from the group, unless 162 // the row group was empty in which case nothing is written to the file. 163 // 164 // The content of the row group is flushed to the writer; after the method 165 // returns successfully, the row group will be empty and in ready to be reused. 166 func (w *Writer) WriteRowGroup(rowGroup RowGroup) (int64, error) { 167 rowGroupSchema := rowGroup.Schema() 168 switch { 169 case rowGroupSchema == nil: 170 return 0, ErrRowGroupSchemaMissing 171 case w.schema == nil: 172 w.configure(rowGroupSchema) 173 case !nodesAreEqual(w.schema, rowGroupSchema): 174 return 0, ErrRowGroupSchemaMismatch 175 } 176 if err := w.writer.flush(); err != nil { 177 return 0, err 178 } 179 w.writer.configureBloomFilters(rowGroup.ColumnChunks()) 180 n, err := CopyRows(w.writer, rowGroup.Rows()) 181 if err != nil { 182 return n, err 183 } 184 return w.writer.writeRowGroup(rowGroup.Schema(), rowGroup.SortingColumns()) 185 } 186 187 // ReadRowsFrom reads rows from the reader passed as arguments and writes them 188 // to w. 189 // 190 // This is similar to calling WriteRow repeatedly, but will be more efficient 191 // if optimizations are supported by the reader. 192 func (w *Writer) ReadRowsFrom(rows RowReader) (written int64, err error) { 193 if w.schema == nil { 194 if r, ok := rows.(RowReaderWithSchema); ok { 195 w.configure(r.Schema()) 196 } 197 } 198 if cap(w.rowbuf) < defaultRowBufferSize { 199 w.rowbuf = make([]Row, defaultRowBufferSize) 200 } else { 201 w.rowbuf = w.rowbuf[:cap(w.rowbuf)] 202 } 203 return copyRows(w.writer, rows, w.rowbuf) 204 } 205 206 // Schema returns the schema of rows written by w. 207 // 208 // The returned value will be nil if no schema has yet been configured on w. 209 func (w *Writer) Schema() *Schema { return w.schema } 210 211 type writer struct { 212 buffer *bufio.Writer 213 writer offsetTrackingWriter 214 values [][]Value 215 216 createdBy string 217 metadata []format.KeyValue 218 219 columns []*writerColumn 220 columnChunk []format.ColumnChunk 221 columnIndex []format.ColumnIndex 222 offsetIndex []format.OffsetIndex 223 encodingStats [][]format.PageEncodingStats 224 225 columnOrders []format.ColumnOrder 226 schemaElements []format.SchemaElement 227 rowGroups []format.RowGroup 228 columnIndexes [][]format.ColumnIndex 229 offsetIndexes [][]format.OffsetIndex 230 sortingColumns []format.SortingColumn 231 } 232 233 func newWriter(output io.Writer, config *WriterConfig) *writer { 234 w := new(writer) 235 if config.WriteBufferSize <= 0 { 236 w.writer.Reset(output) 237 } else { 238 w.buffer = bufio.NewWriterSize(output, config.WriteBufferSize) 239 w.writer.Reset(w.buffer) 240 } 241 w.createdBy = config.CreatedBy 242 w.metadata = make([]format.KeyValue, 0, len(config.KeyValueMetadata)) 243 for k, v := range config.KeyValueMetadata { 244 w.metadata = append(w.metadata, format.KeyValue{Key: k, Value: v}) 245 } 246 sortKeyValueMetadata(w.metadata) 247 w.sortingColumns = make([]format.SortingColumn, len(config.SortingColumns)) 248 249 config.Schema.forEachNode(func(name string, node Node) { 250 nodeType := node.Type() 251 252 repetitionType := (*format.FieldRepetitionType)(nil) 253 if node != config.Schema { // the root has no repetition type 254 repetitionType = fieldRepetitionTypePtrOf(node) 255 } 256 257 // For backward compatibility with older readers, the parquet specification 258 // recommends to set the scale and precision on schema elements when the 259 // column is of logical type decimal. 260 logicalType := nodeType.LogicalType() 261 scale, precision := (*int32)(nil), (*int32)(nil) 262 if logicalType != nil && logicalType.Decimal != nil { 263 scale = &logicalType.Decimal.Scale 264 precision = &logicalType.Decimal.Precision 265 } 266 267 typeLength := (*int32)(nil) 268 if n := int32(nodeType.Length()); n > 0 { 269 typeLength = &n 270 } 271 272 w.schemaElements = append(w.schemaElements, format.SchemaElement{ 273 Type: nodeType.PhysicalType(), 274 TypeLength: typeLength, 275 RepetitionType: repetitionType, 276 Name: name, 277 NumChildren: int32(len(node.Fields())), 278 ConvertedType: nodeType.ConvertedType(), 279 Scale: scale, 280 Precision: precision, 281 LogicalType: logicalType, 282 }) 283 }) 284 285 dataPageType := format.DataPage 286 if config.DataPageVersion == 2 { 287 dataPageType = format.DataPageV2 288 } 289 290 defaultCompression := config.Compression 291 if defaultCompression == nil { 292 defaultCompression = &Uncompressed 293 } 294 295 // Those buffers are scratch space used to generate the page header and 296 // content, they are shared by all column chunks because they are only 297 // used during calls to writeDictionaryPage or writeDataPage, which are 298 // not done concurrently. 299 buffers := new(writerBuffers) 300 301 forEachLeafColumnOf(config.Schema, func(leaf leafColumn) { 302 encoding := encodingOf(leaf.node) 303 dictionary := Dictionary(nil) 304 columnType := leaf.node.Type() 305 columnIndex := int(leaf.columnIndex) 306 compression := leaf.node.Compression() 307 308 if compression == nil { 309 compression = defaultCompression 310 } 311 312 if isDictionaryEncoding(encoding) { 313 dictionary = columnType.NewDictionary(columnIndex, 0, make([]byte, 0, defaultDictBufferSize)) 314 columnType = dictionary.Type() 315 } 316 317 c := &writerColumn{ 318 buffers: buffers, 319 pool: config.ColumnPageBuffers, 320 columnPath: leaf.path, 321 columnType: columnType, 322 columnIndex: columnType.NewColumnIndexer(config.ColumnIndexSizeLimit), 323 columnFilter: searchBloomFilterColumn(config.BloomFilters, leaf.path), 324 compression: compression, 325 dictionary: dictionary, 326 dataPageType: dataPageType, 327 maxRepetitionLevel: leaf.maxRepetitionLevel, 328 maxDefinitionLevel: leaf.maxDefinitionLevel, 329 bufferIndex: int32(leaf.columnIndex), 330 bufferSize: int32(config.PageBufferSize), 331 writePageStats: config.DataPageStatistics, 332 encodings: make([]format.Encoding, 0, 3), 333 // Data pages in version 2 can omit compression when dictionary 334 // encoding is employed; only the dictionary page needs to be 335 // compressed, the data pages are encoded with the hybrid 336 // RLE/Bit-Pack encoding which doesn't benefit from an extra 337 // compression layer. 338 isCompressed: isCompressed(compression) && (dataPageType != format.DataPageV2 || dictionary == nil), 339 } 340 341 c.header.encoder.Reset(c.header.protocol.NewWriter(&buffers.header)) 342 343 if leaf.maxDefinitionLevel > 0 { 344 c.encodings = addEncoding(c.encodings, format.RLE) 345 } 346 347 if isDictionaryEncoding(encoding) { 348 c.encodings = addEncoding(c.encodings, format.Plain) 349 } 350 351 c.page.encoding = encoding 352 c.encodings = addEncoding(c.encodings, c.page.encoding.Encoding()) 353 sortPageEncodings(c.encodings) 354 355 w.columns = append(w.columns, c) 356 357 if sortingIndex := searchSortingColumn(config.SortingColumns, leaf.path); sortingIndex < len(w.sortingColumns) { 358 w.sortingColumns[sortingIndex] = format.SortingColumn{ 359 ColumnIdx: int32(leaf.columnIndex), 360 Descending: config.SortingColumns[sortingIndex].Descending(), 361 NullsFirst: config.SortingColumns[sortingIndex].NullsFirst(), 362 } 363 } 364 }) 365 366 // Pre-allocate the backing array so that in most cases where the rows 367 // contain a single value we will hit collocated memory areas when writing 368 // rows to the writer. This won't benefit repeated columns much but in that 369 // case we would just waste a bit of memory which we can afford. 370 values := make([]Value, len(w.columns)) 371 w.values = make([][]Value, len(w.columns)) 372 for i := range values { 373 w.values[i] = values[i : i : i+1] 374 } 375 376 w.columnChunk = make([]format.ColumnChunk, len(w.columns)) 377 w.columnIndex = make([]format.ColumnIndex, len(w.columns)) 378 w.offsetIndex = make([]format.OffsetIndex, len(w.columns)) 379 w.columnOrders = make([]format.ColumnOrder, len(w.columns)) 380 381 for i, c := range w.columns { 382 w.columnChunk[i] = format.ColumnChunk{ 383 MetaData: format.ColumnMetaData{ 384 Type: format.Type(c.columnType.Kind()), 385 Encoding: c.encodings, 386 PathInSchema: c.columnPath, 387 Codec: c.compression.CompressionCodec(), 388 KeyValueMetadata: nil, // TODO 389 }, 390 } 391 } 392 393 for i, c := range w.columns { 394 c.columnChunk = &w.columnChunk[i] 395 c.offsetIndex = &w.offsetIndex[i] 396 } 397 398 for i, c := range w.columns { 399 w.columnOrders[i] = *c.columnType.ColumnOrder() 400 } 401 402 return w 403 } 404 405 func (w *writer) reset(writer io.Writer) { 406 if w.buffer == nil { 407 w.writer.Reset(writer) 408 } else { 409 w.buffer.Reset(writer) 410 w.writer.Reset(w.buffer) 411 } 412 for _, c := range w.columns { 413 c.reset() 414 } 415 for i := range w.rowGroups { 416 w.rowGroups[i] = format.RowGroup{} 417 } 418 for i := range w.columnIndexes { 419 w.columnIndexes[i] = nil 420 } 421 for i := range w.offsetIndexes { 422 w.offsetIndexes[i] = nil 423 } 424 w.rowGroups = w.rowGroups[:0] 425 w.columnIndexes = w.columnIndexes[:0] 426 w.offsetIndexes = w.offsetIndexes[:0] 427 } 428 429 func (w *writer) close() error { 430 if err := w.writeFileHeader(); err != nil { 431 return err 432 } 433 if err := w.flush(); err != nil { 434 return err 435 } 436 if err := w.writeFileFooter(); err != nil { 437 return err 438 } 439 if w.buffer != nil { 440 return w.buffer.Flush() 441 } 442 return nil 443 } 444 445 func (w *writer) flush() error { 446 _, err := w.writeRowGroup(nil, nil) 447 return err 448 } 449 450 func (w *writer) writeFileHeader() error { 451 if w.writer.writer == nil { 452 return io.ErrClosedPipe 453 } 454 if w.writer.offset == 0 { 455 _, err := w.writer.WriteString("PAR1") 456 return err 457 } 458 return nil 459 } 460 461 func (w *writer) configureBloomFilters(columnChunks []ColumnChunk) { 462 for i, c := range w.columns { 463 if c.columnFilter != nil { 464 c.resizeBloomFilter(columnChunks[i].NumValues()) 465 } 466 } 467 } 468 469 func (w *writer) writeFileFooter() error { 470 // The page index is composed of two sections: column and offset indexes. 471 // They are written after the row groups, right before the footer (which 472 // is written by the parent Writer.Close call). 473 // 474 // This section both writes the page index and generates the values of 475 // ColumnIndexOffset, ColumnIndexLength, OffsetIndexOffset, and 476 // OffsetIndexLength in the corresponding columns of the file metadata. 477 // 478 // Note: the page index is always written, even if we created data pages v1 479 // because the parquet format is backward compatible in this case. Older 480 // readers will simply ignore this section since they do not know how to 481 // decode its content, nor have loaded any metadata to reference it. 482 protocol := new(thrift.CompactProtocol) 483 encoder := thrift.NewEncoder(protocol.NewWriter(&w.writer)) 484 485 for i, columnIndexes := range w.columnIndexes { 486 rowGroup := &w.rowGroups[i] 487 for j := range columnIndexes { 488 column := &rowGroup.Columns[j] 489 column.ColumnIndexOffset = w.writer.offset 490 if err := encoder.Encode(&columnIndexes[j]); err != nil { 491 return err 492 } 493 column.ColumnIndexLength = int32(w.writer.offset - column.ColumnIndexOffset) 494 } 495 } 496 497 for i, offsetIndexes := range w.offsetIndexes { 498 rowGroup := &w.rowGroups[i] 499 for j := range offsetIndexes { 500 column := &rowGroup.Columns[j] 501 column.OffsetIndexOffset = w.writer.offset 502 if err := encoder.Encode(&offsetIndexes[j]); err != nil { 503 return err 504 } 505 column.OffsetIndexLength = int32(w.writer.offset - column.OffsetIndexOffset) 506 } 507 } 508 509 numRows := int64(0) 510 for rowGroupIndex := range w.rowGroups { 511 numRows += w.rowGroups[rowGroupIndex].NumRows 512 } 513 514 footer, err := thrift.Marshal(new(thrift.CompactProtocol), &format.FileMetaData{ 515 Version: 1, 516 Schema: w.schemaElements, 517 NumRows: numRows, 518 RowGroups: w.rowGroups, 519 KeyValueMetadata: w.metadata, 520 CreatedBy: w.createdBy, 521 ColumnOrders: w.columnOrders, 522 }) 523 if err != nil { 524 return err 525 } 526 527 length := len(footer) 528 footer = append(footer, 0, 0, 0, 0) 529 footer = append(footer, "PAR1"...) 530 binary.LittleEndian.PutUint32(footer[length:], uint32(length)) 531 532 _, err = w.writer.Write(footer) 533 return err 534 } 535 536 func (w *writer) writeRowGroup(rowGroupSchema *Schema, rowGroupSortingColumns []SortingColumn) (int64, error) { 537 numRows := w.columns[0].totalRowCount() 538 if numRows == 0 { 539 return 0, nil 540 } 541 542 defer func() { 543 for _, c := range w.columns { 544 c.reset() 545 } 546 for i := range w.columnIndex { 547 w.columnIndex[i] = format.ColumnIndex{} 548 } 549 }() 550 551 for _, c := range w.columns { 552 if err := c.flush(); err != nil { 553 return 0, err 554 } 555 if err := c.flushFilterPages(); err != nil { 556 return 0, err 557 } 558 } 559 560 if err := w.writeFileHeader(); err != nil { 561 return 0, err 562 } 563 fileOffset := w.writer.offset 564 565 for _, c := range w.columns { 566 if len(c.filter.bits) > 0 { 567 c.columnChunk.MetaData.BloomFilterOffset = w.writer.offset 568 if err := c.writeBloomFilter(&w.writer); err != nil { 569 return 0, err 570 } 571 } 572 } 573 574 for i, c := range w.columns { 575 w.columnIndex[i] = format.ColumnIndex(c.columnIndex.ColumnIndex()) 576 577 if c.dictionary != nil { 578 c.columnChunk.MetaData.DictionaryPageOffset = w.writer.offset 579 if err := c.writeDictionaryPage(&w.writer, c.dictionary); err != nil { 580 return 0, fmt.Errorf("writing dictionary page of row group colum %d: %w", i, err) 581 } 582 } 583 584 dataPageOffset := w.writer.offset 585 c.columnChunk.MetaData.DataPageOffset = dataPageOffset 586 for j := range c.offsetIndex.PageLocations { 587 c.offsetIndex.PageLocations[j].Offset += dataPageOffset 588 } 589 590 for _, page := range c.pages { 591 if _, err := io.Copy(&w.writer, page); err != nil { 592 return 0, fmt.Errorf("writing buffered pages of row group column %d: %w", i, err) 593 } 594 } 595 } 596 597 totalByteSize := int64(0) 598 totalCompressedSize := int64(0) 599 600 for i := range w.columnChunk { 601 c := &w.columnChunk[i].MetaData 602 sortPageEncodingStats(c.EncodingStats) 603 totalByteSize += int64(c.TotalUncompressedSize) 604 totalCompressedSize += int64(c.TotalCompressedSize) 605 } 606 607 sortingColumns := w.sortingColumns 608 if len(sortingColumns) == 0 && len(rowGroupSortingColumns) > 0 { 609 sortingColumns = make([]format.SortingColumn, 0, len(rowGroupSortingColumns)) 610 forEachLeafColumnOf(rowGroupSchema, func(leaf leafColumn) { 611 if sortingIndex := searchSortingColumn(rowGroupSortingColumns, leaf.path); sortingIndex < len(sortingColumns) { 612 sortingColumns[sortingIndex] = format.SortingColumn{ 613 ColumnIdx: int32(leaf.columnIndex), 614 Descending: rowGroupSortingColumns[sortingIndex].Descending(), 615 NullsFirst: rowGroupSortingColumns[sortingIndex].NullsFirst(), 616 } 617 } 618 }) 619 } 620 621 columns := make([]format.ColumnChunk, len(w.columnChunk)) 622 copy(columns, w.columnChunk) 623 624 columnIndex := make([]format.ColumnIndex, len(w.columnIndex)) 625 copy(columnIndex, w.columnIndex) 626 627 offsetIndex := make([]format.OffsetIndex, len(w.offsetIndex)) 628 copy(offsetIndex, w.offsetIndex) 629 630 w.rowGroups = append(w.rowGroups, format.RowGroup{ 631 Columns: columns, 632 TotalByteSize: totalByteSize, 633 NumRows: numRows, 634 SortingColumns: sortingColumns, 635 FileOffset: fileOffset, 636 TotalCompressedSize: totalCompressedSize, 637 Ordinal: int16(len(w.rowGroups)), 638 }) 639 640 w.columnIndexes = append(w.columnIndexes, columnIndex) 641 w.offsetIndexes = append(w.offsetIndexes, offsetIndex) 642 return numRows, nil 643 } 644 645 func (w *writer) WriteRows(rows []Row) (int, error) { 646 defer func() { 647 for i, values := range w.values { 648 clearValues(values) 649 w.values[i] = values[:0] 650 } 651 }() 652 653 // TODO: if an error occurs in this method the writer may be left in an 654 // partially functional state. Applications are not expected to continue 655 // using the writer after getting an error, but maybe we could ensure that 656 // we are preventing further use as well? 657 for _, row := range rows { 658 for _, value := range row { 659 columnIndex := value.Column() 660 w.values[columnIndex] = append(w.values[columnIndex], value) 661 } 662 } 663 664 for i, values := range w.values { 665 if len(values) > 0 { 666 if err := w.columns[i].writeRows(values); err != nil { 667 return 0, err 668 } 669 } 670 } 671 672 return len(rows), nil 673 } 674 675 // The WriteValues method is intended to work in pair with WritePage to allow 676 // programs to target writing values to specific columns of of the writer. 677 func (w *writer) WriteValues(values []Value) (numValues int, err error) { 678 return w.columns[values[0].Column()].WriteValues(values) 679 } 680 681 // This WritePage method satisfies the PageWriter interface as a mechanism to 682 // allow writing whole pages of values instead of individual rows. It is called 683 // indirectly by readers that implement WriteRowsTo and are able to leverage 684 // the method to optimize writes. 685 func (w *writer) WritePage(page Page) (int64, error) { 686 return w.columns[page.Column()].WritePage(page) 687 } 688 689 // One writerBuffers is used by each writer instance, the memory buffers here 690 // are shared by all columns of the writer because serialization is not done 691 // concurrently, which helps keep memory utilization low, both in the total 692 // footprint and GC cost. 693 // 694 // The type also exposes helper methods to facilitate the generation of parquet 695 // pages. A scratch space is used when serialization requires combining multiple 696 // buffers or compressing the page data, with double-buffering technique being 697 // employed by swapping the scratch and page buffers to minimize memory copies. 698 type writerBuffers struct { 699 header bytes.Buffer // buffer where page headers are encoded 700 repetitions []byte // buffer used to encode repetition levels 701 definitions []byte // buffer used to encode definition levels 702 page []byte // page buffer holding the page data 703 scratch []byte // scratch space used for compression 704 } 705 706 func (wb *writerBuffers) crc32() (checksum uint32) { 707 checksum = crc32.Update(checksum, crc32.IEEETable, wb.repetitions) 708 checksum = crc32.Update(checksum, crc32.IEEETable, wb.definitions) 709 checksum = crc32.Update(checksum, crc32.IEEETable, wb.page) 710 return checksum 711 } 712 713 func (wb *writerBuffers) size() int { 714 return len(wb.repetitions) + len(wb.definitions) + len(wb.page) 715 } 716 717 func (wb *writerBuffers) reset() { 718 wb.repetitions = wb.repetitions[:0] 719 wb.definitions = wb.definitions[:0] 720 wb.page = wb.page[:0] 721 } 722 723 func (wb *writerBuffers) encodeRepetitionLevels(page BufferedPage, maxRepetitionLevel byte) (err error) { 724 bitWidth := bits.Len8(maxRepetitionLevel) 725 encoding := &levelEncodingsRLE[bitWidth-1] 726 wb.repetitions, err = encoding.EncodeLevels(wb.repetitions[:0], page.RepetitionLevels()) 727 return err 728 } 729 730 func (wb *writerBuffers) encodeDefinitionLevels(page BufferedPage, maxDefinitionLevel byte) (err error) { 731 bitWidth := bits.Len8(maxDefinitionLevel) 732 encoding := &levelEncodingsRLE[bitWidth-1] 733 wb.definitions, err = encoding.EncodeLevels(wb.definitions[:0], page.DefinitionLevels()) 734 return err 735 } 736 737 func (wb *writerBuffers) prependLevelsToDataPageV1(maxRepetitionLevel, maxDefinitionLevel byte) { 738 hasRepetitionLevels := maxRepetitionLevel > 0 739 hasDefinitionLevels := maxDefinitionLevel > 0 740 741 if hasRepetitionLevels || hasDefinitionLevels { 742 wb.scratch = wb.scratch[:0] 743 // In data pages v1, the repetition and definition levels are prefixed 744 // with the 4 bytes length of the sections. While the parquet-format 745 // documentation indicates that the length prefix is part of the hybrid 746 // RLE/Bit-Pack encoding, this is the only condition where it is used 747 // so we treat it as a special case rather than implementing it in the 748 // encoding. 749 // 750 // Reference https://github.com/apache/parquet-format/blob/master/Encodings.md#run-length-encoding--bit-packing-hybrid-rle--3 751 if hasRepetitionLevels { 752 wb.scratch = plain.AppendInt32(wb.scratch, int32(len(wb.repetitions))) 753 wb.scratch = append(wb.scratch, wb.repetitions...) 754 wb.repetitions = wb.repetitions[:0] 755 } 756 if hasDefinitionLevels { 757 wb.scratch = plain.AppendInt32(wb.scratch, int32(len(wb.definitions))) 758 wb.scratch = append(wb.scratch, wb.definitions...) 759 wb.definitions = wb.definitions[:0] 760 } 761 wb.scratch = append(wb.scratch, wb.page...) 762 wb.swapPageAndScratchBuffers() 763 } 764 } 765 766 func (wb *writerBuffers) encode(page BufferedPage, enc encoding.Encoding) (err error) { 767 pageType := page.Type() 768 pageData := page.Data() 769 wb.page, err = pageType.Encode(wb.page[:0], pageData, enc) 770 return err 771 } 772 773 func (wb *writerBuffers) compress(codec compress.Codec) (err error) { 774 wb.scratch, err = codec.Encode(wb.scratch[:0], wb.page) 775 wb.swapPageAndScratchBuffers() 776 return err 777 } 778 779 func (wb *writerBuffers) swapPageAndScratchBuffers() { 780 wb.page, wb.scratch = wb.scratch, wb.page[:0] 781 } 782 783 type writerColumn struct { 784 pool PageBufferPool 785 pages []io.ReadWriter 786 787 columnPath columnPath 788 columnType Type 789 columnIndex ColumnIndexer 790 columnBuffer ColumnBuffer 791 columnFilter BloomFilterColumn 792 compression compress.Codec 793 dictionary Dictionary 794 795 dataPageType format.PageType 796 maxRepetitionLevel byte 797 maxDefinitionLevel byte 798 799 buffers *writerBuffers 800 801 header struct { 802 protocol thrift.CompactProtocol 803 encoder thrift.Encoder 804 } 805 806 page struct { 807 encoding encoding.Encoding 808 } 809 810 filter struct { 811 bits []byte 812 pages []BufferedPage 813 } 814 815 numRows int64 816 maxValues int32 817 numValues int32 818 bufferIndex int32 819 bufferSize int32 820 writePageStats bool 821 isCompressed bool 822 encodings []format.Encoding 823 824 columnChunk *format.ColumnChunk 825 offsetIndex *format.OffsetIndex 826 } 827 828 func (c *writerColumn) reset() { 829 if c.columnBuffer != nil { 830 c.columnBuffer.Reset() 831 } 832 if c.columnIndex != nil { 833 c.columnIndex.Reset() 834 } 835 if c.dictionary != nil { 836 c.dictionary.Reset() 837 } 838 for _, page := range c.pages { 839 c.pool.PutPageBuffer(page) 840 } 841 for i := range c.pages { 842 c.pages[i] = nil 843 } 844 for i := range c.filter.pages { 845 c.filter.pages[i] = nil 846 } 847 c.pages = c.pages[:0] 848 // Bloom filters may change in size between row groups, but we retain the 849 // buffer to avoid reallocating large memory blocks. 850 c.filter.bits = c.filter.bits[:0] 851 c.filter.pages = c.filter.pages[:0] 852 c.numRows = 0 853 c.numValues = 0 854 // Reset the fields of column chunks that change between row groups, 855 // but keep the ones that remain unchanged. 856 c.columnChunk.MetaData.NumValues = 0 857 c.columnChunk.MetaData.TotalUncompressedSize = 0 858 c.columnChunk.MetaData.TotalCompressedSize = 0 859 c.columnChunk.MetaData.DataPageOffset = 0 860 c.columnChunk.MetaData.DictionaryPageOffset = 0 861 c.columnChunk.MetaData.Statistics = format.Statistics{} 862 c.columnChunk.MetaData.EncodingStats = make([]format.PageEncodingStats, 0, cap(c.columnChunk.MetaData.EncodingStats)) 863 c.columnChunk.MetaData.BloomFilterOffset = 0 864 // Retain the previous capacity in the new page locations array, assuming 865 // the number of pages should be roughly the same between row groups written 866 // by the writer. 867 c.offsetIndex.PageLocations = make([]format.PageLocation, 0, cap(c.offsetIndex.PageLocations)) 868 } 869 870 func (c *writerColumn) totalRowCount() int64 { 871 n := c.numRows 872 if c.columnBuffer != nil { 873 n += int64(c.columnBuffer.Len()) 874 } 875 return n 876 } 877 878 func (c *writerColumn) canFlush() bool { 879 return c.columnBuffer.Size() >= int64(c.bufferSize/2) 880 } 881 882 func (c *writerColumn) flush() (err error) { 883 if c.numValues != 0 { 884 c.numValues = 0 885 defer c.columnBuffer.Reset() 886 _, err = c.writeBufferedPage(c.columnBuffer.Page()) 887 } 888 return err 889 } 890 891 func (c *writerColumn) flushFilterPages() error { 892 if c.columnFilter != nil { 893 // If there is a dictionary, it contains all the values that we need to 894 // write to the filter. 895 if dict := c.dictionary; dict != nil { 896 if c.filter.bits == nil { 897 c.resizeBloomFilter(int64(dict.Len())) 898 } 899 if err := c.writePageToFilter(dict.Page()); err != nil { 900 return err 901 } 902 } 903 904 if len(c.filter.pages) > 0 { 905 numValues := int64(0) 906 for _, page := range c.filter.pages { 907 numValues += page.NumValues() 908 } 909 c.resizeBloomFilter(numValues) 910 for _, page := range c.filter.pages { 911 if err := c.writePageToFilter(page); err != nil { 912 return err 913 } 914 } 915 } 916 } 917 return nil 918 } 919 920 func (c *writerColumn) resizeBloomFilter(numValues int64) { 921 const bitsPerValue = 10 // TODO: make this configurable 922 filterSize := c.columnFilter.Size(numValues, bitsPerValue) 923 if cap(c.filter.bits) < filterSize { 924 c.filter.bits = make([]byte, filterSize) 925 } else { 926 c.filter.bits = c.filter.bits[:filterSize] 927 for i := range c.filter.bits { 928 c.filter.bits[i] = 0 929 } 930 } 931 } 932 933 func (c *writerColumn) newColumnBuffer() ColumnBuffer { 934 columnBufferCapacity := sort.Search(math.MaxInt32, func(i int) bool { 935 return c.columnType.EstimateSize(i) >= int64(c.bufferSize) 936 }) 937 column := c.columnType.NewColumnBuffer(int(c.bufferIndex), columnBufferCapacity) 938 switch { 939 case c.maxRepetitionLevel > 0: 940 column = newRepeatedColumnBuffer(column, c.maxRepetitionLevel, c.maxDefinitionLevel, nullsGoLast) 941 case c.maxDefinitionLevel > 0: 942 column = newOptionalColumnBuffer(column, c.maxDefinitionLevel, nullsGoLast) 943 } 944 return column 945 } 946 947 func (c *writerColumn) writeRows(rows []Value) error { 948 if c.columnBuffer == nil { 949 // Lazily create the row group column so we don't need to allocate it if 950 // rows are not written individually to the column. 951 c.columnBuffer = c.newColumnBuffer() 952 c.maxValues = int32(c.columnBuffer.Cap()) 953 } 954 955 if c.numValues > 0 && c.numValues > (c.maxValues-int32(len(rows))) { 956 if err := c.flush(); err != nil { 957 return err 958 } 959 } 960 961 if _, err := c.columnBuffer.WriteValues(rows); err != nil { 962 return err 963 } 964 c.numValues += int32(len(rows)) 965 return nil 966 } 967 968 func (c *writerColumn) WriteValues(values []Value) (numValues int, err error) { 969 if c.columnBuffer == nil { 970 c.columnBuffer = c.newColumnBuffer() 971 c.maxValues = int32(c.columnBuffer.Cap()) 972 } 973 numValues, err = c.columnBuffer.WriteValues(values) 974 c.numValues += int32(numValues) 975 return numValues, err 976 } 977 978 func (c *writerColumn) WritePage(page Page) (numValues int64, err error) { 979 // Page write optimizations are only available the column is not reindexing 980 // the values. If a dictionary is present, the column needs to see each 981 // individual value in order to re-index them in the dictionary. 982 if c.dictionary == nil || c.dictionary == page.Dictionary() { 983 // If the column had buffered values, we continue writing values from 984 // the page into the column buffer if it would have caused producing a 985 // page less than half the size of the target; if there were enough 986 // buffered values already, we have to flush the buffered page instead 987 // otherwise values would get reordered. 988 if c.numValues > 0 && c.canFlush() { 989 if err := c.flush(); err != nil { 990 return 0, err 991 } 992 } 993 994 // If we were successful at flushing buffered values, we attempt to 995 // optimize the write path by copying whole pages without decoding them 996 // into a sequence of values. 997 if c.numValues == 0 { 998 switch p := page.(type) { 999 case BufferedPage: 1000 // Buffered pages may be larger than the target page size on the 1001 // column, in which case multiple pages get written by slicing 1002 // the original page into sub-pages. 1003 err = forEachPageSlice(p, int64(c.bufferSize), func(p BufferedPage) error { 1004 n, err := c.writeBufferedPage(p) 1005 numValues += n 1006 return err 1007 }) 1008 return numValues, err 1009 1010 case CompressedPage: 1011 // Compressed pages are written as-is to the compressed page 1012 // buffers; those pages should be coming from parquet files that 1013 // are being copied into a new file, they are simply copied to 1014 // amortize the cost of decoding and re-encoding the pages, which 1015 // often includes costly compression steps. 1016 return c.writeCompressedPage(p) 1017 } 1018 } 1019 } 1020 1021 // Pages that implement neither of those interfaces can still be 1022 // written by copying their values into the column buffer and flush 1023 // them to compressed page buffers as if the program had written 1024 // rows individually. 1025 return c.writePageValues(page.Values()) 1026 } 1027 1028 func (c *writerColumn) writePageValues(page ValueReader) (numValues int64, err error) { 1029 numValues, err = CopyValues(c, page) 1030 if err == nil && c.canFlush() { 1031 // Always attempt to flush after writing a full page if we have enough 1032 // buffered values; the intent is to leave the column clean so that 1033 // subsequent calls to the WritePage method can use optimized write path 1034 // to bypass buffering. 1035 err = c.flush() 1036 } 1037 return numValues, err 1038 } 1039 1040 func (c *writerColumn) writeBloomFilter(w io.Writer) error { 1041 e := thrift.NewEncoder(c.header.protocol.NewWriter(w)) 1042 h := bloomFilterHeader(c.columnFilter) 1043 h.NumBytes = int32(len(c.filter.bits)) 1044 if err := e.Encode(&h); err != nil { 1045 return err 1046 } 1047 _, err := w.Write(c.filter.bits) 1048 return err 1049 } 1050 1051 func (c *writerColumn) writeBufferedPage(page BufferedPage) (int64, error) { 1052 numValues := page.NumValues() 1053 if numValues == 0 { 1054 return 0, nil 1055 } 1056 1057 buf := c.buffers 1058 buf.reset() 1059 1060 if c.maxRepetitionLevel > 0 { 1061 buf.encodeRepetitionLevels(page, c.maxRepetitionLevel) 1062 } 1063 if c.maxDefinitionLevel > 0 { 1064 buf.encodeDefinitionLevels(page, c.maxDefinitionLevel) 1065 } 1066 1067 if err := buf.encode(page, c.page.encoding); err != nil { 1068 return 0, fmt.Errorf("encoding parquet data page: %w", err) 1069 } 1070 if c.dataPageType == format.DataPage { 1071 buf.prependLevelsToDataPageV1(c.maxDefinitionLevel, c.maxDefinitionLevel) 1072 } 1073 1074 uncompressedPageSize := buf.size() 1075 if c.isCompressed { 1076 if err := buf.compress(c.compression); err != nil { 1077 return 0, fmt.Errorf("compressing parquet data page: %w", err) 1078 } 1079 } 1080 1081 if page.Dictionary() == nil { 1082 switch { 1083 case len(c.filter.bits) > 0: 1084 // When the writer knows the number of values in advance (e.g. when 1085 // writing a full row group), the filter encoding is set and the page 1086 // can be directly applied to the filter, which minimizes memory usage 1087 // since there is no need to buffer the values in order to determine 1088 // the size of the filter. 1089 if err := c.writePageToFilter(page); err != nil { 1090 return 0, err 1091 } 1092 case c.columnFilter != nil: 1093 // If the column uses a dictionary encoding, all possible values exist 1094 // in the dictionary and there is no need to buffer the pages, but if 1095 // the column is supposed to generate a filter and the number of values 1096 // wasn't known, we must buffer all the pages in order to properly size 1097 // the filter. 1098 c.filter.pages = append(c.filter.pages, page.Clone()) 1099 } 1100 } 1101 1102 statistics := format.Statistics{} 1103 if c.writePageStats { 1104 statistics = c.makePageStatistics(page) 1105 } 1106 1107 pageHeader := &format.PageHeader{ 1108 Type: c.dataPageType, 1109 UncompressedPageSize: int32(uncompressedPageSize), 1110 CompressedPageSize: int32(buf.size()), 1111 CRC: int32(buf.crc32()), 1112 } 1113 1114 numRows := page.NumRows() 1115 numNulls := page.NumNulls() 1116 switch c.dataPageType { 1117 case format.DataPage: 1118 pageHeader.DataPageHeader = &format.DataPageHeader{ 1119 NumValues: int32(numValues), 1120 Encoding: c.page.encoding.Encoding(), 1121 DefinitionLevelEncoding: format.RLE, 1122 RepetitionLevelEncoding: format.RLE, 1123 Statistics: statistics, 1124 } 1125 case format.DataPageV2: 1126 pageHeader.DataPageHeaderV2 = &format.DataPageHeaderV2{ 1127 NumValues: int32(numValues), 1128 NumNulls: int32(numNulls), 1129 NumRows: int32(numRows), 1130 Encoding: c.page.encoding.Encoding(), 1131 DefinitionLevelsByteLength: int32(len(buf.definitions)), 1132 RepetitionLevelsByteLength: int32(len(buf.repetitions)), 1133 IsCompressed: &c.isCompressed, 1134 Statistics: statistics, 1135 } 1136 } 1137 1138 buf.header.Reset() 1139 if err := c.header.encoder.Encode(pageHeader); err != nil { 1140 return 0, err 1141 } 1142 1143 size := int64(buf.header.Len()) + 1144 int64(len(buf.repetitions)) + 1145 int64(len(buf.definitions)) + 1146 int64(len(buf.page)) 1147 1148 err := c.writePage(size, func(output io.Writer) (written int64, err error) { 1149 for _, data := range [...][]byte{ 1150 buf.header.Bytes(), 1151 buf.repetitions, 1152 buf.definitions, 1153 buf.page, 1154 } { 1155 wn, err := output.Write(data) 1156 written += int64(wn) 1157 if err != nil { 1158 return written, err 1159 } 1160 } 1161 return written, nil 1162 }) 1163 if err != nil { 1164 return 0, err 1165 } 1166 1167 c.recordPageStats(int32(buf.header.Len()), pageHeader, page) 1168 return numValues, nil 1169 } 1170 1171 func (c *writerColumn) writeCompressedPage(page CompressedPage) (int64, error) { 1172 if page.Dictionary() == nil { 1173 switch { 1174 case len(c.filter.bits) > 0: 1175 // TODO: modify the Buffer method to accept some kind of buffer pool as 1176 // argument so we can use a pre-allocated page buffer to load the page 1177 // and reduce the memory footprint. 1178 bufferedPage := page.Buffer() 1179 // The compressed page must be decompressed here in order to generate 1180 // the bloom filter. Note that we don't re-compress it which still saves 1181 // most of the compute cost (compression algorithms are usually designed 1182 // to make decompressing much cheaper than compressing since it happens 1183 // more often). 1184 if err := c.writePageToFilter(bufferedPage); err != nil { 1185 return 0, err 1186 } 1187 case c.columnFilter != nil && c.dictionary == nil: 1188 // When a column filter is configured but no page filter was allocated, 1189 // we need to buffer the page in order to have access to the number of 1190 // values and properly size the bloom filter when writing the row group. 1191 c.filter.pages = append(c.filter.pages, page.Buffer()) 1192 } 1193 } 1194 1195 pageHeader := &format.PageHeader{ 1196 UncompressedPageSize: int32(page.Size()), 1197 CompressedPageSize: int32(page.PageSize()), 1198 CRC: int32(page.CRC()), 1199 } 1200 1201 switch h := page.PageHeader().(type) { 1202 case DataPageHeaderV1: 1203 pageHeader.DataPageHeader = h.header 1204 case DataPageHeaderV2: 1205 pageHeader.DataPageHeaderV2 = h.header 1206 default: 1207 return 0, fmt.Errorf("writing compressed page type of unknown type: %s", h.PageType()) 1208 } 1209 1210 header := &c.buffers.header 1211 header.Reset() 1212 if err := c.header.encoder.Encode(pageHeader); err != nil { 1213 return 0, err 1214 } 1215 headerSize := int32(header.Len()) 1216 compressedSize := int64(headerSize + pageHeader.CompressedPageSize) 1217 1218 err := c.writePage(compressedSize, func(output io.Writer) (int64, error) { 1219 headerSize, err := header.WriteTo(output) 1220 if err != nil { 1221 return headerSize, err 1222 } 1223 dataSize, err := io.Copy(output, page.PageData()) 1224 return headerSize + dataSize, err 1225 }) 1226 if err != nil { 1227 return 0, err 1228 } 1229 c.recordPageStats(headerSize, pageHeader, page) 1230 return page.NumValues(), nil 1231 } 1232 1233 func (c *writerColumn) writeDictionaryPage(output io.Writer, dict Dictionary) (err error) { 1234 buf := c.buffers 1235 buf.reset() 1236 1237 if err := buf.encode(dict.Page(), &Plain); err != nil { 1238 return fmt.Errorf("writing parquet dictionary page: %w", err) 1239 } 1240 1241 uncompressedPageSize := buf.size() 1242 if isCompressed(c.compression) { 1243 if err := buf.compress(c.compression); err != nil { 1244 return fmt.Errorf("copmressing parquet dictionary page: %w", err) 1245 } 1246 } 1247 1248 pageHeader := &format.PageHeader{ 1249 Type: format.DictionaryPage, 1250 UncompressedPageSize: int32(uncompressedPageSize), 1251 CompressedPageSize: int32(buf.size()), 1252 CRC: int32(buf.crc32()), 1253 DictionaryPageHeader: &format.DictionaryPageHeader{ 1254 NumValues: int32(dict.Len()), 1255 Encoding: format.Plain, 1256 IsSorted: false, 1257 }, 1258 } 1259 1260 header := &c.buffers.header 1261 header.Reset() 1262 if err := c.header.encoder.Encode(pageHeader); err != nil { 1263 return err 1264 } 1265 if _, err := output.Write(header.Bytes()); err != nil { 1266 return err 1267 } 1268 if _, err := output.Write(buf.page); err != nil { 1269 return err 1270 } 1271 c.recordPageStats(int32(header.Len()), pageHeader, nil) 1272 return nil 1273 } 1274 1275 func (w *writerColumn) writePageToFilter(page BufferedPage) (err error) { 1276 pageType := page.Type() 1277 pageData := page.Data() 1278 w.filter.bits, err = pageType.Encode(w.filter.bits, pageData, w.columnFilter.Encoding()) 1279 return err 1280 } 1281 1282 func (c *writerColumn) writePage(size int64, writeTo func(io.Writer) (int64, error)) error { 1283 buffer := c.pool.GetPageBuffer() 1284 defer func() { 1285 if buffer != nil { 1286 c.pool.PutPageBuffer(buffer) 1287 } 1288 }() 1289 written, err := writeTo(buffer) 1290 if err != nil { 1291 return err 1292 } 1293 if written != size { 1294 return fmt.Errorf("writing parquet column page expected %dB but got %dB: %w", size, written, io.ErrShortWrite) 1295 } 1296 c.pages, buffer = append(c.pages, buffer), nil 1297 return nil 1298 } 1299 1300 func (c *writerColumn) makePageStatistics(page Page) format.Statistics { 1301 numNulls := page.NumNulls() 1302 minValue, maxValue, _ := page.Bounds() 1303 minValueBytes := minValue.Bytes() 1304 maxValueBytes := maxValue.Bytes() 1305 return format.Statistics{ 1306 Min: minValueBytes, // deprecated 1307 Max: maxValueBytes, // deprecated 1308 NullCount: numNulls, 1309 MinValue: minValueBytes, 1310 MaxValue: maxValueBytes, 1311 } 1312 } 1313 1314 func (c *writerColumn) recordPageStats(headerSize int32, header *format.PageHeader, page Page) { 1315 uncompressedSize := headerSize + header.UncompressedPageSize 1316 compressedSize := headerSize + header.CompressedPageSize 1317 1318 if page != nil { 1319 numNulls := page.NumNulls() 1320 numValues := page.NumValues() 1321 minValue, maxValue, _ := page.Bounds() 1322 c.columnIndex.IndexPage(numValues, numNulls, minValue, maxValue) 1323 c.columnChunk.MetaData.NumValues += numValues 1324 1325 c.offsetIndex.PageLocations = append(c.offsetIndex.PageLocations, format.PageLocation{ 1326 Offset: c.columnChunk.MetaData.TotalCompressedSize, 1327 CompressedPageSize: compressedSize, 1328 FirstRowIndex: c.numRows, 1329 }) 1330 1331 c.numRows += page.NumRows() 1332 } 1333 1334 pageType := header.Type 1335 encoding := format.Encoding(-1) 1336 switch pageType { 1337 case format.DataPageV2: 1338 encoding = header.DataPageHeaderV2.Encoding 1339 case format.DataPage: 1340 encoding = header.DataPageHeader.Encoding 1341 case format.DictionaryPage: 1342 encoding = header.DictionaryPageHeader.Encoding 1343 } 1344 1345 c.columnChunk.MetaData.TotalUncompressedSize += int64(uncompressedSize) 1346 c.columnChunk.MetaData.TotalCompressedSize += int64(compressedSize) 1347 c.columnChunk.MetaData.EncodingStats = addPageEncodingStats(c.columnChunk.MetaData.EncodingStats, format.PageEncodingStats{ 1348 PageType: pageType, 1349 Encoding: encoding, 1350 Count: 1, 1351 }) 1352 } 1353 1354 func addEncoding(encodings []format.Encoding, add format.Encoding) []format.Encoding { 1355 for _, enc := range encodings { 1356 if enc == add { 1357 return encodings 1358 } 1359 } 1360 return append(encodings, add) 1361 } 1362 1363 func addPageEncodingStats(stats []format.PageEncodingStats, pages ...format.PageEncodingStats) []format.PageEncodingStats { 1364 addPages: 1365 for _, add := range pages { 1366 for i, st := range stats { 1367 if st.PageType == add.PageType && st.Encoding == add.Encoding { 1368 stats[i].Count += add.Count 1369 continue addPages 1370 } 1371 } 1372 stats = append(stats, add) 1373 } 1374 return stats 1375 } 1376 1377 func sortPageEncodings(encodings []format.Encoding) { 1378 sort.Slice(encodings, func(i, j int) bool { 1379 return encodings[i] < encodings[j] 1380 }) 1381 } 1382 1383 func sortPageEncodingStats(stats []format.PageEncodingStats) { 1384 sort.Slice(stats, func(i, j int) bool { 1385 s1 := &stats[i] 1386 s2 := &stats[j] 1387 if s1.PageType != s2.PageType { 1388 return s1.PageType < s2.PageType 1389 } 1390 return s1.Encoding < s2.Encoding 1391 }) 1392 } 1393 1394 type offsetTrackingWriter struct { 1395 writer io.Writer 1396 offset int64 1397 } 1398 1399 func (w *offsetTrackingWriter) Reset(writer io.Writer) { 1400 w.writer = writer 1401 w.offset = 0 1402 } 1403 1404 func (w *offsetTrackingWriter) Write(b []byte) (int, error) { 1405 n, err := w.writer.Write(b) 1406 w.offset += int64(n) 1407 return n, err 1408 } 1409 1410 func (w *offsetTrackingWriter) WriteString(s string) (int, error) { 1411 n, err := io.WriteString(w.writer, s) 1412 w.offset += int64(n) 1413 return n, err 1414 } 1415 1416 var ( 1417 _ RowWriterWithSchema = (*Writer)(nil) 1418 _ RowReaderFrom = (*Writer)(nil) 1419 _ RowGroupWriter = (*Writer)(nil) 1420 1421 _ RowWriter = (*writer)(nil) 1422 _ PageWriter = (*writer)(nil) 1423 _ ValueWriter = (*writer)(nil) 1424 1425 _ PageWriter = (*writerColumn)(nil) 1426 _ ValueWriter = (*writerColumn)(nil) 1427 )