github.com/apache/arrow/go/v14@v14.0.1/arrow/ipc/writer.go (about) 1 // Licensed to the Apache Software Foundation (ASF) under one 2 // or more contributor license agreements. See the NOTICE file 3 // distributed with this work for additional information 4 // regarding copyright ownership. The ASF licenses this file 5 // to you under the Apache License, Version 2.0 (the 6 // "License"); you may not use this file except in compliance 7 // with the License. You may obtain a copy of the License at 8 // 9 // http://www.apache.org/licenses/LICENSE-2.0 10 // 11 // Unless required by applicable law or agreed to in writing, software 12 // distributed under the License is distributed on an "AS IS" BASIS, 13 // WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. 14 // See the License for the specific language governing permissions and 15 // limitations under the License. 16 17 package ipc 18 19 import ( 20 "context" 21 "encoding/binary" 22 "errors" 23 "fmt" 24 "io" 25 "math" 26 "sync" 27 "unsafe" 28 29 "github.com/apache/arrow/go/v14/arrow" 30 "github.com/apache/arrow/go/v14/arrow/array" 31 "github.com/apache/arrow/go/v14/arrow/bitutil" 32 "github.com/apache/arrow/go/v14/arrow/internal" 33 "github.com/apache/arrow/go/v14/arrow/internal/debug" 34 "github.com/apache/arrow/go/v14/arrow/internal/dictutils" 35 "github.com/apache/arrow/go/v14/arrow/internal/flatbuf" 36 "github.com/apache/arrow/go/v14/arrow/memory" 37 ) 38 39 type swriter struct { 40 w io.Writer 41 pos int64 42 } 43 44 func (w *swriter) Start() error { return nil } 45 func (w *swriter) Close() error { 46 _, err := w.Write(kEOS[:]) 47 return err 48 } 49 50 func (w *swriter) WritePayload(p Payload) error { 51 _, err := writeIPCPayload(w, p) 52 if err != nil { 53 return err 54 } 55 return nil 56 } 57 58 func (w *swriter) Write(p []byte) (int, error) { 59 n, err := w.w.Write(p) 60 w.pos += int64(n) 61 return n, err 62 } 63 64 func hasNestedDict(data arrow.ArrayData) bool { 65 if data.DataType().ID() == arrow.DICTIONARY { 66 return true 67 } 68 for _, c := range data.Children() { 69 if hasNestedDict(c) { 70 return true 71 } 72 } 73 return false 74 } 75 76 // Writer is an Arrow stream writer. 77 type Writer struct { 78 w io.Writer 79 80 mem memory.Allocator 81 pw PayloadWriter 82 83 started bool 84 schema *arrow.Schema 85 mapper dictutils.Mapper 86 codec flatbuf.CompressionType 87 compressNP int 88 minSpaceSavings *float64 89 90 // map of the last written dictionaries by id 91 // so we can avoid writing the same dictionary over and over 92 lastWrittenDicts map[int64]arrow.Array 93 emitDictDeltas bool 94 } 95 96 // NewWriterWithPayloadWriter constructs a writer with the provided payload writer 97 // instead of the default stream payload writer. This makes the writer more 98 // reusable such as by the Arrow Flight writer. 99 func NewWriterWithPayloadWriter(pw PayloadWriter, opts ...Option) *Writer { 100 cfg := newConfig(opts...) 101 return &Writer{ 102 mem: cfg.alloc, 103 pw: pw, 104 schema: cfg.schema, 105 codec: cfg.codec, 106 compressNP: cfg.compressNP, 107 minSpaceSavings: cfg.minSpaceSavings, 108 emitDictDeltas: cfg.emitDictDeltas, 109 } 110 } 111 112 // NewWriter returns a writer that writes records to the provided output stream. 113 func NewWriter(w io.Writer, opts ...Option) *Writer { 114 cfg := newConfig(opts...) 115 return &Writer{ 116 w: w, 117 mem: cfg.alloc, 118 pw: &swriter{w: w}, 119 schema: cfg.schema, 120 codec: cfg.codec, 121 emitDictDeltas: cfg.emitDictDeltas, 122 } 123 } 124 125 func (w *Writer) Close() error { 126 if !w.started { 127 err := w.start() 128 if err != nil { 129 return err 130 } 131 } 132 133 if w.pw == nil { 134 return nil 135 } 136 137 err := w.pw.Close() 138 if err != nil { 139 return fmt.Errorf("arrow/ipc: could not close payload writer: %w", err) 140 } 141 w.pw = nil 142 143 for _, d := range w.lastWrittenDicts { 144 d.Release() 145 } 146 147 return nil 148 } 149 150 func (w *Writer) Write(rec arrow.Record) (err error) { 151 defer func() { 152 if pErr := recover(); pErr != nil { 153 err = fmt.Errorf("arrow/ipc: unknown error while writing: %v", pErr) 154 } 155 }() 156 157 if !w.started { 158 err := w.start() 159 if err != nil { 160 return err 161 } 162 } 163 164 schema := rec.Schema() 165 if schema == nil || !schema.Equal(w.schema) { 166 return errInconsistentSchema 167 } 168 169 const allow64b = true 170 var ( 171 data = Payload{msg: MessageRecordBatch} 172 enc = newRecordEncoder(w.mem, 0, kMaxNestingDepth, allow64b, w.codec, w.compressNP, w.minSpaceSavings) 173 ) 174 defer data.Release() 175 176 err = writeDictionaryPayloads(w.mem, rec, false, w.emitDictDeltas, &w.mapper, w.lastWrittenDicts, w.pw, enc) 177 if err != nil { 178 return fmt.Errorf("arrow/ipc: failure writing dictionary batches: %w", err) 179 } 180 181 enc.reset() 182 if err := enc.Encode(&data, rec); err != nil { 183 return fmt.Errorf("arrow/ipc: could not encode record to payload: %w", err) 184 } 185 186 return w.pw.WritePayload(data) 187 } 188 189 func writeDictionaryPayloads(mem memory.Allocator, batch arrow.Record, isFileFormat bool, emitDictDeltas bool, mapper *dictutils.Mapper, lastWrittenDicts map[int64]arrow.Array, pw PayloadWriter, encoder *recordEncoder) error { 190 dictionaries, err := dictutils.CollectDictionaries(batch, mapper) 191 if err != nil { 192 return err 193 } 194 defer func() { 195 for _, d := range dictionaries { 196 d.Dict.Release() 197 } 198 }() 199 200 eqopt := array.WithNaNsEqual(true) 201 for _, pair := range dictionaries { 202 encoder.reset() 203 var ( 204 deltaStart int64 205 enc = dictEncoder{encoder} 206 ) 207 lastDict, exists := lastWrittenDicts[pair.ID] 208 if exists { 209 if lastDict.Data() == pair.Dict.Data() { 210 continue 211 } 212 newLen, lastLen := pair.Dict.Len(), lastDict.Len() 213 if lastLen == newLen && array.ApproxEqual(lastDict, pair.Dict, eqopt) { 214 // same dictionary by value 215 // might cost CPU, but required for IPC file format 216 continue 217 } 218 if isFileFormat { 219 return errors.New("arrow/ipc: Dictionary replacement detected when writing IPC file format. Arrow IPC File only supports single dictionary per field") 220 } 221 222 if newLen > lastLen && 223 emitDictDeltas && 224 !hasNestedDict(pair.Dict.Data()) && 225 (array.SliceApproxEqual(lastDict, 0, int64(lastLen), pair.Dict, 0, int64(lastLen), eqopt)) { 226 deltaStart = int64(lastLen) 227 } 228 } 229 230 var data = Payload{msg: MessageDictionaryBatch} 231 defer data.Release() 232 233 dict := pair.Dict 234 if deltaStart > 0 { 235 dict = array.NewSlice(dict, deltaStart, int64(dict.Len())) 236 defer dict.Release() 237 } 238 if err := enc.Encode(&data, pair.ID, deltaStart > 0, dict); err != nil { 239 return err 240 } 241 242 if err := pw.WritePayload(data); err != nil { 243 return err 244 } 245 246 lastWrittenDicts[pair.ID] = pair.Dict 247 if lastDict != nil { 248 lastDict.Release() 249 } 250 pair.Dict.Retain() 251 } 252 return nil 253 } 254 255 func (w *Writer) start() error { 256 w.started = true 257 258 w.mapper.ImportSchema(w.schema) 259 w.lastWrittenDicts = make(map[int64]arrow.Array) 260 261 // write out schema payloads 262 ps := payloadFromSchema(w.schema, w.mem, &w.mapper) 263 defer ps.Release() 264 265 for _, data := range ps { 266 err := w.pw.WritePayload(data) 267 if err != nil { 268 return err 269 } 270 } 271 272 return nil 273 } 274 275 type dictEncoder struct { 276 *recordEncoder 277 } 278 279 func (d *dictEncoder) encodeMetadata(p *Payload, isDelta bool, id, nrows int64) error { 280 p.meta = writeDictionaryMessage(d.mem, id, isDelta, nrows, p.size, d.fields, d.meta, d.codec) 281 return nil 282 } 283 284 func (d *dictEncoder) Encode(p *Payload, id int64, isDelta bool, dict arrow.Array) error { 285 d.start = 0 286 defer func() { 287 d.start = 0 288 }() 289 290 schema := arrow.NewSchema([]arrow.Field{{Name: "dictionary", Type: dict.DataType(), Nullable: true}}, nil) 291 batch := array.NewRecord(schema, []arrow.Array{dict}, int64(dict.Len())) 292 defer batch.Release() 293 if err := d.encode(p, batch); err != nil { 294 return err 295 } 296 297 return d.encodeMetadata(p, isDelta, id, batch.NumRows()) 298 } 299 300 type recordEncoder struct { 301 mem memory.Allocator 302 303 fields []fieldMetadata 304 meta []bufferMetadata 305 306 depth int64 307 start int64 308 allow64b bool 309 codec flatbuf.CompressionType 310 compressNP int 311 minSpaceSavings *float64 312 } 313 314 func newRecordEncoder(mem memory.Allocator, startOffset, maxDepth int64, allow64b bool, codec flatbuf.CompressionType, compressNP int, minSpaceSavings *float64) *recordEncoder { 315 return &recordEncoder{ 316 mem: mem, 317 start: startOffset, 318 depth: maxDepth, 319 allow64b: allow64b, 320 codec: codec, 321 compressNP: compressNP, 322 minSpaceSavings: minSpaceSavings, 323 } 324 } 325 326 func (w *recordEncoder) shouldCompress(uncompressed, compressed int) bool { 327 debug.Assert(uncompressed > 0, "uncompressed size is 0") 328 if w.minSpaceSavings == nil { 329 return true 330 } 331 332 savings := 1.0 - float64(compressed)/float64(uncompressed) 333 return savings >= *w.minSpaceSavings 334 } 335 336 func (w *recordEncoder) reset() { 337 w.start = 0 338 w.fields = make([]fieldMetadata, 0) 339 } 340 341 func (w *recordEncoder) compressBodyBuffers(p *Payload) error { 342 compress := func(idx int, codec compressor) error { 343 if p.body[idx] == nil || p.body[idx].Len() == 0 { 344 return nil 345 } 346 347 buf := memory.NewResizableBuffer(w.mem) 348 buf.Reserve(codec.MaxCompressedLen(p.body[idx].Len()) + arrow.Int64SizeBytes) 349 350 binary.LittleEndian.PutUint64(buf.Buf(), uint64(p.body[idx].Len())) 351 bw := &bufferWriter{buf: buf, pos: arrow.Int64SizeBytes} 352 codec.Reset(bw) 353 354 n, err := codec.Write(p.body[idx].Bytes()) 355 if err != nil { 356 return err 357 } 358 if err := codec.Close(); err != nil { 359 return err 360 } 361 362 finalLen := bw.pos 363 compressedLen := bw.pos - arrow.Int64SizeBytes 364 if !w.shouldCompress(n, compressedLen) { 365 n = copy(buf.Buf()[arrow.Int64SizeBytes:], p.body[idx].Bytes()) 366 // size of -1 indicates to the reader that the body 367 // doesn't need to be decompressed 368 var noprefix int64 = -1 369 binary.LittleEndian.PutUint64(buf.Buf(), uint64(noprefix)) 370 finalLen = n + arrow.Int64SizeBytes 371 } 372 bw.buf.Resize(finalLen) 373 p.body[idx].Release() 374 p.body[idx] = buf 375 return nil 376 } 377 378 if w.compressNP <= 1 { 379 codec := getCompressor(w.codec) 380 for idx := range p.body { 381 if err := compress(idx, codec); err != nil { 382 return err 383 } 384 } 385 return nil 386 } 387 388 var ( 389 wg sync.WaitGroup 390 ch = make(chan int) 391 errch = make(chan error) 392 ctx, cancel = context.WithCancel(context.Background()) 393 ) 394 defer cancel() 395 396 for i := 0; i < w.compressNP; i++ { 397 wg.Add(1) 398 go func() { 399 defer wg.Done() 400 codec := getCompressor(w.codec) 401 for { 402 select { 403 case idx, ok := <-ch: 404 if !ok { 405 // we're done, channel is closed! 406 return 407 } 408 409 if err := compress(idx, codec); err != nil { 410 errch <- err 411 cancel() 412 return 413 } 414 case <-ctx.Done(): 415 // cancelled, return early 416 return 417 } 418 } 419 }() 420 } 421 422 for idx := range p.body { 423 ch <- idx 424 } 425 426 close(ch) 427 wg.Wait() 428 close(errch) 429 430 return <-errch 431 } 432 433 func (w *recordEncoder) encode(p *Payload, rec arrow.Record) error { 434 // perform depth-first traversal of the row-batch 435 for i, col := range rec.Columns() { 436 err := w.visit(p, col) 437 if err != nil { 438 return fmt.Errorf("arrow/ipc: could not encode column %d (%q): %w", i, rec.ColumnName(i), err) 439 } 440 } 441 442 if w.codec != -1 { 443 if w.minSpaceSavings != nil { 444 pct := *w.minSpaceSavings 445 if pct < 0 || pct > 1 { 446 p.Release() 447 return fmt.Errorf("%w: minSpaceSavings not in range [0,1]. Provided %.05f", 448 arrow.ErrInvalid, pct) 449 } 450 } 451 w.compressBodyBuffers(p) 452 } 453 454 // position for the start of a buffer relative to the passed frame of reference. 455 // may be 0 or some other position in an address space. 456 offset := w.start 457 w.meta = make([]bufferMetadata, len(p.body)) 458 459 // construct the metadata for the record batch header 460 for i, buf := range p.body { 461 var ( 462 size int64 463 padding int64 464 ) 465 // the buffer might be null if we are handling zero row lengths. 466 if buf != nil { 467 size = int64(buf.Len()) 468 padding = bitutil.CeilByte64(size) - size 469 } 470 w.meta[i] = bufferMetadata{ 471 Offset: offset, 472 // even though we add padding, we need the Len to be correct 473 // so that decompressing works properly. 474 Len: size, 475 } 476 offset += size + padding 477 } 478 479 p.size = offset - w.start 480 if !bitutil.IsMultipleOf8(p.size) { 481 panic("not aligned") 482 } 483 484 return nil 485 } 486 487 func (w *recordEncoder) visit(p *Payload, arr arrow.Array) error { 488 if w.depth <= 0 { 489 return errMaxRecursion 490 } 491 492 if !w.allow64b && arr.Len() > math.MaxInt32 { 493 return errBigArray 494 } 495 496 if arr.DataType().ID() == arrow.EXTENSION { 497 arr := arr.(array.ExtensionArray) 498 err := w.visit(p, arr.Storage()) 499 if err != nil { 500 return fmt.Errorf("failed visiting storage of for array %T: %w", arr, err) 501 } 502 return nil 503 } 504 505 if arr.DataType().ID() == arrow.DICTIONARY { 506 arr := arr.(*array.Dictionary) 507 return w.visit(p, arr.Indices()) 508 } 509 510 // add all common elements 511 w.fields = append(w.fields, fieldMetadata{ 512 Len: int64(arr.Len()), 513 Nulls: int64(arr.NullN()), 514 Offset: 0, 515 }) 516 517 if arr.DataType().ID() == arrow.NULL { 518 return nil 519 } 520 521 if internal.HasValidityBitmap(arr.DataType().ID(), flatbuf.MetadataVersion(currentMetadataVersion)) { 522 switch arr.NullN() { 523 case 0: 524 // there are no null values, drop the null bitmap 525 p.body = append(p.body, nil) 526 default: 527 data := arr.Data() 528 var bitmap *memory.Buffer 529 if data.NullN() == data.Len() { 530 // every value is null, just use a new zero-initialized bitmap to avoid the expense of copying 531 bitmap = memory.NewResizableBuffer(w.mem) 532 minLength := paddedLength(bitutil.BytesForBits(int64(data.Len())), kArrowAlignment) 533 bitmap.Resize(int(minLength)) 534 } else { 535 // otherwise truncate and copy the bits 536 bitmap = newTruncatedBitmap(w.mem, int64(data.Offset()), int64(data.Len()), data.Buffers()[0]) 537 } 538 p.body = append(p.body, bitmap) 539 } 540 } 541 542 switch dtype := arr.DataType().(type) { 543 case *arrow.NullType: 544 // ok. NullArrays are completely empty. 545 546 case *arrow.BooleanType: 547 var ( 548 data = arr.Data() 549 bitm *memory.Buffer 550 ) 551 552 if data.Len() != 0 { 553 bitm = newTruncatedBitmap(w.mem, int64(data.Offset()), int64(data.Len()), data.Buffers()[1]) 554 } 555 p.body = append(p.body, bitm) 556 557 case arrow.FixedWidthDataType: 558 data := arr.Data() 559 values := data.Buffers()[1] 560 arrLen := int64(arr.Len()) 561 typeWidth := int64(dtype.BitWidth() / 8) 562 minLength := paddedLength(arrLen*typeWidth, kArrowAlignment) 563 564 switch { 565 case needTruncate(int64(data.Offset()), values, minLength): 566 // non-zero offset: slice the buffer 567 offset := int64(data.Offset()) * typeWidth 568 // send padding if available 569 len := minI64(bitutil.CeilByte64(arrLen*typeWidth), int64(values.Len())-offset) 570 values = memory.NewBufferBytes(values.Bytes()[offset : offset+len]) 571 default: 572 if values != nil { 573 values.Retain() 574 } 575 } 576 p.body = append(p.body, values) 577 578 case *arrow.BinaryType, *arrow.LargeBinaryType, *arrow.StringType, *arrow.LargeStringType: 579 arr := arr.(array.BinaryLike) 580 voffsets := w.getZeroBasedValueOffsets(arr) 581 data := arr.Data() 582 values := data.Buffers()[2] 583 584 var totalDataBytes int64 585 if voffsets != nil { 586 totalDataBytes = int64(len(arr.ValueBytes())) 587 } 588 589 switch { 590 case needTruncate(int64(data.Offset()), values, totalDataBytes): 591 // slice data buffer to include the range we need now. 592 var ( 593 beg = arr.ValueOffset64(0) 594 len = minI64(paddedLength(totalDataBytes, kArrowAlignment), int64(totalDataBytes)) 595 ) 596 values = memory.NewBufferBytes(data.Buffers()[2].Bytes()[beg : beg+len]) 597 default: 598 if values != nil { 599 values.Retain() 600 } 601 } 602 p.body = append(p.body, voffsets) 603 p.body = append(p.body, values) 604 605 case *arrow.StructType: 606 w.depth-- 607 arr := arr.(*array.Struct) 608 for i := 0; i < arr.NumField(); i++ { 609 err := w.visit(p, arr.Field(i)) 610 if err != nil { 611 return fmt.Errorf("could not visit field %d of struct-array: %w", i, err) 612 } 613 } 614 w.depth++ 615 616 case *arrow.SparseUnionType: 617 offset, length := arr.Data().Offset(), arr.Len() 618 arr := arr.(*array.SparseUnion) 619 typeCodes := getTruncatedBuffer(int64(offset), int64(length), int32(unsafe.Sizeof(arrow.UnionTypeCode(0))), arr.TypeCodes()) 620 p.body = append(p.body, typeCodes) 621 622 w.depth-- 623 for i := 0; i < arr.NumFields(); i++ { 624 err := w.visit(p, arr.Field(i)) 625 if err != nil { 626 return fmt.Errorf("could not visit field %d of sparse union array: %w", i, err) 627 } 628 } 629 w.depth++ 630 case *arrow.DenseUnionType: 631 offset, length := arr.Data().Offset(), arr.Len() 632 arr := arr.(*array.DenseUnion) 633 typeCodes := getTruncatedBuffer(int64(offset), int64(length), int32(unsafe.Sizeof(arrow.UnionTypeCode(0))), arr.TypeCodes()) 634 p.body = append(p.body, typeCodes) 635 636 w.depth-- 637 dt := arr.UnionType() 638 639 // union type codes are not necessarily 0-indexed 640 maxCode := dt.MaxTypeCode() 641 642 // allocate an array of child offsets. Set all to -1 to indicate we 643 // haven't observed a first occurrence of a particular child yet 644 offsets := make([]int32, maxCode+1) 645 lengths := make([]int32, maxCode+1) 646 offsets[0], lengths[0] = -1, 0 647 for i := 1; i < len(offsets); i *= 2 { 648 copy(offsets[i:], offsets[:i]) 649 copy(lengths[i:], lengths[:i]) 650 } 651 652 var valueOffsets *memory.Buffer 653 if offset != 0 { 654 valueOffsets = w.rebaseDenseUnionValueOffsets(arr, offsets, lengths) 655 } else { 656 valueOffsets = getTruncatedBuffer(int64(offset), int64(length), int32(arrow.Int32SizeBytes), arr.ValueOffsets()) 657 } 658 p.body = append(p.body, valueOffsets) 659 660 // visit children and slice accordingly 661 for i := range dt.Fields() { 662 child := arr.Field(i) 663 // for sliced unions it's tricky to know how much to truncate 664 // the children. For now we'll truncate the children to be 665 // no longer than the parent union. 666 667 if offset != 0 { 668 code := dt.TypeCodes()[i] 669 childOffset := offsets[code] 670 childLen := lengths[code] 671 672 if childOffset > 0 { 673 child = array.NewSlice(child, int64(childOffset), int64(childOffset+childLen)) 674 defer child.Release() 675 } else if childLen < int32(child.Len()) { 676 child = array.NewSlice(child, 0, int64(childLen)) 677 defer child.Release() 678 } 679 } 680 if err := w.visit(p, child); err != nil { 681 return fmt.Errorf("could not visit field %d of dense union array: %w", i, err) 682 } 683 } 684 w.depth++ 685 case *arrow.MapType, *arrow.ListType, *arrow.LargeListType: 686 arr := arr.(array.ListLike) 687 voffsets := w.getZeroBasedValueOffsets(arr) 688 p.body = append(p.body, voffsets) 689 690 w.depth-- 691 var ( 692 values = arr.ListValues() 693 mustRelease = false 694 values_offset int64 695 values_end int64 696 ) 697 defer func() { 698 if mustRelease { 699 values.Release() 700 } 701 }() 702 703 if arr.Len() > 0 && voffsets != nil { 704 values_offset, _ = arr.ValueOffsets(0) 705 _, values_end = arr.ValueOffsets(arr.Len() - 1) 706 } 707 708 if arr.Len() != 0 || values_end < int64(values.Len()) { 709 // must also slice the values 710 values = array.NewSlice(values, values_offset, values_end) 711 mustRelease = true 712 } 713 err := w.visit(p, values) 714 715 if err != nil { 716 return fmt.Errorf("could not visit list element for array %T: %w", arr, err) 717 } 718 w.depth++ 719 720 case *arrow.ListViewType, *arrow.LargeListViewType: 721 data := arr.Data() 722 arr := arr.(array.VarLenListLike) 723 offsetTraits := arr.DataType().(arrow.OffsetsDataType).OffsetTypeTraits() 724 rngOff, rngLen := array.RangeOfValuesUsed(arr) 725 voffsets := w.getValueOffsetsAtBaseValue(arr, rngOff) 726 p.body = append(p.body, voffsets) 727 728 vsizes := data.Buffers()[2] 729 if vsizes != nil { 730 if data.Offset() != 0 || vsizes.Len() > offsetTraits.BytesRequired(arr.Len()) { 731 beg := offsetTraits.BytesRequired(data.Offset()) 732 end := beg + offsetTraits.BytesRequired(data.Len()) 733 vsizes = memory.NewBufferBytes(vsizes.Bytes()[beg:end]) 734 } else { 735 vsizes.Retain() 736 } 737 } 738 p.body = append(p.body, vsizes) 739 740 w.depth-- 741 var ( 742 values = arr.ListValues() 743 mustRelease = false 744 values_offset = int64(rngOff) 745 values_end = int64(rngOff + rngLen) 746 ) 747 defer func() { 748 if mustRelease { 749 values.Release() 750 } 751 }() 752 753 if arr.Len() > 0 && values_end < int64(values.Len()) { 754 // must also slice the values 755 values = array.NewSlice(values, values_offset, values_end) 756 mustRelease = true 757 } 758 err := w.visit(p, values) 759 760 if err != nil { 761 return fmt.Errorf("could not visit list element for array %T: %w", arr, err) 762 } 763 w.depth++ 764 765 case *arrow.FixedSizeListType: 766 arr := arr.(*array.FixedSizeList) 767 768 w.depth-- 769 770 size := int64(arr.DataType().(*arrow.FixedSizeListType).Len()) 771 beg := int64(arr.Offset()) * size 772 end := int64(arr.Offset()+arr.Len()) * size 773 774 values := array.NewSlice(arr.ListValues(), beg, end) 775 defer values.Release() 776 777 err := w.visit(p, values) 778 779 if err != nil { 780 return fmt.Errorf("could not visit list element for array %T: %w", arr, err) 781 } 782 w.depth++ 783 784 case *arrow.RunEndEncodedType: 785 arr := arr.(*array.RunEndEncoded) 786 w.depth-- 787 child := arr.LogicalRunEndsArray(w.mem) 788 defer child.Release() 789 if err := w.visit(p, child); err != nil { 790 return err 791 } 792 child = arr.LogicalValuesArray() 793 defer child.Release() 794 if err := w.visit(p, child); err != nil { 795 return err 796 } 797 w.depth++ 798 799 default: 800 panic(fmt.Errorf("arrow/ipc: unknown array %T (dtype=%T)", arr, dtype)) 801 } 802 803 return nil 804 } 805 806 func (w *recordEncoder) getZeroBasedValueOffsets(arr arrow.Array) *memory.Buffer { 807 data := arr.Data() 808 voffsets := data.Buffers()[1] 809 offsetTraits := arr.DataType().(arrow.OffsetsDataType).OffsetTypeTraits() 810 offsetBytesNeeded := offsetTraits.BytesRequired(data.Len() + 1) 811 812 if voffsets == nil || voffsets.Len() == 0 { 813 return nil 814 } 815 816 // if we have a non-zero offset, then the value offsets do not start at 817 // zero. we must a) create a new offsets array with shifted offsets and 818 // b) slice the values array accordingly 819 // 820 // or if there are more value offsets than values (the array has been sliced) 821 // we need to trim off the trailing offsets 822 needsTruncateAndShift := data.Offset() != 0 || offsetBytesNeeded < voffsets.Len() 823 824 if needsTruncateAndShift { 825 shiftedOffsets := memory.NewResizableBuffer(w.mem) 826 shiftedOffsets.Resize(offsetBytesNeeded) 827 828 switch arr.DataType().Layout().Buffers[1].ByteWidth { 829 case 8: 830 dest := arrow.Int64Traits.CastFromBytes(shiftedOffsets.Bytes()) 831 offsets := arrow.Int64Traits.CastFromBytes(voffsets.Bytes())[data.Offset() : data.Offset()+data.Len()+1] 832 833 startOffset := offsets[0] 834 for i, o := range offsets { 835 dest[i] = o - startOffset 836 } 837 838 default: 839 debug.Assert(arr.DataType().Layout().Buffers[1].ByteWidth == 4, "invalid offset bytewidth") 840 dest := arrow.Int32Traits.CastFromBytes(shiftedOffsets.Bytes()) 841 offsets := arrow.Int32Traits.CastFromBytes(voffsets.Bytes())[data.Offset() : data.Offset()+data.Len()+1] 842 843 startOffset := offsets[0] 844 for i, o := range offsets { 845 dest[i] = o - startOffset 846 } 847 } 848 849 voffsets = shiftedOffsets 850 } else { 851 voffsets.Retain() 852 } 853 854 return voffsets 855 } 856 857 // Truncates the offsets if needed and shifts the values if minOffset > 0. 858 // The offsets returned are corrected assuming the child values are truncated 859 // and now start at minOffset. 860 // 861 // This function only works on offset buffers of ListViews and LargeListViews. 862 // TODO(felipecrv): Unify this with getZeroBasedValueOffsets. 863 func (w *recordEncoder) getValueOffsetsAtBaseValue(arr arrow.Array, minOffset int) *memory.Buffer { 864 data := arr.Data() 865 voffsets := data.Buffers()[1] 866 offsetTraits := arr.DataType().(arrow.OffsetsDataType).OffsetTypeTraits() 867 offsetBytesNeeded := offsetTraits.BytesRequired(data.Len()) 868 869 if voffsets == nil || voffsets.Len() == 0 { 870 return nil 871 } 872 873 needsTruncate := data.Offset() != 0 || offsetBytesNeeded < voffsets.Len() 874 needsShift := minOffset > 0 875 876 if needsTruncate || needsShift { 877 shiftedOffsets := memory.NewResizableBuffer(w.mem) 878 shiftedOffsets.Resize(offsetBytesNeeded) 879 880 switch arr.DataType().Layout().Buffers[1].ByteWidth { 881 case 8: 882 dest := arrow.Int64Traits.CastFromBytes(shiftedOffsets.Bytes()) 883 offsets := arrow.Int64Traits.CastFromBytes(voffsets.Bytes())[data.Offset() : data.Offset()+data.Len()] 884 885 if minOffset > 0 { 886 for i, o := range offsets { 887 dest[i] = o - int64(minOffset) 888 } 889 } else { 890 copy(dest, offsets) 891 } 892 default: 893 debug.Assert(arr.DataType().Layout().Buffers[1].ByteWidth == 4, "invalid offset bytewidth") 894 dest := arrow.Int32Traits.CastFromBytes(shiftedOffsets.Bytes()) 895 offsets := arrow.Int32Traits.CastFromBytes(voffsets.Bytes())[data.Offset() : data.Offset()+data.Len()] 896 897 if minOffset > 0 { 898 for i, o := range offsets { 899 dest[i] = o - int32(minOffset) 900 } 901 } else { 902 copy(dest, offsets) 903 } 904 } 905 906 voffsets = shiftedOffsets 907 } else { 908 voffsets.Retain() 909 } 910 911 return voffsets 912 } 913 914 func (w *recordEncoder) rebaseDenseUnionValueOffsets(arr *array.DenseUnion, offsets, lengths []int32) *memory.Buffer { 915 // this case sucks. Because the offsets are different for each 916 // child array, when we have a sliced array, we need to re-base 917 // the value offsets for each array! ew. 918 unshiftedOffsets := arr.RawValueOffsets() 919 codes := arr.RawTypeCodes() 920 921 shiftedOffsetsBuf := memory.NewResizableBuffer(w.mem) 922 shiftedOffsetsBuf.Resize(arrow.Int32Traits.BytesRequired(arr.Len())) 923 shiftedOffsets := arrow.Int32Traits.CastFromBytes(shiftedOffsetsBuf.Bytes()) 924 925 // compute shifted offsets by subtracting child offset 926 for i, c := range codes { 927 if offsets[c] == -1 { 928 // offsets are guaranteed to be increasing according to the spec 929 // so the first offset we find for a child is the initial offset 930 // and will become the "0" for this child. 931 offsets[c] = unshiftedOffsets[i] 932 shiftedOffsets[i] = 0 933 } else { 934 shiftedOffsets[i] = unshiftedOffsets[i] - offsets[c] 935 } 936 lengths[c] = maxI32(lengths[c], shiftedOffsets[i]+1) 937 } 938 return shiftedOffsetsBuf 939 } 940 941 func (w *recordEncoder) Encode(p *Payload, rec arrow.Record) error { 942 if err := w.encode(p, rec); err != nil { 943 return err 944 } 945 return w.encodeMetadata(p, rec.NumRows()) 946 } 947 948 func (w *recordEncoder) encodeMetadata(p *Payload, nrows int64) error { 949 p.meta = writeRecordMessage(w.mem, nrows, p.size, w.fields, w.meta, w.codec) 950 return nil 951 } 952 953 func newTruncatedBitmap(mem memory.Allocator, offset, length int64, input *memory.Buffer) *memory.Buffer { 954 if input == nil { 955 return nil 956 } 957 958 minLength := paddedLength(bitutil.BytesForBits(length), kArrowAlignment) 959 switch { 960 case offset != 0 || minLength < int64(input.Len()): 961 // with a sliced array / non-zero offset, we must copy the bitmap 962 buf := memory.NewResizableBuffer(mem) 963 buf.Resize(int(minLength)) 964 bitutil.CopyBitmap(input.Bytes(), int(offset), int(length), buf.Bytes(), 0) 965 return buf 966 default: 967 input.Retain() 968 return input 969 } 970 } 971 972 func getTruncatedBuffer(offset, length int64, byteWidth int32, buf *memory.Buffer) *memory.Buffer { 973 if buf == nil { 974 return buf 975 } 976 977 paddedLen := paddedLength(length*int64(byteWidth), kArrowAlignment) 978 if offset != 0 || paddedLen < int64(buf.Len()) { 979 return memory.SliceBuffer(buf, int(offset*int64(byteWidth)), int(minI64(paddedLen, int64(buf.Len())))) 980 } 981 buf.Retain() 982 return buf 983 } 984 985 func needTruncate(offset int64, buf *memory.Buffer, minLength int64) bool { 986 if buf == nil { 987 return false 988 } 989 return offset != 0 || minLength < int64(buf.Len()) 990 } 991 992 func minI64(a, b int64) int64 { 993 if a < b { 994 return a 995 } 996 return b 997 } 998 999 func maxI32(a, b int32) int32 { 1000 if a > b { 1001 return a 1002 } 1003 return b 1004 }