github.com/parquet-go/parquet-go@v0.20.0/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 "reflect" 13 "sort" 14 15 "github.com/parquet-go/parquet-go/compress" 16 "github.com/parquet-go/parquet-go/encoding" 17 "github.com/parquet-go/parquet-go/encoding/plain" 18 "github.com/parquet-go/parquet-go/format" 19 "github.com/segmentio/encoding/thrift" 20 ) 21 22 const ( 23 // The uncompressed page size is stored as int32 and must not be larger than the 24 // maximum int32 value (see format.PageHeader). 25 maxUncompressedPageSize = math.MaxInt32 26 ) 27 28 // GenericWriter is similar to a Writer but uses a type parameter to define the 29 // Go type representing the schema of rows being written. 30 // 31 // Using this type over Writer has multiple advantages: 32 // 33 // - By leveraging type information, the Go compiler can provide greater 34 // guarantees that the code is correct. For example, the parquet.Writer.Write 35 // method accepts an argument of type interface{}, which delays type checking 36 // until runtime. The parquet.GenericWriter[T].Write method ensures at 37 // compile time that the values it receives will be of type T, reducing the 38 // risk of introducing errors. 39 // 40 // - Since type information is known at compile time, the implementation of 41 // parquet.GenericWriter[T] can make safe assumptions, removing the need for 42 // runtime validation of how the parameters are passed to its methods. 43 // Optimizations relying on type information are more effective, some of the 44 // writer's state can be precomputed at initialization, which was not possible 45 // with parquet.Writer. 46 // 47 // - The parquet.GenericWriter[T].Write method uses a data-oriented design, 48 // accepting an slice of T instead of a single value, creating more 49 // opportunities to amortize the runtime cost of abstractions. 50 // This optimization is not available for parquet.Writer because its Write 51 // method's argument would be of type []interface{}, which would require 52 // conversions back and forth from concrete types to empty interfaces (since 53 // a []T cannot be interpreted as []interface{} in Go), would make the API 54 // more difficult to use and waste compute resources in the type conversions, 55 // defeating the purpose of the optimization in the first place. 56 // 57 // Note that this type is only available when compiling with Go 1.18 or later. 58 type GenericWriter[T any] struct { 59 // At this time GenericWriter is expressed in terms of Writer to reuse the 60 // underlying logic. In the future, and if we accepted to break backward 61 // compatibility on the Write method, we could modify Writer to be an alias 62 // to GenericWriter with: 63 // 64 // type Writer = GenericWriter[any] 65 // 66 base Writer 67 // This function writes rows of type T to the writer, it gets generated by 68 // the NewGenericWriter function based on the type T and the underlying 69 // schema of the parquet file. 70 write writeFunc[T] 71 // This field is used to leverage the optimized writeRowsFunc algorithms. 72 columns []ColumnBuffer 73 } 74 75 // NewGenericWriter is like NewWriter but returns a GenericWriter[T] suited to 76 // write rows of Go type T. 77 // 78 // The type parameter T should be a map, struct, or any. Any other types will 79 // cause a panic at runtime. Type checking is a lot more effective when the 80 // generic parameter is a struct type, using map and interface types is somewhat 81 // similar to using a Writer. 82 // 83 // If the option list may explicitly declare a schema, it must be compatible 84 // with the schema generated from T. 85 // 86 // Sorting columns may be set on the writer to configure the generated row 87 // groups metadata. However, rows are always written in the order they were 88 // seen, no reordering is performed, the writer expects the application to 89 // ensure proper correlation between the order of rows and the list of sorting 90 // columns. See SortingWriter[T] for a writer which handles reordering rows 91 // based on the configured sorting columns. 92 func NewGenericWriter[T any](output io.Writer, options ...WriterOption) *GenericWriter[T] { 93 config, err := NewWriterConfig(options...) 94 if err != nil { 95 panic(err) 96 } 97 98 schema := config.Schema 99 t := typeOf[T]() 100 101 if schema == nil && t != nil { 102 schema = schemaOf(dereference(t)) 103 config.Schema = schema 104 } 105 106 if config.Schema == nil { 107 panic("generic writer must be instantiated with schema or concrete type.") 108 } 109 110 return &GenericWriter[T]{ 111 base: Writer{ 112 output: output, 113 config: config, 114 schema: schema, 115 writer: newWriter(output, config), 116 }, 117 write: writeFuncOf[T](t, config.Schema), 118 } 119 } 120 121 type writeFunc[T any] func(*GenericWriter[T], []T) (int, error) 122 123 func writeFuncOf[T any](t reflect.Type, schema *Schema) writeFunc[T] { 124 if t == nil { 125 return (*GenericWriter[T]).writeAny 126 } 127 switch t.Kind() { 128 case reflect.Interface, reflect.Map: 129 return (*GenericWriter[T]).writeRows 130 131 case reflect.Struct: 132 return makeWriteFunc[T](t, schema) 133 134 case reflect.Pointer: 135 if e := t.Elem(); e.Kind() == reflect.Struct { 136 return makeWriteFunc[T](t, schema) 137 } 138 } 139 panic("cannot create writer for values of type " + t.String()) 140 } 141 142 func makeWriteFunc[T any](t reflect.Type, schema *Schema) writeFunc[T] { 143 writeRows := writeRowsFuncOf(t, schema, nil) 144 return func(w *GenericWriter[T], rows []T) (n int, err error) { 145 if w.columns == nil { 146 w.columns = make([]ColumnBuffer, len(w.base.writer.columns)) 147 for i, c := range w.base.writer.columns { 148 // These fields are usually lazily initialized when writing rows, 149 // we need them to exist now tho. 150 c.columnBuffer = c.newColumnBuffer() 151 w.columns[i] = c.columnBuffer 152 } 153 } 154 err = writeRows(w.columns, makeArrayOf(rows), columnLevels{}) 155 if err == nil { 156 n = len(rows) 157 } 158 return n, err 159 } 160 } 161 162 func (w *GenericWriter[T]) Close() error { 163 return w.base.Close() 164 } 165 166 func (w *GenericWriter[T]) Flush() error { 167 return w.base.Flush() 168 } 169 170 func (w *GenericWriter[T]) Reset(output io.Writer) { 171 w.base.Reset(output) 172 } 173 174 func (w *GenericWriter[T]) Write(rows []T) (int, error) { 175 return w.base.writer.writeRows(len(rows), func(i, j int) (int, error) { 176 n, err := w.write(w, rows[i:j:j]) 177 if err != nil { 178 return n, err 179 } 180 181 for _, c := range w.base.writer.columns { 182 if c.columnBuffer.Size() >= int64(c.bufferSize) { 183 if err := c.flush(); err != nil { 184 return n, err 185 } 186 } 187 } 188 189 return n, nil 190 }) 191 } 192 193 func (w *GenericWriter[T]) WriteRows(rows []Row) (int, error) { 194 return w.base.WriteRows(rows) 195 } 196 197 func (w *GenericWriter[T]) WriteRowGroup(rowGroup RowGroup) (int64, error) { 198 return w.base.WriteRowGroup(rowGroup) 199 } 200 201 // SetKeyValueMetadata sets a key/value pair in the Parquet file metadata. 202 // 203 // Keys are assumed to be unique, if the same key is repeated multiple times the 204 // last value is retained. While the parquet format does not require unique keys, 205 // this design decision was made to optimize for the most common use case where 206 // applications leverage this extension mechanism to associate single values to 207 // keys. This may create incompatibilities with other parquet libraries, or may 208 // cause some key/value pairs to be lost when open parquet files written with 209 // repeated keys. We can revisit this decision if it ever becomes a blocker. 210 func (w *GenericWriter[T]) SetKeyValueMetadata(key, value string) { 211 w.base.SetKeyValueMetadata(key, value) 212 } 213 214 func (w *GenericWriter[T]) ReadRowsFrom(rows RowReader) (int64, error) { 215 return w.base.ReadRowsFrom(rows) 216 } 217 218 func (w *GenericWriter[T]) Schema() *Schema { 219 return w.base.Schema() 220 } 221 222 func (w *GenericWriter[T]) writeRows(rows []T) (int, error) { 223 if cap(w.base.rowbuf) < len(rows) { 224 w.base.rowbuf = make([]Row, len(rows)) 225 } else { 226 w.base.rowbuf = w.base.rowbuf[:len(rows)] 227 } 228 defer clearRows(w.base.rowbuf) 229 230 schema := w.base.Schema() 231 for i := range rows { 232 w.base.rowbuf[i] = schema.Deconstruct(w.base.rowbuf[i], &rows[i]) 233 } 234 235 return w.base.WriteRows(w.base.rowbuf) 236 } 237 238 func (w *GenericWriter[T]) writeAny(rows []T) (n int, err error) { 239 for i := range rows { 240 if err = w.base.Write(rows[i]); err != nil { 241 return n, err 242 } 243 n++ 244 } 245 return n, nil 246 } 247 248 var ( 249 _ RowWriterWithSchema = (*GenericWriter[any])(nil) 250 _ RowReaderFrom = (*GenericWriter[any])(nil) 251 _ RowGroupWriter = (*GenericWriter[any])(nil) 252 253 _ RowWriterWithSchema = (*GenericWriter[struct{}])(nil) 254 _ RowReaderFrom = (*GenericWriter[struct{}])(nil) 255 _ RowGroupWriter = (*GenericWriter[struct{}])(nil) 256 257 _ RowWriterWithSchema = (*GenericWriter[map[struct{}]struct{}])(nil) 258 _ RowReaderFrom = (*GenericWriter[map[struct{}]struct{}])(nil) 259 _ RowGroupWriter = (*GenericWriter[map[struct{}]struct{}])(nil) 260 ) 261 262 // Deprecated: A Writer uses a parquet schema and sequence of Go values to 263 // produce a parquet file to an io.Writer. 264 // 265 // This example showcases a typical use of parquet writers: 266 // 267 // writer := parquet.NewWriter(output) 268 // 269 // for _, row := range rows { 270 // if err := writer.Write(row); err != nil { 271 // ... 272 // } 273 // } 274 // 275 // if err := writer.Close(); err != nil { 276 // ... 277 // } 278 // 279 // The Writer type optimizes for minimal memory usage, each page is written as 280 // soon as it has been filled so only a single page per column needs to be held 281 // in memory and as a result, there are no opportunities to sort rows within an 282 // entire row group. Programs that need to produce parquet files with sorted 283 // row groups should use the Buffer type to buffer and sort the rows prior to 284 // writing them to a Writer. 285 // 286 // For programs building with Go 1.18 or later, the GenericWriter[T] type 287 // supersedes this one. 288 type Writer struct { 289 output io.Writer 290 config *WriterConfig 291 schema *Schema 292 writer *writer 293 rowbuf []Row 294 } 295 296 // NewWriter constructs a parquet writer writing a file to the given io.Writer. 297 // 298 // The function panics if the writer configuration is invalid. Programs that 299 // cannot guarantee the validity of the options passed to NewWriter should 300 // construct the writer configuration independently prior to calling this 301 // function: 302 // 303 // config, err := parquet.NewWriterConfig(options...) 304 // if err != nil { 305 // // handle the configuration error 306 // ... 307 // } else { 308 // // this call to create a writer is guaranteed not to panic 309 // writer := parquet.NewWriter(output, config) 310 // ... 311 // } 312 func NewWriter(output io.Writer, options ...WriterOption) *Writer { 313 config, err := NewWriterConfig(options...) 314 if err != nil { 315 panic(err) 316 } 317 w := &Writer{ 318 output: output, 319 config: config, 320 } 321 if config.Schema != nil { 322 w.configure(config.Schema) 323 } 324 return w 325 } 326 327 func (w *Writer) configure(schema *Schema) { 328 if schema != nil { 329 w.config.Schema = schema 330 w.schema = schema 331 w.writer = newWriter(w.output, w.config) 332 } 333 } 334 335 // Close must be called after all values were produced to the writer in order to 336 // flush all buffers and write the parquet footer. 337 func (w *Writer) Close() error { 338 if w.writer != nil { 339 return w.writer.close() 340 } 341 return nil 342 } 343 344 // Flush flushes all buffers into a row group to the underlying io.Writer. 345 // 346 // Flush is called automatically on Close, it is only useful to call explicitly 347 // if the application needs to limit the size of row groups or wants to produce 348 // multiple row groups per file. 349 // 350 // If the writer attempts to create more than MaxRowGroups row groups the method 351 // returns ErrTooManyRowGroups. 352 func (w *Writer) Flush() error { 353 if w.writer != nil { 354 return w.writer.flush() 355 } 356 return nil 357 } 358 359 // Reset clears the state of the writer without flushing any of the buffers, 360 // and setting the output to the io.Writer passed as argument, allowing the 361 // writer to be reused to produce another parquet file. 362 // 363 // Reset may be called at any time, including after a writer was closed. 364 func (w *Writer) Reset(output io.Writer) { 365 if w.output = output; w.writer != nil { 366 w.writer.reset(w.output) 367 } 368 } 369 370 // Write is called to write another row to the parquet file. 371 // 372 // The method uses the parquet schema configured on w to traverse the Go value 373 // and decompose it into a set of columns and values. If no schema were passed 374 // to NewWriter, it is deducted from the Go type of the row, which then have to 375 // be a struct or pointer to struct. 376 func (w *Writer) Write(row interface{}) error { 377 if w.schema == nil { 378 w.configure(SchemaOf(row)) 379 } 380 if cap(w.rowbuf) == 0 { 381 w.rowbuf = make([]Row, 1) 382 } else { 383 w.rowbuf = w.rowbuf[:1] 384 } 385 defer clearRows(w.rowbuf) 386 w.rowbuf[0] = w.schema.Deconstruct(w.rowbuf[0][:0], row) 387 _, err := w.WriteRows(w.rowbuf) 388 return err 389 } 390 391 // WriteRows is called to write rows to the parquet file. 392 // 393 // The Writer must have been given a schema when NewWriter was called, otherwise 394 // the structure of the parquet file cannot be determined from the row only. 395 // 396 // The row is expected to contain values for each column of the writer's schema, 397 // in the order produced by the parquet.(*Schema).Deconstruct method. 398 func (w *Writer) WriteRows(rows []Row) (int, error) { 399 return w.writer.WriteRows(rows) 400 } 401 402 // WriteRowGroup writes a row group to the parquet file. 403 // 404 // Buffered rows will be flushed prior to writing rows from the group, unless 405 // the row group was empty in which case nothing is written to the file. 406 // 407 // The content of the row group is flushed to the writer; after the method 408 // returns successfully, the row group will be empty and in ready to be reused. 409 func (w *Writer) WriteRowGroup(rowGroup RowGroup) (int64, error) { 410 rowGroupSchema := rowGroup.Schema() 411 switch { 412 case rowGroupSchema == nil: 413 return 0, ErrRowGroupSchemaMissing 414 case w.schema == nil: 415 w.configure(rowGroupSchema) 416 case !nodesAreEqual(w.schema, rowGroupSchema): 417 return 0, ErrRowGroupSchemaMismatch 418 } 419 if err := w.writer.flush(); err != nil { 420 return 0, err 421 } 422 w.writer.configureBloomFilters(rowGroup.ColumnChunks()) 423 rows := rowGroup.Rows() 424 defer rows.Close() 425 n, err := CopyRows(w.writer, rows) 426 if err != nil { 427 return n, err 428 } 429 return w.writer.writeRowGroup(rowGroup.Schema(), rowGroup.SortingColumns()) 430 } 431 432 // ReadRowsFrom reads rows from the reader passed as arguments and writes them 433 // to w. 434 // 435 // This is similar to calling WriteRow repeatedly, but will be more efficient 436 // if optimizations are supported by the reader. 437 func (w *Writer) ReadRowsFrom(rows RowReader) (written int64, err error) { 438 if w.schema == nil { 439 if r, ok := rows.(RowReaderWithSchema); ok { 440 w.configure(r.Schema()) 441 } 442 } 443 if cap(w.rowbuf) < defaultRowBufferSize { 444 w.rowbuf = make([]Row, defaultRowBufferSize) 445 } else { 446 w.rowbuf = w.rowbuf[:cap(w.rowbuf)] 447 } 448 return copyRows(w.writer, rows, w.rowbuf) 449 } 450 451 // Schema returns the schema of rows written by w. 452 // 453 // The returned value will be nil if no schema has yet been configured on w. 454 func (w *Writer) Schema() *Schema { return w.schema } 455 456 // SetKeyValueMetadata sets a key/value pair in the Parquet file metadata. 457 // 458 // Keys are assumed to be unique, if the same key is repeated multiple times the 459 // last value is retained. While the parquet format does not require unique keys, 460 // this design decision was made to optimize for the most common use case where 461 // applications leverage this extension mechanism to associate single values to 462 // keys. This may create incompatibilities with other parquet libraries, or may 463 // cause some key/value pairs to be lost when open parquet files written with 464 // repeated keys. We can revisit this decision if it ever becomes a blocker. 465 func (w *Writer) SetKeyValueMetadata(key, value string) { 466 for i, kv := range w.writer.metadata { 467 if kv.Key == key { 468 kv.Value = value 469 w.writer.metadata[i] = kv 470 return 471 } 472 } 473 w.writer.metadata = append(w.writer.metadata, format.KeyValue{ 474 Key: key, 475 Value: value, 476 }) 477 } 478 479 type writer struct { 480 buffer *bufio.Writer 481 writer offsetTrackingWriter 482 values [][]Value 483 numRows int64 484 maxRows int64 485 486 createdBy string 487 metadata []format.KeyValue 488 489 columns []*writerColumn 490 columnChunk []format.ColumnChunk 491 columnIndex []format.ColumnIndex 492 offsetIndex []format.OffsetIndex 493 494 columnOrders []format.ColumnOrder 495 schemaElements []format.SchemaElement 496 rowGroups []format.RowGroup 497 columnIndexes [][]format.ColumnIndex 498 offsetIndexes [][]format.OffsetIndex 499 sortingColumns []format.SortingColumn 500 } 501 502 func newWriter(output io.Writer, config *WriterConfig) *writer { 503 w := new(writer) 504 if config.WriteBufferSize <= 0 { 505 w.writer.Reset(output) 506 } else { 507 w.buffer = bufio.NewWriterSize(output, config.WriteBufferSize) 508 w.writer.Reset(w.buffer) 509 } 510 w.maxRows = config.MaxRowsPerRowGroup 511 w.createdBy = config.CreatedBy 512 w.metadata = make([]format.KeyValue, 0, len(config.KeyValueMetadata)) 513 for k, v := range config.KeyValueMetadata { 514 w.metadata = append(w.metadata, format.KeyValue{Key: k, Value: v}) 515 } 516 sortKeyValueMetadata(w.metadata) 517 w.sortingColumns = make([]format.SortingColumn, len(config.Sorting.SortingColumns)) 518 519 config.Schema.forEachNode(func(name string, node Node) { 520 nodeType := node.Type() 521 522 repetitionType := (*format.FieldRepetitionType)(nil) 523 if node != config.Schema { // the root has no repetition type 524 repetitionType = fieldRepetitionTypePtrOf(node) 525 } 526 // For backward compatibility with older readers, the parquet specification 527 // recommends to set the scale and precision on schema elements when the 528 // column is of logical type decimal. 529 logicalType := nodeType.LogicalType() 530 scale, precision := (*int32)(nil), (*int32)(nil) 531 if logicalType != nil && logicalType.Decimal != nil { 532 scale = &logicalType.Decimal.Scale 533 precision = &logicalType.Decimal.Precision 534 } 535 536 typeLength := (*int32)(nil) 537 if n := int32(nodeType.Length()); n > 0 { 538 typeLength = &n 539 } 540 541 w.schemaElements = append(w.schemaElements, format.SchemaElement{ 542 Type: nodeType.PhysicalType(), 543 TypeLength: typeLength, 544 RepetitionType: repetitionType, 545 Name: name, 546 NumChildren: int32(len(node.Fields())), 547 ConvertedType: nodeType.ConvertedType(), 548 Scale: scale, 549 Precision: precision, 550 FieldID: int32(node.ID()), 551 LogicalType: logicalType, 552 }) 553 }) 554 555 dataPageType := format.DataPage 556 if config.DataPageVersion == 2 { 557 dataPageType = format.DataPageV2 558 } 559 560 defaultCompression := config.Compression 561 if defaultCompression == nil { 562 defaultCompression = &Uncompressed 563 } 564 565 // Those buffers are scratch space used to generate the page header and 566 // content, they are shared by all column chunks because they are only 567 // used during calls to writeDictionaryPage or writeDataPage, which are 568 // not done concurrently. 569 buffers := new(writerBuffers) 570 571 forEachLeafColumnOf(config.Schema, func(leaf leafColumn) { 572 encoding := encodingOf(leaf.node) 573 dictionary := Dictionary(nil) 574 columnType := leaf.node.Type() 575 columnIndex := int(leaf.columnIndex) 576 compression := leaf.node.Compression() 577 578 if compression == nil { 579 compression = defaultCompression 580 } 581 582 if isDictionaryEncoding(encoding) { 583 dictBuffer := columnType.NewValues( 584 make([]byte, 0, defaultDictBufferSize), 585 nil, 586 ) 587 dictionary = columnType.NewDictionary(columnIndex, 0, dictBuffer) 588 columnType = dictionary.Type() 589 } 590 591 c := &writerColumn{ 592 buffers: buffers, 593 pool: config.ColumnPageBuffers, 594 columnPath: leaf.path, 595 columnType: columnType, 596 columnIndex: columnType.NewColumnIndexer(config.ColumnIndexSizeLimit), 597 columnFilter: searchBloomFilterColumn(config.BloomFilters, leaf.path), 598 compression: compression, 599 dictionary: dictionary, 600 dataPageType: dataPageType, 601 maxRepetitionLevel: leaf.maxRepetitionLevel, 602 maxDefinitionLevel: leaf.maxDefinitionLevel, 603 bufferIndex: int32(leaf.columnIndex), 604 bufferSize: int32(float64(config.PageBufferSize) * 0.98), 605 writePageStats: config.DataPageStatistics, 606 encodings: make([]format.Encoding, 0, 3), 607 // Data pages in version 2 can omit compression when dictionary 608 // encoding is employed; only the dictionary page needs to be 609 // compressed, the data pages are encoded with the hybrid 610 // RLE/Bit-Pack encoding which doesn't benefit from an extra 611 // compression layer. 612 isCompressed: isCompressed(compression) && (dataPageType != format.DataPageV2 || dictionary == nil), 613 } 614 615 c.header.encoder.Reset(c.header.protocol.NewWriter(&buffers.header)) 616 617 if leaf.maxDefinitionLevel > 0 { 618 c.encodings = addEncoding(c.encodings, format.RLE) 619 } 620 621 if isDictionaryEncoding(encoding) { 622 c.encodings = addEncoding(c.encodings, format.Plain) 623 } 624 625 c.encoding = encoding 626 c.encodings = addEncoding(c.encodings, c.encoding.Encoding()) 627 sortPageEncodings(c.encodings) 628 629 w.columns = append(w.columns, c) 630 631 if sortingIndex := searchSortingColumn(config.Sorting.SortingColumns, leaf.path); sortingIndex < len(w.sortingColumns) { 632 w.sortingColumns[sortingIndex] = format.SortingColumn{ 633 ColumnIdx: int32(leaf.columnIndex), 634 Descending: config.Sorting.SortingColumns[sortingIndex].Descending(), 635 NullsFirst: config.Sorting.SortingColumns[sortingIndex].NullsFirst(), 636 } 637 } 638 }) 639 640 // Pre-allocate the backing array so that in most cases where the rows 641 // contain a single value we will hit collocated memory areas when writing 642 // rows to the writer. This won't benefit repeated columns much but in that 643 // case we would just waste a bit of memory which we can afford. 644 values := make([]Value, len(w.columns)) 645 w.values = make([][]Value, len(w.columns)) 646 for i := range values { 647 w.values[i] = values[i : i : i+1] 648 } 649 650 w.columnChunk = make([]format.ColumnChunk, len(w.columns)) 651 w.columnIndex = make([]format.ColumnIndex, len(w.columns)) 652 w.offsetIndex = make([]format.OffsetIndex, len(w.columns)) 653 w.columnOrders = make([]format.ColumnOrder, len(w.columns)) 654 655 for i, c := range w.columns { 656 w.columnChunk[i] = format.ColumnChunk{ 657 MetaData: format.ColumnMetaData{ 658 Type: format.Type(c.columnType.Kind()), 659 Encoding: c.encodings, 660 PathInSchema: c.columnPath, 661 Codec: c.compression.CompressionCodec(), 662 KeyValueMetadata: nil, // TODO 663 }, 664 } 665 } 666 667 for i, c := range w.columns { 668 c.columnChunk = &w.columnChunk[i] 669 c.offsetIndex = &w.offsetIndex[i] 670 } 671 672 for i, c := range w.columns { 673 w.columnOrders[i] = *c.columnType.ColumnOrder() 674 } 675 676 return w 677 } 678 679 func (w *writer) reset(writer io.Writer) { 680 if w.buffer == nil { 681 w.writer.Reset(writer) 682 } else { 683 w.buffer.Reset(writer) 684 w.writer.Reset(w.buffer) 685 } 686 for _, c := range w.columns { 687 c.reset() 688 } 689 for i := range w.rowGroups { 690 w.rowGroups[i] = format.RowGroup{} 691 } 692 for i := range w.columnIndexes { 693 w.columnIndexes[i] = nil 694 } 695 for i := range w.offsetIndexes { 696 w.offsetIndexes[i] = nil 697 } 698 w.rowGroups = w.rowGroups[:0] 699 w.columnIndexes = w.columnIndexes[:0] 700 w.offsetIndexes = w.offsetIndexes[:0] 701 } 702 703 func (w *writer) close() error { 704 if err := w.writeFileHeader(); err != nil { 705 return err 706 } 707 if err := w.flush(); err != nil { 708 return err 709 } 710 if err := w.writeFileFooter(); err != nil { 711 return err 712 } 713 if w.buffer != nil { 714 return w.buffer.Flush() 715 } 716 return nil 717 } 718 719 func (w *writer) flush() error { 720 _, err := w.writeRowGroup(nil, nil) 721 return err 722 } 723 724 func (w *writer) writeFileHeader() error { 725 if w.writer.writer == nil { 726 return io.ErrClosedPipe 727 } 728 if w.writer.offset == 0 { 729 _, err := w.writer.WriteString("PAR1") 730 return err 731 } 732 return nil 733 } 734 735 func (w *writer) configureBloomFilters(columnChunks []ColumnChunk) { 736 for i, c := range w.columns { 737 if c.columnFilter != nil { 738 c.resizeBloomFilter(columnChunks[i].NumValues()) 739 } 740 } 741 } 742 743 func (w *writer) writeFileFooter() error { 744 // The page index is composed of two sections: column and offset indexes. 745 // They are written after the row groups, right before the footer (which 746 // is written by the parent Writer.Close call). 747 // 748 // This section both writes the page index and generates the values of 749 // ColumnIndexOffset, ColumnIndexLength, OffsetIndexOffset, and 750 // OffsetIndexLength in the corresponding columns of the file metadata. 751 // 752 // Note: the page index is always written, even if we created data pages v1 753 // because the parquet format is backward compatible in this case. Older 754 // readers will simply ignore this section since they do not know how to 755 // decode its content, nor have loaded any metadata to reference it. 756 protocol := new(thrift.CompactProtocol) 757 encoder := thrift.NewEncoder(protocol.NewWriter(&w.writer)) 758 759 for i, columnIndexes := range w.columnIndexes { 760 rowGroup := &w.rowGroups[i] 761 for j := range columnIndexes { 762 column := &rowGroup.Columns[j] 763 column.ColumnIndexOffset = w.writer.offset 764 if err := encoder.Encode(&columnIndexes[j]); err != nil { 765 return err 766 } 767 column.ColumnIndexLength = int32(w.writer.offset - column.ColumnIndexOffset) 768 } 769 } 770 771 for i, offsetIndexes := range w.offsetIndexes { 772 rowGroup := &w.rowGroups[i] 773 for j := range offsetIndexes { 774 column := &rowGroup.Columns[j] 775 column.OffsetIndexOffset = w.writer.offset 776 if err := encoder.Encode(&offsetIndexes[j]); err != nil { 777 return err 778 } 779 column.OffsetIndexLength = int32(w.writer.offset - column.OffsetIndexOffset) 780 } 781 } 782 783 numRows := int64(0) 784 for rowGroupIndex := range w.rowGroups { 785 numRows += w.rowGroups[rowGroupIndex].NumRows 786 } 787 788 footer, err := thrift.Marshal(new(thrift.CompactProtocol), &format.FileMetaData{ 789 Version: 1, 790 Schema: w.schemaElements, 791 NumRows: numRows, 792 RowGroups: w.rowGroups, 793 KeyValueMetadata: w.metadata, 794 CreatedBy: w.createdBy, 795 ColumnOrders: w.columnOrders, 796 }) 797 if err != nil { 798 return err 799 } 800 801 length := len(footer) 802 footer = append(footer, 0, 0, 0, 0) 803 footer = append(footer, "PAR1"...) 804 binary.LittleEndian.PutUint32(footer[length:], uint32(length)) 805 806 _, err = w.writer.Write(footer) 807 return err 808 } 809 810 func (w *writer) writeRowGroup(rowGroupSchema *Schema, rowGroupSortingColumns []SortingColumn) (int64, error) { 811 numRows := w.columns[0].totalRowCount() 812 if numRows == 0 { 813 return 0, nil 814 } 815 816 if len(w.rowGroups) == MaxRowGroups { 817 return 0, ErrTooManyRowGroups 818 } 819 820 defer func() { 821 w.numRows = 0 822 for _, c := range w.columns { 823 c.reset() 824 } 825 for i := range w.columnIndex { 826 w.columnIndex[i] = format.ColumnIndex{} 827 } 828 }() 829 830 for _, c := range w.columns { 831 if err := c.flush(); err != nil { 832 return 0, err 833 } 834 if err := c.flushFilterPages(); err != nil { 835 return 0, err 836 } 837 } 838 839 if err := w.writeFileHeader(); err != nil { 840 return 0, err 841 } 842 fileOffset := w.writer.offset 843 844 for _, c := range w.columns { 845 if len(c.filter) > 0 { 846 c.columnChunk.MetaData.BloomFilterOffset = w.writer.offset 847 if err := c.writeBloomFilter(&w.writer); err != nil { 848 return 0, err 849 } 850 } 851 } 852 853 for i, c := range w.columns { 854 w.columnIndex[i] = format.ColumnIndex(c.columnIndex.ColumnIndex()) 855 856 if c.dictionary != nil { 857 c.columnChunk.MetaData.DictionaryPageOffset = w.writer.offset 858 if err := c.writeDictionaryPage(&w.writer, c.dictionary); err != nil { 859 return 0, fmt.Errorf("writing dictionary page of row group colum %d: %w", i, err) 860 } 861 } 862 863 dataPageOffset := w.writer.offset 864 c.columnChunk.MetaData.DataPageOffset = dataPageOffset 865 for j := range c.offsetIndex.PageLocations { 866 c.offsetIndex.PageLocations[j].Offset += dataPageOffset 867 } 868 869 for _, page := range c.pages { 870 if _, err := io.Copy(&w.writer, page); err != nil { 871 return 0, fmt.Errorf("writing buffered pages of row group column %d: %w", i, err) 872 } 873 } 874 } 875 876 totalByteSize := int64(0) 877 totalCompressedSize := int64(0) 878 879 for i := range w.columnChunk { 880 c := &w.columnChunk[i].MetaData 881 sortPageEncodingStats(c.EncodingStats) 882 totalByteSize += int64(c.TotalUncompressedSize) 883 totalCompressedSize += int64(c.TotalCompressedSize) 884 } 885 886 sortingColumns := w.sortingColumns 887 if len(sortingColumns) == 0 && len(rowGroupSortingColumns) > 0 { 888 sortingColumns = make([]format.SortingColumn, 0, len(rowGroupSortingColumns)) 889 forEachLeafColumnOf(rowGroupSchema, func(leaf leafColumn) { 890 if sortingIndex := searchSortingColumn(rowGroupSortingColumns, leaf.path); sortingIndex < len(sortingColumns) { 891 sortingColumns[sortingIndex] = format.SortingColumn{ 892 ColumnIdx: int32(leaf.columnIndex), 893 Descending: rowGroupSortingColumns[sortingIndex].Descending(), 894 NullsFirst: rowGroupSortingColumns[sortingIndex].NullsFirst(), 895 } 896 } 897 }) 898 } 899 900 columns := make([]format.ColumnChunk, len(w.columnChunk)) 901 copy(columns, w.columnChunk) 902 903 columnIndex := make([]format.ColumnIndex, len(w.columnIndex)) 904 copy(columnIndex, w.columnIndex) 905 906 offsetIndex := make([]format.OffsetIndex, len(w.offsetIndex)) 907 copy(offsetIndex, w.offsetIndex) 908 909 for i := range columns { 910 c := &columns[i] 911 c.MetaData.EncodingStats = make([]format.PageEncodingStats, len(c.MetaData.EncodingStats)) 912 copy(c.MetaData.EncodingStats, w.columnChunk[i].MetaData.EncodingStats) 913 } 914 915 for i := range offsetIndex { 916 c := &offsetIndex[i] 917 c.PageLocations = make([]format.PageLocation, len(c.PageLocations)) 918 copy(c.PageLocations, w.offsetIndex[i].PageLocations) 919 } 920 921 w.rowGroups = append(w.rowGroups, format.RowGroup{ 922 Columns: columns, 923 TotalByteSize: totalByteSize, 924 NumRows: numRows, 925 SortingColumns: sortingColumns, 926 FileOffset: fileOffset, 927 TotalCompressedSize: totalCompressedSize, 928 Ordinal: int16(len(w.rowGroups)), 929 }) 930 931 w.columnIndexes = append(w.columnIndexes, columnIndex) 932 w.offsetIndexes = append(w.offsetIndexes, offsetIndex) 933 return numRows, nil 934 } 935 936 func (w *writer) WriteRows(rows []Row) (int, error) { 937 return w.writeRows(len(rows), func(start, end int) (int, error) { 938 defer func() { 939 for i, values := range w.values { 940 clearValues(values) 941 w.values[i] = values[:0] 942 } 943 }() 944 945 // TODO: if an error occurs in this method the writer may be left in an 946 // partially functional state. Applications are not expected to continue 947 // using the writer after getting an error, but maybe we could ensure that 948 // we are preventing further use as well? 949 for _, row := range rows[start:end] { 950 row.Range(func(columnIndex int, columnValues []Value) bool { 951 w.values[columnIndex] = append(w.values[columnIndex], columnValues...) 952 return true 953 }) 954 } 955 956 for i, values := range w.values { 957 if len(values) > 0 { 958 if err := w.columns[i].writeRows(values); err != nil { 959 return 0, err 960 } 961 } 962 } 963 964 return end - start, nil 965 }) 966 } 967 968 func (w *writer) writeRows(numRows int, write func(i, j int) (int, error)) (int, error) { 969 written := 0 970 971 for written < numRows { 972 remain := w.maxRows - w.numRows 973 length := numRows - written 974 975 if remain == 0 { 976 remain = w.maxRows 977 978 if err := w.flush(); err != nil { 979 return written, err 980 } 981 } 982 983 if remain < int64(length) { 984 length = int(remain) 985 } 986 987 // Since the writer cannot flush pages across row boundaries, calls to 988 // WriteRows with very large slices can result in greatly exceeding the 989 // target page size. To set a limit to the impact of these large writes 990 // we chunk the input in slices of 64 rows. 991 // 992 // Note that this mechanism isn't perfect; for example, values may hold 993 // large byte slices which could still cause the column buffers to grow 994 // beyond the target page size. 995 const maxRowsPerWrite = 64 996 if length > maxRowsPerWrite { 997 length = maxRowsPerWrite 998 } 999 1000 n, err := write(written, written+length) 1001 written += n 1002 w.numRows += int64(n) 1003 if err != nil { 1004 return written, err 1005 } 1006 } 1007 1008 return written, nil 1009 } 1010 1011 // The WriteValues method is intended to work in pair with WritePage to allow 1012 // programs to target writing values to specific columns of of the writer. 1013 func (w *writer) WriteValues(values []Value) (numValues int, err error) { 1014 return w.columns[values[0].Column()].WriteValues(values) 1015 } 1016 1017 // One writerBuffers is used by each writer instance, the memory buffers here 1018 // are shared by all columns of the writer because serialization is not done 1019 // concurrently, which helps keep memory utilization low, both in the total 1020 // footprint and GC cost. 1021 // 1022 // The type also exposes helper methods to facilitate the generation of parquet 1023 // pages. A scratch space is used when serialization requires combining multiple 1024 // buffers or compressing the page data, with double-buffering technique being 1025 // employed by swapping the scratch and page buffers to minimize memory copies. 1026 type writerBuffers struct { 1027 header bytes.Buffer // buffer where page headers are encoded 1028 repetitions []byte // buffer used to encode repetition levels 1029 definitions []byte // buffer used to encode definition levels 1030 page []byte // page buffer holding the page data 1031 scratch []byte // scratch space used for compression 1032 } 1033 1034 func (wb *writerBuffers) crc32() (checksum uint32) { 1035 checksum = crc32.Update(checksum, crc32.IEEETable, wb.repetitions) 1036 checksum = crc32.Update(checksum, crc32.IEEETable, wb.definitions) 1037 checksum = crc32.Update(checksum, crc32.IEEETable, wb.page) 1038 return checksum 1039 } 1040 1041 func (wb *writerBuffers) size() int { 1042 return len(wb.repetitions) + len(wb.definitions) + len(wb.page) 1043 } 1044 1045 func (wb *writerBuffers) reset() { 1046 wb.repetitions = wb.repetitions[:0] 1047 wb.definitions = wb.definitions[:0] 1048 wb.page = wb.page[:0] 1049 } 1050 1051 func encodeLevels(dst, src []byte, maxLevel byte) ([]byte, error) { 1052 bitWidth := bits.Len8(maxLevel) 1053 return levelEncodingsRLE[bitWidth-1].EncodeLevels(dst, src) 1054 } 1055 1056 func (wb *writerBuffers) encodeRepetitionLevels(page Page, maxRepetitionLevel byte) (err error) { 1057 wb.repetitions, err = encodeLevels(wb.repetitions, page.RepetitionLevels(), maxRepetitionLevel) 1058 return 1059 } 1060 1061 func (wb *writerBuffers) encodeDefinitionLevels(page Page, maxDefinitionLevel byte) (err error) { 1062 wb.definitions, err = encodeLevels(wb.definitions, page.DefinitionLevels(), maxDefinitionLevel) 1063 return 1064 } 1065 1066 func (wb *writerBuffers) prependLevelsToDataPageV1(maxRepetitionLevel, maxDefinitionLevel byte) { 1067 hasRepetitionLevels := maxRepetitionLevel > 0 1068 hasDefinitionLevels := maxDefinitionLevel > 0 1069 1070 if hasRepetitionLevels || hasDefinitionLevels { 1071 wb.scratch = wb.scratch[:0] 1072 // In data pages v1, the repetition and definition levels are prefixed 1073 // with the 4 bytes length of the sections. While the parquet-format 1074 // documentation indicates that the length prefix is part of the hybrid 1075 // RLE/Bit-Pack encoding, this is the only condition where it is used 1076 // so we treat it as a special case rather than implementing it in the 1077 // encoding. 1078 // 1079 // Reference https://github.com/apache/parquet-format/blob/master/Encodings.md#run-length-encoding--bit-packing-hybrid-rle--3 1080 if hasRepetitionLevels { 1081 wb.scratch = plain.AppendInt32(wb.scratch, int32(len(wb.repetitions))) 1082 wb.scratch = append(wb.scratch, wb.repetitions...) 1083 wb.repetitions = wb.repetitions[:0] 1084 } 1085 if hasDefinitionLevels { 1086 wb.scratch = plain.AppendInt32(wb.scratch, int32(len(wb.definitions))) 1087 wb.scratch = append(wb.scratch, wb.definitions...) 1088 wb.definitions = wb.definitions[:0] 1089 } 1090 wb.scratch = append(wb.scratch, wb.page...) 1091 wb.swapPageAndScratchBuffers() 1092 } 1093 } 1094 1095 func (wb *writerBuffers) encode(page Page, enc encoding.Encoding) (err error) { 1096 pageType := page.Type() 1097 pageData := page.Data() 1098 wb.page, err = pageType.Encode(wb.page[:0], pageData, enc) 1099 return err 1100 } 1101 1102 func (wb *writerBuffers) compress(codec compress.Codec) (err error) { 1103 wb.scratch, err = codec.Encode(wb.scratch[:0], wb.page) 1104 wb.swapPageAndScratchBuffers() 1105 return err 1106 } 1107 1108 func (wb *writerBuffers) swapPageAndScratchBuffers() { 1109 wb.page, wb.scratch = wb.scratch, wb.page[:0] 1110 } 1111 1112 type writerColumn struct { 1113 pool BufferPool 1114 pages []io.ReadWriteSeeker 1115 1116 columnPath columnPath 1117 columnType Type 1118 columnIndex ColumnIndexer 1119 columnBuffer ColumnBuffer 1120 columnFilter BloomFilterColumn 1121 encoding encoding.Encoding 1122 compression compress.Codec 1123 dictionary Dictionary 1124 1125 dataPageType format.PageType 1126 maxRepetitionLevel byte 1127 maxDefinitionLevel byte 1128 1129 buffers *writerBuffers 1130 1131 header struct { 1132 protocol thrift.CompactProtocol 1133 encoder thrift.Encoder 1134 } 1135 1136 filter []byte 1137 numRows int64 1138 bufferIndex int32 1139 bufferSize int32 1140 writePageStats bool 1141 isCompressed bool 1142 encodings []format.Encoding 1143 1144 columnChunk *format.ColumnChunk 1145 offsetIndex *format.OffsetIndex 1146 } 1147 1148 func (c *writerColumn) reset() { 1149 if c.columnBuffer != nil { 1150 c.columnBuffer.Reset() 1151 } 1152 if c.columnIndex != nil { 1153 c.columnIndex.Reset() 1154 } 1155 if c.dictionary != nil { 1156 c.dictionary.Reset() 1157 } 1158 for _, page := range c.pages { 1159 c.pool.PutBuffer(page) 1160 } 1161 for i := range c.pages { 1162 c.pages[i] = nil 1163 } 1164 c.pages = c.pages[:0] 1165 // Bloom filters may change in size between row groups, but we retain the 1166 // buffer to avoid reallocating large memory blocks. 1167 c.filter = c.filter[:0] 1168 c.numRows = 0 1169 // Reset the fields of column chunks that change between row groups, 1170 // but keep the ones that remain unchanged. 1171 c.columnChunk.MetaData.NumValues = 0 1172 c.columnChunk.MetaData.TotalUncompressedSize = 0 1173 c.columnChunk.MetaData.TotalCompressedSize = 0 1174 c.columnChunk.MetaData.DataPageOffset = 0 1175 c.columnChunk.MetaData.DictionaryPageOffset = 0 1176 c.columnChunk.MetaData.Statistics = format.Statistics{} 1177 c.columnChunk.MetaData.EncodingStats = c.columnChunk.MetaData.EncodingStats[:0] 1178 c.columnChunk.MetaData.BloomFilterOffset = 0 1179 c.offsetIndex.PageLocations = c.offsetIndex.PageLocations[:0] 1180 } 1181 1182 func (c *writerColumn) totalRowCount() int64 { 1183 n := c.numRows 1184 if c.columnBuffer != nil { 1185 n += int64(c.columnBuffer.Len()) 1186 } 1187 return n 1188 } 1189 1190 func (c *writerColumn) flush() (err error) { 1191 if c.columnBuffer.Len() > 0 { 1192 defer c.columnBuffer.Reset() 1193 _, err = c.writeDataPage(c.columnBuffer.Page()) 1194 } 1195 return err 1196 } 1197 1198 func (c *writerColumn) flushFilterPages() error { 1199 if c.columnFilter == nil { 1200 return nil 1201 } 1202 1203 // If there is a dictionary, it contains all the values that we need to 1204 // write to the filter. 1205 if dict := c.dictionary; dict != nil { 1206 // Need to always attempt to resize the filter, as the writer might 1207 // be reused after resetting which would have reset the length of 1208 // the filter to 0. 1209 c.resizeBloomFilter(int64(dict.Len())) 1210 return c.writePageToFilter(dict.Page()) 1211 } 1212 1213 // When the filter was already allocated, pages have been written to it as 1214 // they were seen by the column writer. 1215 if len(c.filter) > 0 { 1216 return nil 1217 } 1218 1219 // When the filter was not allocated, the writer did not know how many 1220 // values were going to be seen and therefore could not properly size the 1221 // filter ahead of time. In this case, we read back all the pages that we 1222 // have encoded and copy their values back to the filter. 1223 // 1224 // A prior implementation of the column writer used to create in-memory 1225 // copies of the pages to avoid this decoding step; however, this unbounded 1226 // allocation caused memory exhaustion in production applications. CPU being 1227 // a somewhat more stretchable resource, we prefer spending time on this 1228 // decoding step than having to trigger incident response when production 1229 // systems are getting OOM-Killed. 1230 c.resizeBloomFilter(c.columnChunk.MetaData.NumValues) 1231 1232 column := &Column{ 1233 // Set all the fields required by the decodeDataPage* methods. 1234 typ: c.columnType, 1235 encoding: c.encoding, 1236 compression: c.compression, 1237 maxRepetitionLevel: c.maxRepetitionLevel, 1238 maxDefinitionLevel: c.maxDefinitionLevel, 1239 index: int16(c.bufferIndex), 1240 } 1241 1242 rbuf, pool := getBufioReader(nil, 1024) 1243 pbuf := (*buffer)(nil) 1244 defer func() { 1245 putBufioReader(rbuf, pool) 1246 if pbuf != nil { 1247 pbuf.unref() 1248 } 1249 }() 1250 1251 decoder := thrift.NewDecoder(c.header.protocol.NewReader(rbuf)) 1252 1253 for _, p := range c.pages { 1254 rbuf.Reset(p) 1255 1256 header := new(format.PageHeader) 1257 if err := decoder.Decode(header); err != nil { 1258 return err 1259 } 1260 1261 if pbuf != nil { 1262 pbuf.unref() 1263 } 1264 pbuf = buffers.get(int(header.CompressedPageSize)) 1265 if _, err := io.ReadFull(rbuf, pbuf.data); err != nil { 1266 return err 1267 } 1268 if _, err := p.Seek(0, io.SeekStart); err != nil { 1269 return err 1270 } 1271 1272 var page Page 1273 var err error 1274 1275 switch header.Type { 1276 case format.DataPage: 1277 page, err = column.decodeDataPageV1(DataPageHeaderV1{header.DataPageHeader}, pbuf, nil, header.UncompressedPageSize) 1278 case format.DataPageV2: 1279 page, err = column.decodeDataPageV2(DataPageHeaderV2{header.DataPageHeaderV2}, pbuf, nil, header.UncompressedPageSize) 1280 } 1281 if page != nil { 1282 err = c.writePageToFilter(page) 1283 Release(page) 1284 } 1285 if err != nil { 1286 return err 1287 } 1288 } 1289 1290 return nil 1291 } 1292 1293 func (c *writerColumn) resizeBloomFilter(numValues int64) { 1294 filterSize := c.columnFilter.Size(numValues) 1295 if cap(c.filter) < filterSize { 1296 c.filter = make([]byte, filterSize) 1297 } else { 1298 c.filter = c.filter[:filterSize] 1299 for i := range c.filter { 1300 c.filter[i] = 0 1301 } 1302 } 1303 } 1304 1305 func (c *writerColumn) newColumnBuffer() ColumnBuffer { 1306 column := c.columnType.NewColumnBuffer(int(c.bufferIndex), c.columnType.EstimateNumValues(int(c.bufferSize))) 1307 switch { 1308 case c.maxRepetitionLevel > 0: 1309 column = newRepeatedColumnBuffer(column, c.maxRepetitionLevel, c.maxDefinitionLevel, nullsGoLast) 1310 case c.maxDefinitionLevel > 0: 1311 column = newOptionalColumnBuffer(column, c.maxDefinitionLevel, nullsGoLast) 1312 } 1313 return column 1314 } 1315 1316 func (c *writerColumn) writeRows(rows []Value) error { 1317 if c.columnBuffer == nil { 1318 // Lazily create the row group column so we don't need to allocate it if 1319 // rows are not written individually to the column. 1320 c.columnBuffer = c.newColumnBuffer() 1321 } 1322 if _, err := c.columnBuffer.WriteValues(rows); err != nil { 1323 return err 1324 } 1325 if c.columnBuffer.Size() >= int64(c.bufferSize) { 1326 return c.flush() 1327 } 1328 return nil 1329 } 1330 1331 func (c *writerColumn) WriteValues(values []Value) (numValues int, err error) { 1332 if c.columnBuffer == nil { 1333 c.columnBuffer = c.newColumnBuffer() 1334 } 1335 return c.columnBuffer.WriteValues(values) 1336 } 1337 1338 func (c *writerColumn) writeBloomFilter(w io.Writer) error { 1339 e := thrift.NewEncoder(c.header.protocol.NewWriter(w)) 1340 h := bloomFilterHeader(c.columnFilter) 1341 h.NumBytes = int32(len(c.filter)) 1342 if err := e.Encode(&h); err != nil { 1343 return err 1344 } 1345 _, err := w.Write(c.filter) 1346 return err 1347 } 1348 1349 func (c *writerColumn) writeDataPage(page Page) (int64, error) { 1350 numValues := page.NumValues() 1351 if numValues == 0 { 1352 return 0, nil 1353 } 1354 1355 buf := c.buffers 1356 buf.reset() 1357 1358 if c.maxRepetitionLevel > 0 { 1359 buf.encodeRepetitionLevels(page, c.maxRepetitionLevel) 1360 } 1361 if c.maxDefinitionLevel > 0 { 1362 buf.encodeDefinitionLevels(page, c.maxDefinitionLevel) 1363 } 1364 1365 if err := buf.encode(page, c.encoding); err != nil { 1366 return 0, fmt.Errorf("encoding parquet data page: %w", err) 1367 } 1368 if c.dataPageType == format.DataPage { 1369 buf.prependLevelsToDataPageV1(c.maxDefinitionLevel, c.maxDefinitionLevel) 1370 } 1371 1372 uncompressedPageSize := buf.size() 1373 if uncompressedPageSize > maxUncompressedPageSize { 1374 return 0, fmt.Errorf("page size limit exceeded: %d>%d", uncompressedPageSize, maxUncompressedPageSize) 1375 } 1376 if c.isCompressed { 1377 if err := buf.compress(c.compression); err != nil { 1378 return 0, fmt.Errorf("compressing parquet data page: %w", err) 1379 } 1380 } 1381 1382 if page.Dictionary() == nil && len(c.filter) > 0 { 1383 // When the writer knows the number of values in advance (e.g. when 1384 // writing a full row group), the filter encoding is set and the page 1385 // can be directly applied to the filter, which minimizes memory usage 1386 // since there is no need to buffer the values in order to determine 1387 // the size of the filter. 1388 if err := c.writePageToFilter(page); err != nil { 1389 return 0, err 1390 } 1391 } 1392 1393 statistics := format.Statistics{} 1394 if c.writePageStats { 1395 statistics = c.makePageStatistics(page) 1396 } 1397 1398 pageHeader := &format.PageHeader{ 1399 Type: c.dataPageType, 1400 UncompressedPageSize: int32(uncompressedPageSize), 1401 CompressedPageSize: int32(buf.size()), 1402 CRC: int32(buf.crc32()), 1403 } 1404 1405 numRows := page.NumRows() 1406 numNulls := page.NumNulls() 1407 switch c.dataPageType { 1408 case format.DataPage: 1409 pageHeader.DataPageHeader = &format.DataPageHeader{ 1410 NumValues: int32(numValues), 1411 Encoding: c.encoding.Encoding(), 1412 DefinitionLevelEncoding: format.RLE, 1413 RepetitionLevelEncoding: format.RLE, 1414 Statistics: statistics, 1415 } 1416 case format.DataPageV2: 1417 pageHeader.DataPageHeaderV2 = &format.DataPageHeaderV2{ 1418 NumValues: int32(numValues), 1419 NumNulls: int32(numNulls), 1420 NumRows: int32(numRows), 1421 Encoding: c.encoding.Encoding(), 1422 DefinitionLevelsByteLength: int32(len(buf.definitions)), 1423 RepetitionLevelsByteLength: int32(len(buf.repetitions)), 1424 IsCompressed: &c.isCompressed, 1425 Statistics: statistics, 1426 } 1427 } 1428 1429 buf.header.Reset() 1430 if err := c.header.encoder.Encode(pageHeader); err != nil { 1431 return 0, err 1432 } 1433 1434 size := int64(buf.header.Len()) + 1435 int64(len(buf.repetitions)) + 1436 int64(len(buf.definitions)) + 1437 int64(len(buf.page)) 1438 1439 err := c.writePageTo(size, func(output io.Writer) (written int64, err error) { 1440 for _, data := range [...][]byte{ 1441 buf.header.Bytes(), 1442 buf.repetitions, 1443 buf.definitions, 1444 buf.page, 1445 } { 1446 wn, err := output.Write(data) 1447 written += int64(wn) 1448 if err != nil { 1449 return written, err 1450 } 1451 } 1452 return written, nil 1453 }) 1454 if err != nil { 1455 return 0, err 1456 } 1457 1458 c.recordPageStats(int32(buf.header.Len()), pageHeader, page) 1459 return numValues, nil 1460 } 1461 1462 func (c *writerColumn) writeDictionaryPage(output io.Writer, dict Dictionary) (err error) { 1463 buf := c.buffers 1464 buf.reset() 1465 1466 if err := buf.encode(dict.Page(), &Plain); err != nil { 1467 return fmt.Errorf("writing parquet dictionary page: %w", err) 1468 } 1469 1470 uncompressedPageSize := buf.size() 1471 if uncompressedPageSize > maxUncompressedPageSize { 1472 return fmt.Errorf("page size limit exceeded: %d>%d", uncompressedPageSize, maxUncompressedPageSize) 1473 } 1474 if isCompressed(c.compression) { 1475 if err := buf.compress(c.compression); err != nil { 1476 return fmt.Errorf("copmressing parquet dictionary page: %w", err) 1477 } 1478 } 1479 1480 pageHeader := &format.PageHeader{ 1481 Type: format.DictionaryPage, 1482 UncompressedPageSize: int32(uncompressedPageSize), 1483 CompressedPageSize: int32(buf.size()), 1484 CRC: int32(buf.crc32()), 1485 DictionaryPageHeader: &format.DictionaryPageHeader{ 1486 NumValues: int32(dict.Len()), 1487 Encoding: format.Plain, 1488 IsSorted: false, 1489 }, 1490 } 1491 1492 header := &c.buffers.header 1493 header.Reset() 1494 if err := c.header.encoder.Encode(pageHeader); err != nil { 1495 return err 1496 } 1497 if _, err := output.Write(header.Bytes()); err != nil { 1498 return err 1499 } 1500 if _, err := output.Write(buf.page); err != nil { 1501 return err 1502 } 1503 c.recordPageStats(int32(header.Len()), pageHeader, nil) 1504 return nil 1505 } 1506 1507 func (w *writerColumn) writePageToFilter(page Page) (err error) { 1508 pageType := page.Type() 1509 pageData := page.Data() 1510 w.filter, err = pageType.Encode(w.filter, pageData, w.columnFilter.Encoding()) 1511 return err 1512 } 1513 1514 func (c *writerColumn) writePageTo(size int64, writeTo func(io.Writer) (int64, error)) error { 1515 buffer := c.pool.GetBuffer() 1516 defer func() { 1517 if buffer != nil { 1518 c.pool.PutBuffer(buffer) 1519 } 1520 }() 1521 written, err := writeTo(buffer) 1522 if err != nil { 1523 return err 1524 } 1525 if written != size { 1526 return fmt.Errorf("writing parquet column page expected %dB but got %dB: %w", size, written, io.ErrShortWrite) 1527 } 1528 offset, err := buffer.Seek(0, io.SeekStart) 1529 if err != nil { 1530 return err 1531 } 1532 if offset != 0 { 1533 return fmt.Errorf("resetting parquet page buffer to the start expected offset zero but got %d", offset) 1534 } 1535 c.pages, buffer = append(c.pages, buffer), nil 1536 return nil 1537 } 1538 1539 func (c *writerColumn) makePageStatistics(page Page) format.Statistics { 1540 numNulls := page.NumNulls() 1541 minValue, maxValue, _ := page.Bounds() 1542 minValueBytes := minValue.Bytes() 1543 maxValueBytes := maxValue.Bytes() 1544 return format.Statistics{ 1545 Min: minValueBytes, // deprecated 1546 Max: maxValueBytes, // deprecated 1547 NullCount: numNulls, 1548 MinValue: minValueBytes, 1549 MaxValue: maxValueBytes, 1550 } 1551 } 1552 1553 func (c *writerColumn) recordPageStats(headerSize int32, header *format.PageHeader, page Page) { 1554 uncompressedSize := headerSize + header.UncompressedPageSize 1555 compressedSize := headerSize + header.CompressedPageSize 1556 1557 if page != nil { 1558 numNulls := page.NumNulls() 1559 numValues := page.NumValues() 1560 minValue, maxValue, pageHasBounds := page.Bounds() 1561 c.columnIndex.IndexPage(numValues, numNulls, minValue, maxValue) 1562 c.columnChunk.MetaData.NumValues += numValues 1563 c.columnChunk.MetaData.Statistics.NullCount += numNulls 1564 1565 if pageHasBounds { 1566 var existingMaxValue, existingMinValue Value 1567 1568 if c.columnChunk.MetaData.Statistics.MaxValue != nil && c.columnChunk.MetaData.Statistics.MinValue != nil { 1569 existingMaxValue = c.columnType.Kind().Value(c.columnChunk.MetaData.Statistics.MaxValue) 1570 existingMinValue = c.columnType.Kind().Value(c.columnChunk.MetaData.Statistics.MinValue) 1571 } 1572 1573 if existingMaxValue.isNull() || c.columnType.Compare(maxValue, existingMaxValue) > 0 { 1574 buf := c.columnChunk.MetaData.Statistics.MaxValue[:0] 1575 c.columnChunk.MetaData.Statistics.MaxValue = maxValue.AppendBytes(buf) 1576 } 1577 1578 if existingMinValue.isNull() || c.columnType.Compare(minValue, existingMinValue) < 0 { 1579 buf := c.columnChunk.MetaData.Statistics.MinValue[:0] 1580 c.columnChunk.MetaData.Statistics.MinValue = minValue.AppendBytes(buf) 1581 } 1582 } 1583 1584 c.offsetIndex.PageLocations = append(c.offsetIndex.PageLocations, format.PageLocation{ 1585 Offset: c.columnChunk.MetaData.TotalCompressedSize, 1586 CompressedPageSize: compressedSize, 1587 FirstRowIndex: c.numRows, 1588 }) 1589 1590 c.numRows += page.NumRows() 1591 } 1592 1593 pageType := header.Type 1594 encoding := format.Encoding(-1) 1595 switch pageType { 1596 case format.DataPageV2: 1597 encoding = header.DataPageHeaderV2.Encoding 1598 case format.DataPage: 1599 encoding = header.DataPageHeader.Encoding 1600 case format.DictionaryPage: 1601 encoding = header.DictionaryPageHeader.Encoding 1602 } 1603 1604 c.columnChunk.MetaData.TotalUncompressedSize += int64(uncompressedSize) 1605 c.columnChunk.MetaData.TotalCompressedSize += int64(compressedSize) 1606 c.columnChunk.MetaData.EncodingStats = addPageEncodingStats(c.columnChunk.MetaData.EncodingStats, format.PageEncodingStats{ 1607 PageType: pageType, 1608 Encoding: encoding, 1609 Count: 1, 1610 }) 1611 } 1612 1613 func addEncoding(encodings []format.Encoding, add format.Encoding) []format.Encoding { 1614 for _, enc := range encodings { 1615 if enc == add { 1616 return encodings 1617 } 1618 } 1619 return append(encodings, add) 1620 } 1621 1622 func addPageEncodingStats(stats []format.PageEncodingStats, pages ...format.PageEncodingStats) []format.PageEncodingStats { 1623 addPages: 1624 for _, add := range pages { 1625 for i, st := range stats { 1626 if st.PageType == add.PageType && st.Encoding == add.Encoding { 1627 stats[i].Count += add.Count 1628 continue addPages 1629 } 1630 } 1631 stats = append(stats, add) 1632 } 1633 return stats 1634 } 1635 1636 func sortPageEncodings(encodings []format.Encoding) { 1637 sort.Slice(encodings, func(i, j int) bool { 1638 return encodings[i] < encodings[j] 1639 }) 1640 } 1641 1642 func sortPageEncodingStats(stats []format.PageEncodingStats) { 1643 sort.Slice(stats, func(i, j int) bool { 1644 s1 := &stats[i] 1645 s2 := &stats[j] 1646 if s1.PageType != s2.PageType { 1647 return s1.PageType < s2.PageType 1648 } 1649 return s1.Encoding < s2.Encoding 1650 }) 1651 } 1652 1653 type offsetTrackingWriter struct { 1654 writer io.Writer 1655 offset int64 1656 } 1657 1658 func (w *offsetTrackingWriter) Reset(writer io.Writer) { 1659 w.writer = writer 1660 w.offset = 0 1661 } 1662 1663 func (w *offsetTrackingWriter) Write(b []byte) (int, error) { 1664 n, err := w.writer.Write(b) 1665 w.offset += int64(n) 1666 return n, err 1667 } 1668 1669 func (w *offsetTrackingWriter) WriteString(s string) (int, error) { 1670 n, err := io.WriteString(w.writer, s) 1671 w.offset += int64(n) 1672 return n, err 1673 } 1674 1675 func (w *offsetTrackingWriter) ReadFrom(r io.Reader) (int64, error) { 1676 // io.Copy will make use of io.ReaderFrom if w.writer implements it. 1677 n, err := io.Copy(w.writer, r) 1678 w.offset += n 1679 return n, err 1680 } 1681 1682 var ( 1683 _ RowWriterWithSchema = (*Writer)(nil) 1684 _ RowReaderFrom = (*Writer)(nil) 1685 _ RowGroupWriter = (*Writer)(nil) 1686 1687 _ RowWriter = (*writer)(nil) 1688 _ ValueWriter = (*writer)(nil) 1689 1690 _ ValueWriter = (*writerColumn)(nil) 1691 1692 _ io.ReaderFrom = (*offsetTrackingWriter)(nil) 1693 _ io.StringWriter = (*offsetTrackingWriter)(nil) 1694 )