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