github.com/apache/arrow/go/v7@v7.0.1/parquet/pqarrow/encode_arrow_test.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 pqarrow_test 18 19 import ( 20 "bytes" 21 "context" 22 "errors" 23 "fmt" 24 "math" 25 "strconv" 26 "strings" 27 "testing" 28 29 "github.com/apache/arrow/go/v7/arrow" 30 "github.com/apache/arrow/go/v7/arrow/array" 31 "github.com/apache/arrow/go/v7/arrow/bitutil" 32 "github.com/apache/arrow/go/v7/arrow/decimal128" 33 "github.com/apache/arrow/go/v7/arrow/memory" 34 "github.com/apache/arrow/go/v7/parquet" 35 "github.com/apache/arrow/go/v7/parquet/compress" 36 "github.com/apache/arrow/go/v7/parquet/file" 37 "github.com/apache/arrow/go/v7/parquet/internal/encoding" 38 "github.com/apache/arrow/go/v7/parquet/internal/testutils" 39 "github.com/apache/arrow/go/v7/parquet/internal/utils" 40 "github.com/apache/arrow/go/v7/parquet/pqarrow" 41 "github.com/apache/arrow/go/v7/parquet/schema" 42 "github.com/stretchr/testify/assert" 43 "github.com/stretchr/testify/require" 44 "github.com/stretchr/testify/suite" 45 ) 46 47 func makeSimpleTable(values *arrow.Chunked, nullable bool) arrow.Table { 48 sc := arrow.NewSchema([]arrow.Field{{Name: "col", Type: values.DataType(), Nullable: nullable}}, nil) 49 column := arrow.NewColumn(sc.Field(0), values) 50 defer column.Release() 51 return array.NewTable(sc, []arrow.Column{*column}, -1) 52 } 53 54 func makeDateTimeTypesTable(mem memory.Allocator, expected bool, addFieldMeta bool) arrow.Table { 55 isValid := []bool{true, true, true, false, true, true} 56 57 // roundtrip without modification 58 f0 := arrow.Field{Name: "f0", Type: arrow.FixedWidthTypes.Date32, Nullable: true} 59 f1 := arrow.Field{Name: "f1", Type: arrow.FixedWidthTypes.Timestamp_ms, Nullable: true} 60 f2 := arrow.Field{Name: "f2", Type: arrow.FixedWidthTypes.Timestamp_us, Nullable: true} 61 f3 := arrow.Field{Name: "f3", Type: arrow.FixedWidthTypes.Timestamp_ns, Nullable: true} 62 f3X := arrow.Field{Name: "f3", Type: arrow.FixedWidthTypes.Timestamp_us, Nullable: true} 63 f4 := arrow.Field{Name: "f4", Type: arrow.FixedWidthTypes.Time32ms, Nullable: true} 64 f5 := arrow.Field{Name: "f5", Type: arrow.FixedWidthTypes.Time64us, Nullable: true} 65 f6 := arrow.Field{Name: "f6", Type: arrow.FixedWidthTypes.Time64ns, Nullable: true} 66 67 fieldList := []arrow.Field{f0, f1, f2} 68 if expected { 69 fieldList = append(fieldList, f3X) 70 } else { 71 fieldList = append(fieldList, f3) 72 } 73 fieldList = append(fieldList, f4, f5, f6) 74 75 if addFieldMeta { 76 for idx := range fieldList { 77 fieldList[idx].Metadata = arrow.NewMetadata([]string{"PARQUET:field_id"}, []string{strconv.Itoa(idx + 1)}) 78 } 79 } 80 arrsc := arrow.NewSchema(fieldList, nil) 81 82 d32Values := []arrow.Date32{1489269000, 1489270000, 1489271000, 1489272000, 1489272000, 1489273000} 83 ts64nsValues := []arrow.Timestamp{1489269000000, 1489270000000, 1489271000000, 1489272000000, 1489272000000, 1489273000000} 84 ts64usValues := []arrow.Timestamp{1489269000, 1489270000, 1489271000, 1489272000, 1489272000, 1489273000} 85 ts64msValues := []arrow.Timestamp{1489269, 1489270, 1489271, 1489272, 1489272, 1489273} 86 t32Values := []arrow.Time32{1489269000, 1489270000, 1489271000, 1489272000, 1489272000, 1489273000} 87 t64nsValues := []arrow.Time64{1489269000000, 1489270000000, 1489271000000, 1489272000000, 1489272000000, 1489273000000} 88 t64usValues := []arrow.Time64{1489269000, 1489270000, 1489271000, 1489272000, 1489272000, 1489273000} 89 90 builders := make([]array.Builder, 0, len(fieldList)) 91 for _, f := range fieldList { 92 bldr := array.NewBuilder(mem, f.Type) 93 defer bldr.Release() 94 builders = append(builders, bldr) 95 } 96 97 builders[0].(*array.Date32Builder).AppendValues(d32Values, isValid) 98 builders[1].(*array.TimestampBuilder).AppendValues(ts64msValues, isValid) 99 builders[2].(*array.TimestampBuilder).AppendValues(ts64usValues, isValid) 100 if expected { 101 builders[3].(*array.TimestampBuilder).AppendValues(ts64usValues, isValid) 102 } else { 103 builders[3].(*array.TimestampBuilder).AppendValues(ts64nsValues, isValid) 104 } 105 builders[4].(*array.Time32Builder).AppendValues(t32Values, isValid) 106 builders[5].(*array.Time64Builder).AppendValues(t64usValues, isValid) 107 builders[6].(*array.Time64Builder).AppendValues(t64nsValues, isValid) 108 109 cols := make([]arrow.Column, 0, len(fieldList)) 110 for idx, field := range fieldList { 111 arr := builders[idx].NewArray() 112 defer arr.Release() 113 114 chunked := arrow.NewChunked(field.Type, []arrow.Array{arr}) 115 defer chunked.Release() 116 col := arrow.NewColumn(field, chunked) 117 defer col.Release() 118 cols = append(cols, *col) 119 } 120 121 return array.NewTable(arrsc, cols, int64(len(isValid))) 122 } 123 124 func TestWriteArrowCols(t *testing.T) { 125 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 126 defer mem.AssertSize(t, 0) 127 128 tbl := makeDateTimeTypesTable(mem, false, false) 129 defer tbl.Release() 130 131 psc, err := pqarrow.ToParquet(tbl.Schema(), nil, pqarrow.NewArrowWriterProperties(pqarrow.WithAllocator(mem))) 132 require.NoError(t, err) 133 134 manifest, err := pqarrow.NewSchemaManifest(psc, nil, nil) 135 require.NoError(t, err) 136 137 sink := encoding.NewBufferWriter(0, memory.DefaultAllocator) 138 writer := file.NewParquetWriter(sink, psc.Root(), file.WithWriterProps(parquet.NewWriterProperties(parquet.WithVersion(parquet.V2_4)))) 139 140 srgw := writer.AppendRowGroup() 141 ctx := pqarrow.NewArrowWriteContext(context.TODO(), nil) 142 143 for i := int64(0); i < tbl.NumCols(); i++ { 144 acw, err := pqarrow.NewArrowColumnWriter(tbl.Column(int(i)).Data(), 0, tbl.NumRows(), manifest, srgw, int(i)) 145 require.NoError(t, err) 146 require.NoError(t, acw.Write(ctx)) 147 } 148 require.NoError(t, srgw.Close()) 149 require.NoError(t, writer.Close()) 150 151 expected := makeDateTimeTypesTable(mem, true, false) 152 defer expected.Release() 153 154 reader, err := file.NewParquetReader(bytes.NewReader(sink.Bytes())) 155 require.NoError(t, err) 156 157 assert.EqualValues(t, expected.NumCols(), reader.MetaData().Schema.NumColumns()) 158 assert.EqualValues(t, expected.NumRows(), reader.NumRows()) 159 assert.EqualValues(t, 1, reader.NumRowGroups()) 160 161 rgr := reader.RowGroup(0) 162 163 for i := 0; i < int(expected.NumCols()); i++ { 164 var ( 165 total int64 166 read int 167 err error 168 defLevelsOut = make([]int16, int(expected.NumRows())) 169 arr = expected.Column(i).Data().Chunk(0) 170 ) 171 switch expected.Schema().Field(i).Type.(arrow.FixedWidthDataType).BitWidth() { 172 case 32: 173 colReader := rgr.Column(i).(*file.Int32ColumnChunkReader) 174 vals := make([]int32, int(expected.NumRows())) 175 total, read, err = colReader.ReadBatch(expected.NumRows(), vals, defLevelsOut, nil) 176 require.NoError(t, err) 177 178 nulls := 0 179 for j := 0; j < arr.Len(); j++ { 180 if arr.IsNull(j) { 181 nulls++ 182 continue 183 } 184 185 switch v := arr.(type) { 186 case *array.Date32: 187 assert.EqualValues(t, v.Value(j), vals[j-nulls]) 188 case *array.Time32: 189 assert.EqualValues(t, v.Value(j), vals[j-nulls]) 190 } 191 } 192 case 64: 193 colReader := rgr.Column(i).(*file.Int64ColumnChunkReader) 194 vals := make([]int64, int(expected.NumRows())) 195 total, read, err = colReader.ReadBatch(expected.NumRows(), vals, defLevelsOut, nil) 196 require.NoError(t, err) 197 198 nulls := 0 199 for j := 0; j < arr.Len(); j++ { 200 if arr.IsNull(j) { 201 nulls++ 202 continue 203 } 204 205 switch v := arr.(type) { 206 case *array.Date64: 207 assert.EqualValues(t, v.Value(j), vals[j-nulls]) 208 case *array.Time64: 209 assert.EqualValues(t, v.Value(j), vals[j-nulls]) 210 case *array.Timestamp: 211 assert.EqualValues(t, v.Value(j), vals[j-nulls]) 212 } 213 } 214 } 215 assert.EqualValues(t, expected.NumRows(), total) 216 assert.EqualValues(t, expected.NumRows()-1, read) 217 assert.Equal(t, []int16{1, 1, 1, 0, 1, 1}, defLevelsOut) 218 } 219 } 220 221 func TestWriteArrowInt96(t *testing.T) { 222 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 223 defer mem.AssertSize(t, 0) 224 225 tbl := makeDateTimeTypesTable(mem, false, false) 226 defer tbl.Release() 227 228 props := pqarrow.NewArrowWriterProperties(pqarrow.WithDeprecatedInt96Timestamps(true)) 229 psc, err := pqarrow.ToParquet(tbl.Schema(), nil, props) 230 require.NoError(t, err) 231 232 manifest, err := pqarrow.NewSchemaManifest(psc, nil, nil) 233 require.NoError(t, err) 234 235 sink := encoding.NewBufferWriter(0, memory.DefaultAllocator) 236 writer := file.NewParquetWriter(sink, psc.Root()) 237 238 srgw := writer.AppendRowGroup() 239 ctx := pqarrow.NewArrowWriteContext(context.TODO(), &props) 240 241 for i := int64(0); i < tbl.NumCols(); i++ { 242 acw, err := pqarrow.NewArrowColumnWriter(tbl.Column(int(i)).Data(), 0, tbl.NumRows(), manifest, srgw, int(i)) 243 require.NoError(t, err) 244 require.NoError(t, acw.Write(ctx)) 245 } 246 require.NoError(t, srgw.Close()) 247 require.NoError(t, writer.Close()) 248 249 expected := makeDateTimeTypesTable(mem, false, false) 250 defer expected.Release() 251 252 reader, err := file.NewParquetReader(bytes.NewReader(sink.Bytes())) 253 require.NoError(t, err) 254 255 assert.EqualValues(t, expected.NumCols(), reader.MetaData().Schema.NumColumns()) 256 assert.EqualValues(t, expected.NumRows(), reader.NumRows()) 257 assert.EqualValues(t, 1, reader.NumRowGroups()) 258 259 rgr := reader.RowGroup(0) 260 tsRdr := rgr.Column(3) 261 assert.Equal(t, parquet.Types.Int96, tsRdr.Type()) 262 263 rdr := tsRdr.(*file.Int96ColumnChunkReader) 264 vals := make([]parquet.Int96, expected.NumRows()) 265 defLevels := make([]int16, int(expected.NumRows())) 266 267 total, read, _ := rdr.ReadBatch(expected.NumRows(), vals, defLevels, nil) 268 assert.EqualValues(t, expected.NumRows(), total) 269 assert.EqualValues(t, expected.NumRows()-1, read) 270 assert.Equal(t, []int16{1, 1, 1, 0, 1, 1}, defLevels) 271 272 data := expected.Column(3).Data().Chunk(0).(*array.Timestamp) 273 assert.EqualValues(t, data.Value(0), vals[0].ToTime().UnixNano()) 274 assert.EqualValues(t, data.Value(1), vals[1].ToTime().UnixNano()) 275 assert.EqualValues(t, data.Value(2), vals[2].ToTime().UnixNano()) 276 assert.EqualValues(t, data.Value(4), vals[3].ToTime().UnixNano()) 277 assert.EqualValues(t, data.Value(5), vals[4].ToTime().UnixNano()) 278 } 279 280 func writeTableToBuffer(t *testing.T, tbl arrow.Table, rowGroupSize int64, props pqarrow.ArrowWriterProperties) *memory.Buffer { 281 sink := encoding.NewBufferWriter(0, memory.DefaultAllocator) 282 wrprops := parquet.NewWriterProperties(parquet.WithVersion(parquet.V1_0)) 283 psc, err := pqarrow.ToParquet(tbl.Schema(), wrprops, props) 284 require.NoError(t, err) 285 286 manifest, err := pqarrow.NewSchemaManifest(psc, nil, nil) 287 require.NoError(t, err) 288 289 writer := file.NewParquetWriter(sink, psc.Root(), file.WithWriterProps(wrprops)) 290 ctx := pqarrow.NewArrowWriteContext(context.TODO(), &props) 291 292 offset := int64(0) 293 for offset < tbl.NumRows() { 294 sz := utils.Min(rowGroupSize, tbl.NumRows()-offset) 295 srgw := writer.AppendRowGroup() 296 for i := 0; i < int(tbl.NumCols()); i++ { 297 col := tbl.Column(i) 298 acw, err := pqarrow.NewArrowColumnWriter(col.Data(), offset, sz, manifest, srgw, i) 299 require.NoError(t, err) 300 require.NoError(t, acw.Write(ctx)) 301 } 302 srgw.Close() 303 offset += sz 304 } 305 writer.Close() 306 307 return sink.Finish() 308 } 309 310 func simpleRoundTrip(t *testing.T, tbl arrow.Table, rowGroupSize int64) { 311 buf := writeTableToBuffer(t, tbl, rowGroupSize, pqarrow.DefaultWriterProps()) 312 defer buf.Release() 313 314 rdr, err := file.NewParquetReader(bytes.NewReader(buf.Bytes())) 315 require.NoError(t, err) 316 317 ardr, err := pqarrow.NewFileReader(rdr, pqarrow.ArrowReadProperties{}, memory.DefaultAllocator) 318 require.NoError(t, err) 319 320 for i := 0; i < int(tbl.NumCols()); i++ { 321 crdr, err := ardr.GetColumn(context.TODO(), i) 322 require.NoError(t, err) 323 324 chunked, err := crdr.NextBatch(tbl.NumRows()) 325 require.NoError(t, err) 326 327 require.EqualValues(t, tbl.NumRows(), chunked.Len()) 328 329 chunkList := tbl.Column(i).Data().Chunks() 330 offset := int64(0) 331 for _, chnk := range chunkList { 332 slc := array.NewChunkedSlice(chunked, offset, offset+int64(chnk.Len())) 333 defer slc.Release() 334 335 assert.EqualValues(t, chnk.Len(), slc.Len()) 336 if len(slc.Chunks()) == 1 { 337 offset += int64(chnk.Len()) 338 assert.True(t, array.ArrayEqual(chnk, slc.Chunk(0))) 339 } 340 } 341 } 342 } 343 344 func TestArrowReadWriteTableChunkedCols(t *testing.T) { 345 chunkSizes := []int{2, 4, 10, 2} 346 const totalLen = int64(18) 347 348 rng := testutils.NewRandomArrayGenerator(0) 349 350 arr := rng.Int32(totalLen, 0, math.MaxInt32/2, 0.9) 351 defer arr.Release() 352 353 offset := int64(0) 354 chunks := make([]arrow.Array, 0) 355 for _, chnksize := range chunkSizes { 356 chk := array.NewSlice(arr, offset, offset+int64(chnksize)) 357 defer chk.Release() 358 chunks = append(chunks, chk) 359 } 360 361 sc := arrow.NewSchema([]arrow.Field{{Name: "field", Type: arr.DataType(), Nullable: true}}, nil) 362 tbl := array.NewTable(sc, []arrow.Column{*arrow.NewColumn(sc.Field(0), arrow.NewChunked(arr.DataType(), chunks))}, -1) 363 defer tbl.Release() 364 365 simpleRoundTrip(t, tbl, 2) 366 simpleRoundTrip(t, tbl, 10) 367 } 368 369 // set this up for checking our expected results so we can test the functions 370 // that generate them which we export 371 func getLogicalType(typ arrow.DataType) schema.LogicalType { 372 switch typ.ID() { 373 case arrow.INT8: 374 return schema.NewIntLogicalType(8, true) 375 case arrow.UINT8: 376 return schema.NewIntLogicalType(8, false) 377 case arrow.INT16: 378 return schema.NewIntLogicalType(16, true) 379 case arrow.UINT16: 380 return schema.NewIntLogicalType(16, false) 381 case arrow.INT32: 382 return schema.NewIntLogicalType(32, true) 383 case arrow.UINT32: 384 return schema.NewIntLogicalType(32, false) 385 case arrow.INT64: 386 return schema.NewIntLogicalType(64, true) 387 case arrow.UINT64: 388 return schema.NewIntLogicalType(64, false) 389 case arrow.STRING: 390 return schema.StringLogicalType{} 391 case arrow.DATE32: 392 return schema.DateLogicalType{} 393 case arrow.DATE64: 394 return schema.DateLogicalType{} 395 case arrow.TIMESTAMP: 396 ts := typ.(*arrow.TimestampType) 397 adjustedUTC := len(ts.TimeZone) == 0 398 switch ts.Unit { 399 case arrow.Microsecond: 400 return schema.NewTimestampLogicalType(adjustedUTC, schema.TimeUnitMicros) 401 case arrow.Millisecond: 402 return schema.NewTimestampLogicalType(adjustedUTC, schema.TimeUnitMillis) 403 case arrow.Nanosecond: 404 return schema.NewTimestampLogicalType(adjustedUTC, schema.TimeUnitNanos) 405 default: 406 panic("only milli, micro and nano units supported for arrow timestamp") 407 } 408 case arrow.TIME32: 409 return schema.NewTimeLogicalType(false, schema.TimeUnitMillis) 410 case arrow.TIME64: 411 ts := typ.(*arrow.Time64Type) 412 switch ts.Unit { 413 case arrow.Microsecond: 414 return schema.NewTimeLogicalType(false, schema.TimeUnitMicros) 415 case arrow.Nanosecond: 416 return schema.NewTimeLogicalType(false, schema.TimeUnitNanos) 417 default: 418 panic("only micro and nano seconds are supported for arrow TIME64") 419 } 420 case arrow.DECIMAL: 421 dec := typ.(*arrow.Decimal128Type) 422 return schema.NewDecimalLogicalType(dec.Precision, dec.Scale) 423 } 424 return schema.NoLogicalType{} 425 } 426 427 func getPhysicalType(typ arrow.DataType) parquet.Type { 428 switch typ.ID() { 429 case arrow.BOOL: 430 return parquet.Types.Boolean 431 case arrow.UINT8, arrow.INT8, arrow.UINT16, arrow.INT16, arrow.UINT32, arrow.INT32: 432 return parquet.Types.Int32 433 case arrow.INT64, arrow.UINT64: 434 return parquet.Types.Int64 435 case arrow.FLOAT32: 436 return parquet.Types.Float 437 case arrow.FLOAT64: 438 return parquet.Types.Double 439 case arrow.BINARY, arrow.STRING: 440 return parquet.Types.ByteArray 441 case arrow.FIXED_SIZE_BINARY, arrow.DECIMAL: 442 return parquet.Types.FixedLenByteArray 443 case arrow.DATE32: 444 return parquet.Types.Int32 445 case arrow.DATE64: 446 // convert to date32 internally 447 return parquet.Types.Int32 448 case arrow.TIME32: 449 return parquet.Types.Int32 450 case arrow.TIME64, arrow.TIMESTAMP: 451 return parquet.Types.Int64 452 default: 453 return parquet.Types.Int32 454 } 455 } 456 457 const ( 458 boolTestValue = true 459 uint8TestVal = uint8(64) 460 int8TestVal = int8(-64) 461 uint16TestVal = uint16(1024) 462 int16TestVal = int16(-1024) 463 uint32TestVal = uint32(1024) 464 int32TestVal = int32(-1024) 465 uint64TestVal = uint64(1024) 466 int64TestVal = int64(-1024) 467 tsTestValue = arrow.Timestamp(14695634030000) 468 date32TestVal = arrow.Date32(170000) 469 floatTestVal = float32(2.1) 470 doubleTestVal = float64(4.2) 471 strTestVal = "Test" 472 473 smallSize = 100 474 ) 475 476 type ParquetIOTestSuite struct { 477 suite.Suite 478 } 479 480 func (ps *ParquetIOTestSuite) makeSimpleSchema(typ arrow.DataType, rep parquet.Repetition) *schema.GroupNode { 481 byteWidth := int32(-1) 482 483 switch typ := typ.(type) { 484 case *arrow.FixedSizeBinaryType: 485 byteWidth = int32(typ.ByteWidth) 486 case *arrow.Decimal128Type: 487 byteWidth = pqarrow.DecimalSize(typ.Precision) 488 } 489 490 pnode, _ := schema.NewPrimitiveNodeLogical("column1", rep, getLogicalType(typ), getPhysicalType(typ), int(byteWidth), -1) 491 return schema.MustGroup(schema.NewGroupNode("schema", parquet.Repetitions.Required, schema.FieldList{pnode}, -1)) 492 } 493 494 func (ps *ParquetIOTestSuite) makePrimitiveTestCol(size int, typ arrow.DataType) arrow.Array { 495 switch typ.ID() { 496 case arrow.BOOL: 497 bldr := array.NewBooleanBuilder(memory.DefaultAllocator) 498 defer bldr.Release() 499 for i := 0; i < size; i++ { 500 bldr.Append(boolTestValue) 501 } 502 return bldr.NewArray() 503 case arrow.INT8: 504 bldr := array.NewInt8Builder(memory.DefaultAllocator) 505 defer bldr.Release() 506 for i := 0; i < size; i++ { 507 bldr.Append(int8TestVal) 508 } 509 return bldr.NewArray() 510 case arrow.UINT8: 511 bldr := array.NewUint8Builder(memory.DefaultAllocator) 512 defer bldr.Release() 513 for i := 0; i < size; i++ { 514 bldr.Append(uint8TestVal) 515 } 516 return bldr.NewArray() 517 case arrow.INT16: 518 bldr := array.NewInt16Builder(memory.DefaultAllocator) 519 defer bldr.Release() 520 for i := 0; i < size; i++ { 521 bldr.Append(int16TestVal) 522 } 523 return bldr.NewArray() 524 case arrow.UINT16: 525 bldr := array.NewUint16Builder(memory.DefaultAllocator) 526 defer bldr.Release() 527 for i := 0; i < size; i++ { 528 bldr.Append(uint16TestVal) 529 } 530 return bldr.NewArray() 531 case arrow.INT32: 532 bldr := array.NewInt32Builder(memory.DefaultAllocator) 533 defer bldr.Release() 534 for i := 0; i < size; i++ { 535 bldr.Append(int32TestVal) 536 } 537 return bldr.NewArray() 538 case arrow.UINT32: 539 bldr := array.NewUint32Builder(memory.DefaultAllocator) 540 defer bldr.Release() 541 for i := 0; i < size; i++ { 542 bldr.Append(uint32TestVal) 543 } 544 return bldr.NewArray() 545 case arrow.INT64: 546 bldr := array.NewInt64Builder(memory.DefaultAllocator) 547 defer bldr.Release() 548 for i := 0; i < size; i++ { 549 bldr.Append(int64TestVal) 550 } 551 return bldr.NewArray() 552 case arrow.UINT64: 553 bldr := array.NewUint64Builder(memory.DefaultAllocator) 554 defer bldr.Release() 555 for i := 0; i < size; i++ { 556 bldr.Append(uint64TestVal) 557 } 558 return bldr.NewArray() 559 case arrow.FLOAT32: 560 bldr := array.NewFloat32Builder(memory.DefaultAllocator) 561 defer bldr.Release() 562 for i := 0; i < size; i++ { 563 bldr.Append(floatTestVal) 564 } 565 return bldr.NewArray() 566 case arrow.FLOAT64: 567 bldr := array.NewFloat64Builder(memory.DefaultAllocator) 568 defer bldr.Release() 569 for i := 0; i < size; i++ { 570 bldr.Append(doubleTestVal) 571 } 572 return bldr.NewArray() 573 } 574 return nil 575 } 576 577 func (ps *ParquetIOTestSuite) makeTestFile(typ arrow.DataType, arr arrow.Array, numChunks int) []byte { 578 sc := ps.makeSimpleSchema(typ, parquet.Repetitions.Required) 579 sink := encoding.NewBufferWriter(0, memory.DefaultAllocator) 580 writer := file.NewParquetWriter(sink, sc) 581 582 ctx := pqarrow.NewArrowWriteContext(context.TODO(), nil) 583 rowGroupSize := arr.Len() / numChunks 584 585 for i := 0; i < numChunks; i++ { 586 rgw := writer.AppendRowGroup() 587 cw, err := rgw.NextColumn() 588 ps.NoError(err) 589 590 start := i * rowGroupSize 591 ps.NoError(pqarrow.WriteArrowToColumn(ctx, cw, array.NewSlice(arr, int64(start), int64(start+rowGroupSize)), nil, nil, false)) 592 cw.Close() 593 rgw.Close() 594 } 595 writer.Close() 596 buf := sink.Finish() 597 defer buf.Release() 598 return buf.Bytes() 599 } 600 601 func (ps *ParquetIOTestSuite) createReader(data []byte) *pqarrow.FileReader { 602 rdr, err := file.NewParquetReader(bytes.NewReader(data)) 603 ps.NoError(err) 604 605 reader, err := pqarrow.NewFileReader(rdr, pqarrow.ArrowReadProperties{}, memory.DefaultAllocator) 606 ps.NoError(err) 607 return reader 608 } 609 610 func (ps *ParquetIOTestSuite) readTable(rdr *pqarrow.FileReader) arrow.Table { 611 tbl, err := rdr.ReadTable(context.TODO()) 612 ps.NoError(err) 613 ps.NotNil(tbl) 614 return tbl 615 } 616 617 func (ps *ParquetIOTestSuite) checkSingleColumnRequiredTableRead(typ arrow.DataType, numChunks int) { 618 values := ps.makePrimitiveTestCol(smallSize, typ) 619 defer values.Release() 620 621 data := ps.makeTestFile(typ, values, numChunks) 622 reader := ps.createReader(data) 623 624 tbl := ps.readTable(reader) 625 defer tbl.Release() 626 627 ps.EqualValues(1, tbl.NumCols()) 628 ps.EqualValues(smallSize, tbl.NumRows()) 629 630 chunked := tbl.Column(0).Data() 631 ps.Len(chunked.Chunks(), 1) 632 ps.True(array.ArrayEqual(values, chunked.Chunk(0))) 633 } 634 635 func (ps *ParquetIOTestSuite) checkSingleColumnRead(typ arrow.DataType, numChunks int) { 636 values := ps.makePrimitiveTestCol(smallSize, typ) 637 defer values.Release() 638 639 data := ps.makeTestFile(typ, values, numChunks) 640 reader := ps.createReader(data) 641 642 cr, err := reader.GetColumn(context.TODO(), 0) 643 ps.NoError(err) 644 645 chunked, err := cr.NextBatch(smallSize) 646 ps.NoError(err) 647 defer chunked.Release() 648 649 ps.Len(chunked.Chunks(), 1) 650 ps.True(array.ArrayEqual(values, chunked.Chunk(0))) 651 } 652 653 func (ps *ParquetIOTestSuite) TestDateTimeTypesReadWriteTable() { 654 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 655 defer mem.AssertSize(ps.T(), 0) 656 657 toWrite := makeDateTimeTypesTable(mem, false, true) 658 defer toWrite.Release() 659 buf := writeTableToBuffer(ps.T(), toWrite, toWrite.NumRows(), pqarrow.DefaultWriterProps()) 660 defer buf.Release() 661 662 reader := ps.createReader(buf.Bytes()) 663 tbl := ps.readTable(reader) 664 defer tbl.Release() 665 666 expected := makeDateTimeTypesTable(mem, true, true) 667 defer expected.Release() 668 669 ps.Equal(expected.NumCols(), tbl.NumCols()) 670 ps.Equal(expected.NumRows(), tbl.NumRows()) 671 ps.Truef(expected.Schema().Equal(tbl.Schema()), "expected schema: %s\ngot schema: %s", expected.Schema(), tbl.Schema()) 672 673 for i := 0; i < int(expected.NumCols()); i++ { 674 exChunk := expected.Column(i).Data() 675 tblChunk := tbl.Column(i).Data() 676 677 ps.Equal(len(exChunk.Chunks()), len(tblChunk.Chunks())) 678 ps.Truef(array.ArrayEqual(exChunk.Chunk(0), tblChunk.Chunk(0)), "expected %s\ngot %s", exChunk.Chunk(0), tblChunk.Chunk(0)) 679 } 680 } 681 682 func (ps *ParquetIOTestSuite) TestDateTimeTypesWithInt96ReadWriteTable() { 683 mem := memory.NewCheckedAllocator(memory.DefaultAllocator) 684 defer mem.AssertSize(ps.T(), 0) 685 686 expected := makeDateTimeTypesTable(mem, false, true) 687 defer expected.Release() 688 buf := writeTableToBuffer(ps.T(), expected, expected.NumRows(), pqarrow.NewArrowWriterProperties(pqarrow.WithDeprecatedInt96Timestamps(true))) 689 defer buf.Release() 690 691 reader := ps.createReader(buf.Bytes()) 692 tbl := ps.readTable(reader) 693 defer tbl.Release() 694 695 ps.Equal(expected.NumCols(), tbl.NumCols()) 696 ps.Equal(expected.NumRows(), tbl.NumRows()) 697 ps.Truef(expected.Schema().Equal(tbl.Schema()), "expected schema: %s\ngot schema: %s", expected.Schema(), tbl.Schema()) 698 699 for i := 0; i < int(expected.NumCols()); i++ { 700 exChunk := expected.Column(i).Data() 701 tblChunk := tbl.Column(i).Data() 702 703 ps.Equal(len(exChunk.Chunks()), len(tblChunk.Chunks())) 704 ps.Truef(array.ArrayEqual(exChunk.Chunk(0), tblChunk.Chunk(0)), "expected %s\ngot %s", exChunk.Chunk(0), tblChunk.Chunk(0)) 705 } 706 } 707 708 func (ps *ParquetIOTestSuite) TestReadSingleColumnFile() { 709 types := []arrow.DataType{ 710 arrow.FixedWidthTypes.Boolean, 711 arrow.PrimitiveTypes.Uint8, 712 arrow.PrimitiveTypes.Int8, 713 arrow.PrimitiveTypes.Uint16, 714 arrow.PrimitiveTypes.Int16, 715 arrow.PrimitiveTypes.Uint32, 716 arrow.PrimitiveTypes.Int32, 717 arrow.PrimitiveTypes.Uint64, 718 arrow.PrimitiveTypes.Int64, 719 arrow.PrimitiveTypes.Float32, 720 arrow.PrimitiveTypes.Float64, 721 } 722 723 nchunks := []int{1, 4} 724 725 for _, n := range nchunks { 726 for _, dt := range types { 727 ps.Run(fmt.Sprintf("%s %d chunks", dt.Name(), n), func() { 728 ps.checkSingleColumnRead(dt, n) 729 }) 730 } 731 } 732 } 733 734 func (ps *ParquetIOTestSuite) TestSingleColumnRequiredRead() { 735 types := []arrow.DataType{ 736 arrow.FixedWidthTypes.Boolean, 737 arrow.PrimitiveTypes.Uint8, 738 arrow.PrimitiveTypes.Int8, 739 arrow.PrimitiveTypes.Uint16, 740 arrow.PrimitiveTypes.Int16, 741 arrow.PrimitiveTypes.Uint32, 742 arrow.PrimitiveTypes.Int32, 743 arrow.PrimitiveTypes.Uint64, 744 arrow.PrimitiveTypes.Int64, 745 arrow.PrimitiveTypes.Float32, 746 arrow.PrimitiveTypes.Float64, 747 } 748 749 nchunks := []int{1, 4} 750 751 for _, n := range nchunks { 752 for _, dt := range types { 753 ps.Run(fmt.Sprintf("%s %d chunks", dt.Name(), n), func() { 754 ps.checkSingleColumnRequiredTableRead(dt, n) 755 }) 756 } 757 } 758 } 759 760 func (ps *ParquetIOTestSuite) TestReadDecimals() { 761 bigEndian := []parquet.ByteArray{ 762 // 123456 763 []byte{1, 226, 64}, 764 // 987654 765 []byte{15, 18, 6}, 766 // -123456 767 []byte{255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 255, 254, 29, 192}, 768 } 769 770 bldr := array.NewDecimal128Builder(memory.DefaultAllocator, &arrow.Decimal128Type{Precision: 6, Scale: 3}) 771 defer bldr.Release() 772 773 bldr.Append(decimal128.FromU64(123456)) 774 bldr.Append(decimal128.FromU64(987654)) 775 bldr.Append(decimal128.FromI64(-123456)) 776 777 expected := bldr.NewDecimal128Array() 778 defer expected.Release() 779 780 sc := schema.MustGroup(schema.NewGroupNode("schema", parquet.Repetitions.Required, schema.FieldList{ 781 schema.Must(schema.NewPrimitiveNodeLogical("decimals", parquet.Repetitions.Required, schema.NewDecimalLogicalType(6, 3), parquet.Types.ByteArray, -1, -1)), 782 }, -1)) 783 784 sink := encoding.NewBufferWriter(0, memory.DefaultAllocator) 785 writer := file.NewParquetWriter(sink, sc) 786 787 rgw := writer.AppendRowGroup() 788 cw, _ := rgw.NextColumn() 789 cw.(*file.ByteArrayColumnChunkWriter).WriteBatch(bigEndian, nil, nil) 790 cw.Close() 791 rgw.Close() 792 writer.Close() 793 794 rdr := ps.createReader(sink.Bytes()) 795 cr, err := rdr.GetColumn(context.TODO(), 0) 796 ps.NoError(err) 797 798 chunked, err := cr.NextBatch(smallSize) 799 ps.NoError(err) 800 defer chunked.Release() 801 802 ps.Len(chunked.Chunks(), 1) 803 ps.True(array.ArrayEqual(expected, chunked.Chunk(0))) 804 } 805 806 func (ps *ParquetIOTestSuite) writeColumn(sc *schema.GroupNode, values arrow.Array) []byte { 807 var buf bytes.Buffer 808 arrsc, err := pqarrow.FromParquet(schema.NewSchema(sc), nil, nil) 809 ps.NoError(err) 810 811 writer, err := pqarrow.NewFileWriter(arrsc, &buf, parquet.NewWriterProperties(parquet.WithDictionaryDefault(false)), pqarrow.DefaultWriterProps()) 812 ps.NoError(err) 813 814 writer.NewRowGroup() 815 ps.NoError(writer.WriteColumnData(values)) 816 ps.NoError(writer.Close()) 817 ps.NoError(writer.Close()) 818 819 return buf.Bytes() 820 } 821 822 func (ps *ParquetIOTestSuite) readAndCheckSingleColumnFile(data []byte, values arrow.Array) { 823 reader := ps.createReader(data) 824 cr, err := reader.GetColumn(context.TODO(), 0) 825 ps.NoError(err) 826 ps.NotNil(cr) 827 828 chunked, err := cr.NextBatch(smallSize) 829 ps.NoError(err) 830 defer chunked.Release() 831 832 ps.Len(chunked.Chunks(), 1) 833 ps.NotNil(chunked.Chunk(0)) 834 835 ps.True(array.ArrayEqual(values, chunked.Chunk(0))) 836 } 837 838 var fullTypeList = []arrow.DataType{ 839 arrow.FixedWidthTypes.Boolean, 840 arrow.PrimitiveTypes.Uint8, 841 arrow.PrimitiveTypes.Int8, 842 arrow.PrimitiveTypes.Uint16, 843 arrow.PrimitiveTypes.Int16, 844 arrow.PrimitiveTypes.Uint32, 845 arrow.PrimitiveTypes.Int32, 846 arrow.PrimitiveTypes.Uint64, 847 arrow.PrimitiveTypes.Int64, 848 arrow.FixedWidthTypes.Date32, 849 arrow.PrimitiveTypes.Float32, 850 arrow.PrimitiveTypes.Float64, 851 arrow.BinaryTypes.String, 852 arrow.BinaryTypes.Binary, 853 &arrow.FixedSizeBinaryType{ByteWidth: 10}, 854 &arrow.Decimal128Type{Precision: 1, Scale: 0}, 855 &arrow.Decimal128Type{Precision: 5, Scale: 4}, 856 &arrow.Decimal128Type{Precision: 10, Scale: 9}, 857 &arrow.Decimal128Type{Precision: 19, Scale: 18}, 858 &arrow.Decimal128Type{Precision: 23, Scale: 22}, 859 &arrow.Decimal128Type{Precision: 27, Scale: 26}, 860 &arrow.Decimal128Type{Precision: 38, Scale: 37}, 861 } 862 863 func (ps *ParquetIOTestSuite) TestSingleColumnRequiredWrite() { 864 for _, dt := range fullTypeList { 865 ps.Run(dt.Name(), func() { 866 values := testutils.RandomNonNull(dt, smallSize) 867 sc := ps.makeSimpleSchema(dt, parquet.Repetitions.Required) 868 data := ps.writeColumn(sc, values) 869 ps.readAndCheckSingleColumnFile(data, values) 870 }) 871 } 872 } 873 874 func (ps *ParquetIOTestSuite) roundTripTable(expected arrow.Table, storeSchema bool) { 875 var buf bytes.Buffer 876 var props pqarrow.ArrowWriterProperties 877 if storeSchema { 878 props = pqarrow.NewArrowWriterProperties(pqarrow.WithStoreSchema()) 879 } else { 880 props = pqarrow.DefaultWriterProps() 881 } 882 883 ps.Require().NoError(pqarrow.WriteTable(expected, &buf, expected.NumRows(), nil, props)) 884 885 reader := ps.createReader(buf.Bytes()) 886 tbl := ps.readTable(reader) 887 defer tbl.Release() 888 889 ps.Equal(expected.NumCols(), tbl.NumCols()) 890 ps.Equal(expected.NumRows(), tbl.NumRows()) 891 892 exChunk := expected.Column(0).Data() 893 tblChunk := tbl.Column(0).Data() 894 895 ps.Equal(len(exChunk.Chunks()), len(tblChunk.Chunks())) 896 if exChunk.DataType().ID() != arrow.STRUCT { 897 ps.Truef(array.ArrayEqual(exChunk.Chunk(0), tblChunk.Chunk(0)), "expected: %s\ngot: %s", exChunk.Chunk(0), tblChunk.Chunk(0)) 898 } else { 899 // current impl of ArrayEquals for structs doesn't correctly handle nulls in the parent 900 // with a non-nullable child when comparing. Since after the round trip, the data in the 901 // child will have the nulls, not the original data. 902 ex := exChunk.Chunk(0) 903 tb := tblChunk.Chunk(0) 904 ps.Equal(ex.NullN(), tb.NullN()) 905 if ex.NullN() > 0 { 906 ps.Equal(ex.NullBitmapBytes()[:int(bitutil.BytesForBits(int64(ex.Len())))], tb.NullBitmapBytes()[:int(bitutil.BytesForBits(int64(tb.Len())))]) 907 } 908 ps.Equal(ex.Len(), tb.Len()) 909 // only compare the non-null values 910 ps.NoErrorf(utils.VisitSetBitRuns(ex.NullBitmapBytes(), int64(ex.Data().Offset()), int64(ex.Len()), func(pos, length int64) error { 911 if !ps.True(array.ArraySliceEqual(ex, pos, pos+length, tb, pos, pos+length)) { 912 return errors.New("failed") 913 } 914 return nil 915 }), "expected: %s\ngot: %s", ex, tb) 916 } 917 } 918 919 func makeEmptyListsArray(size int) arrow.Array { 920 // allocate an offsets buffer with only zeros 921 offsetsNbytes := arrow.Int32Traits.BytesRequired(size + 1) 922 offsetsBuffer := make([]byte, offsetsNbytes) 923 924 childBuffers := []*memory.Buffer{nil, nil} 925 childData := array.NewData(arrow.PrimitiveTypes.Float32, 0, childBuffers, nil, 0, 0) 926 defer childData.Release() 927 buffers := []*memory.Buffer{nil, memory.NewBufferBytes(offsetsBuffer)} 928 arrayData := array.NewData(arrow.ListOf(childData.DataType()), size, buffers, []arrow.ArrayData{childData}, 0, 0) 929 defer arrayData.Release() 930 return array.MakeFromData(arrayData) 931 } 932 933 func makeListArray(values arrow.Array, size, nullcount int) arrow.Array { 934 nonNullEntries := size - nullcount - 1 935 lengthPerEntry := values.Len() / nonNullEntries 936 937 offsets := make([]byte, arrow.Int32Traits.BytesRequired(size+1)) 938 offsetsArr := arrow.Int32Traits.CastFromBytes(offsets) 939 940 nullBitmap := make([]byte, int(bitutil.BytesForBits(int64(size)))) 941 942 curOffset := 0 943 for i := 0; i < size; i++ { 944 offsetsArr[i] = int32(curOffset) 945 if !(((i % 2) == 0) && ((i / 2) < nullcount)) { 946 // non-null list (list with index 1 is always empty) 947 bitutil.SetBit(nullBitmap, i) 948 if i != 1 { 949 curOffset += lengthPerEntry 950 } 951 } 952 } 953 offsetsArr[size] = int32(values.Len()) 954 955 listData := array.NewData(arrow.ListOf(values.DataType()), size, 956 []*memory.Buffer{memory.NewBufferBytes(nullBitmap), memory.NewBufferBytes(offsets)}, 957 []arrow.ArrayData{values.Data()}, nullcount, 0) 958 defer listData.Release() 959 return array.NewListData(listData) 960 } 961 962 func prepareEmptyListsTable(size int) arrow.Table { 963 lists := makeEmptyListsArray(size) 964 defer lists.Release() 965 chunked := arrow.NewChunked(lists.DataType(), []arrow.Array{lists}) 966 defer chunked.Release() 967 return makeSimpleTable(chunked, true) 968 } 969 970 func prepareListTable(dt arrow.DataType, size int, nullableLists bool, nullableElems bool, nullCount int) arrow.Table { 971 nc := nullCount 972 if !nullableElems { 973 nc = 0 974 } 975 values := testutils.RandomNullable(dt, size*size, nc) 976 defer values.Release() 977 // also test that slice offsets are respected 978 values = array.NewSlice(values, 5, int64(values.Len())) 979 defer values.Release() 980 981 if !nullableLists { 982 nullCount = 0 983 } 984 lists := makeListArray(values, size, nullCount) 985 defer lists.Release() 986 987 chunked := arrow.NewChunked(lists.DataType(), []arrow.Array{lists}) 988 defer chunked.Release() 989 990 return makeSimpleTable(array.NewChunkedSlice(chunked, 3, int64(size)), nullableLists) 991 } 992 993 func prepareListOfListTable(dt arrow.DataType, size, nullCount int, nullableParentLists, nullableLists, nullableElems bool) arrow.Table { 994 nc := nullCount 995 if !nullableElems { 996 nc = 0 997 } 998 999 values := testutils.RandomNullable(dt, size*6, nc) 1000 defer values.Release() 1001 1002 if nullableLists { 1003 nc = nullCount 1004 } else { 1005 nc = 0 1006 } 1007 1008 lists := makeListArray(values, size*3, nc) 1009 defer lists.Release() 1010 1011 if !nullableParentLists { 1012 nullCount = 0 1013 } 1014 1015 parentLists := makeListArray(lists, size, nullCount) 1016 defer parentLists.Release() 1017 1018 chunked := arrow.NewChunked(parentLists.DataType(), []arrow.Array{parentLists}) 1019 defer chunked.Release() 1020 1021 return makeSimpleTable(chunked, nullableParentLists) 1022 } 1023 1024 func (ps *ParquetIOTestSuite) TestSingleEmptyListsColumnReadWrite() { 1025 expected := prepareEmptyListsTable(smallSize) 1026 buf := writeTableToBuffer(ps.T(), expected, smallSize, pqarrow.DefaultWriterProps()) 1027 defer buf.Release() 1028 1029 reader := ps.createReader(buf.Bytes()) 1030 tbl := ps.readTable(reader) 1031 defer tbl.Release() 1032 1033 ps.EqualValues(expected.NumCols(), tbl.NumCols()) 1034 ps.EqualValues(expected.NumRows(), tbl.NumRows()) 1035 1036 exChunk := expected.Column(0).Data() 1037 tblChunk := tbl.Column(0).Data() 1038 1039 ps.Equal(len(exChunk.Chunks()), len(tblChunk.Chunks())) 1040 ps.True(array.ArrayEqual(exChunk.Chunk(0), tblChunk.Chunk(0))) 1041 } 1042 1043 func (ps *ParquetIOTestSuite) TestSingleColumnOptionalReadWrite() { 1044 for _, dt := range fullTypeList { 1045 ps.Run(dt.Name(), func() { 1046 values := testutils.RandomNullable(dt, smallSize, 10) 1047 sc := ps.makeSimpleSchema(dt, parquet.Repetitions.Optional) 1048 data := ps.writeColumn(sc, values) 1049 ps.readAndCheckSingleColumnFile(data, values) 1050 }) 1051 } 1052 } 1053 1054 func (ps *ParquetIOTestSuite) TestSingleNullableListNullableColumnReadWrite() { 1055 for _, dt := range fullTypeList { 1056 ps.Run(dt.Name(), func() { 1057 expected := prepareListTable(dt, smallSize, true, true, 10) 1058 defer expected.Release() 1059 ps.roundTripTable(expected, false) 1060 }) 1061 } 1062 } 1063 1064 func (ps *ParquetIOTestSuite) TestSingleRequiredListNullableColumnReadWrite() { 1065 for _, dt := range fullTypeList { 1066 ps.Run(dt.Name(), func() { 1067 expected := prepareListTable(dt, smallSize, false, true, 10) 1068 defer expected.Release() 1069 ps.roundTripTable(expected, false) 1070 }) 1071 } 1072 } 1073 1074 func (ps *ParquetIOTestSuite) TestSingleNullableListRequiredColumnReadWrite() { 1075 for _, dt := range fullTypeList { 1076 ps.Run(dt.Name(), func() { 1077 expected := prepareListTable(dt, smallSize, true, false, 10) 1078 defer expected.Release() 1079 ps.roundTripTable(expected, false) 1080 }) 1081 } 1082 } 1083 1084 func (ps *ParquetIOTestSuite) TestSingleRequiredListRequiredColumnReadWrite() { 1085 for _, dt := range fullTypeList { 1086 ps.Run(dt.Name(), func() { 1087 expected := prepareListTable(dt, smallSize, false, false, 0) 1088 defer expected.Release() 1089 ps.roundTripTable(expected, false) 1090 }) 1091 } 1092 } 1093 1094 func (ps *ParquetIOTestSuite) TestSingleNullableListRequiredListRequiredColumnReadWrite() { 1095 for _, dt := range fullTypeList { 1096 ps.Run(dt.Name(), func() { 1097 expected := prepareListOfListTable(dt, smallSize, 2, true, false, false) 1098 defer expected.Release() 1099 ps.roundTripTable(expected, false) 1100 }) 1101 } 1102 } 1103 1104 func (ps *ParquetIOTestSuite) TestSimpleStruct() { 1105 links := arrow.StructOf(arrow.Field{Name: "Backward", Type: arrow.PrimitiveTypes.Int64, Nullable: true}, 1106 arrow.Field{Name: "Forward", Type: arrow.PrimitiveTypes.Int64, Nullable: true}) 1107 1108 bldr := array.NewStructBuilder(memory.DefaultAllocator, links) 1109 defer bldr.Release() 1110 1111 backBldr := bldr.FieldBuilder(0).(*array.Int64Builder) 1112 forwardBldr := bldr.FieldBuilder(1).(*array.Int64Builder) 1113 1114 bldr.Append(true) 1115 backBldr.AppendNull() 1116 forwardBldr.Append(20) 1117 1118 bldr.Append(true) 1119 backBldr.Append(10) 1120 forwardBldr.Append(40) 1121 1122 data := bldr.NewArray() 1123 defer data.Release() 1124 1125 tbl := array.NewTable(arrow.NewSchema([]arrow.Field{{Name: "links", Type: links}}, nil), 1126 []arrow.Column{*arrow.NewColumn(arrow.Field{Name: "links", Type: links}, arrow.NewChunked(links, []arrow.Array{data}))}, -1) 1127 defer tbl.Release() 1128 1129 ps.roundTripTable(tbl, false) 1130 } 1131 1132 func (ps *ParquetIOTestSuite) TestSingleColumnNullableStruct() { 1133 links := arrow.StructOf(arrow.Field{Name: "Backward", Type: arrow.PrimitiveTypes.Int64, Nullable: true}) 1134 bldr := array.NewStructBuilder(memory.DefaultAllocator, links) 1135 defer bldr.Release() 1136 1137 backBldr := bldr.FieldBuilder(0).(*array.Int64Builder) 1138 1139 bldr.AppendNull() 1140 bldr.Append(true) 1141 backBldr.Append(10) 1142 1143 data := bldr.NewArray() 1144 defer data.Release() 1145 1146 tbl := array.NewTable(arrow.NewSchema([]arrow.Field{{Name: "links", Type: links, Nullable: true}}, nil), 1147 []arrow.Column{*arrow.NewColumn(arrow.Field{Name: "links", Type: links, Nullable: true}, arrow.NewChunked(links, []arrow.Array{data}))}, -1) 1148 defer tbl.Release() 1149 1150 ps.roundTripTable(tbl, false) 1151 } 1152 1153 func (ps *ParquetIOTestSuite) TestNestedRequiredFieldStruct() { 1154 intField := arrow.Field{Name: "int_array", Type: arrow.PrimitiveTypes.Int32} 1155 intBldr := array.NewInt32Builder(memory.DefaultAllocator) 1156 defer intBldr.Release() 1157 intBldr.AppendValues([]int32{0, 1, 2, 3, 4, 5, 7, 8}, nil) 1158 1159 intArr := intBldr.NewArray() 1160 defer intArr.Release() 1161 1162 validity := memory.NewBufferBytes([]byte{0xCC}) 1163 defer validity.Release() 1164 1165 structField := arrow.Field{Name: "root", Type: arrow.StructOf(intField), Nullable: true} 1166 structData := array.NewData(structField.Type, 8, []*memory.Buffer{validity}, []arrow.ArrayData{intArr.Data()}, 4, 0) 1167 defer structData.Release() 1168 stData := array.NewStructData(structData) 1169 defer stData.Release() 1170 1171 tbl := array.NewTable(arrow.NewSchema([]arrow.Field{structField}, nil), 1172 []arrow.Column{*arrow.NewColumn(structField, 1173 arrow.NewChunked(structField.Type, []arrow.Array{stData}))}, -1) 1174 defer tbl.Release() 1175 1176 ps.roundTripTable(tbl, false) 1177 } 1178 1179 func (ps *ParquetIOTestSuite) TestNestedNullableField() { 1180 intField := arrow.Field{Name: "int_array", Type: arrow.PrimitiveTypes.Int32, Nullable: true} 1181 intBldr := array.NewInt32Builder(memory.DefaultAllocator) 1182 defer intBldr.Release() 1183 intBldr.AppendValues([]int32{0, 1, 2, 3, 4, 5, 7, 8}, []bool{true, false, true, false, true, true, false, true}) 1184 1185 intArr := intBldr.NewArray() 1186 defer intArr.Release() 1187 1188 validity := memory.NewBufferBytes([]byte{0xCC}) 1189 defer validity.Release() 1190 1191 structField := arrow.Field{Name: "root", Type: arrow.StructOf(intField), Nullable: true} 1192 data := array.NewData(structField.Type, 8, []*memory.Buffer{validity}, []arrow.ArrayData{intArr.Data()}, 4, 0) 1193 defer data.Release() 1194 stData := array.NewStructData(data) 1195 defer stData.Release() 1196 1197 tbl := array.NewTable(arrow.NewSchema([]arrow.Field{structField}, nil), 1198 []arrow.Column{*arrow.NewColumn(structField, 1199 arrow.NewChunked(structField.Type, []arrow.Array{stData}))}, -1) 1200 defer tbl.Release() 1201 1202 ps.roundTripTable(tbl, false) 1203 } 1204 1205 func (ps *ParquetIOTestSuite) TestCanonicalNestedRoundTrip() { 1206 docIdField := arrow.Field{Name: "DocID", Type: arrow.PrimitiveTypes.Int64} 1207 linksField := arrow.Field{Name: "Links", Type: arrow.StructOf( 1208 arrow.Field{Name: "Backward", Type: arrow.ListOf(arrow.PrimitiveTypes.Int64)}, 1209 arrow.Field{Name: "Forward", Type: arrow.ListOf(arrow.PrimitiveTypes.Int64)}, 1210 ), Nullable: true} 1211 1212 nameStruct := arrow.StructOf( 1213 arrow.Field{Name: "Language", Nullable: true, Type: arrow.ListOf( 1214 arrow.StructOf(arrow.Field{Name: "Code", Type: arrow.BinaryTypes.String}, 1215 arrow.Field{Name: "Country", Type: arrow.BinaryTypes.String, Nullable: true}))}, 1216 arrow.Field{Name: "Url", Type: arrow.BinaryTypes.String, Nullable: true}) 1217 1218 nameField := arrow.Field{Name: "Name", Type: arrow.ListOf(nameStruct)} 1219 sc := arrow.NewSchema([]arrow.Field{docIdField, linksField, nameField}, nil) 1220 1221 docIDArr, _, err := array.FromJSON(memory.DefaultAllocator, docIdField.Type, strings.NewReader("[10, 20]")) 1222 ps.Require().NoError(err) 1223 defer docIDArr.Release() 1224 1225 linksIDArr, _, err := array.FromJSON(memory.DefaultAllocator, linksField.Type, strings.NewReader(`[{"Backward":[], "Forward":[20, 40, 60]}, {"Backward":[10, 30], "Forward": [80]}]`)) 1226 ps.Require().NoError(err) 1227 defer linksIDArr.Release() 1228 1229 nameArr, _, err := array.FromJSON(memory.DefaultAllocator, nameField.Type, strings.NewReader(` 1230 [[{"Language": [{"Code": "en_us", "Country": "us"}, 1231 {"Code": "en_us", "Country": null}], 1232 "Url": "http://A"}, 1233 {"Url": "http://B", "Language": null}, 1234 {"Language": [{"Code": "en-gb", "Country": "gb"}], "Url": null}], 1235 [{"Url": "http://C", "Language": null}]]`)) 1236 ps.Require().NoError(err) 1237 defer nameArr.Release() 1238 1239 expected := array.NewTable(sc, []arrow.Column{ 1240 *arrow.NewColumn(docIdField, arrow.NewChunked(docIdField.Type, []arrow.Array{docIDArr})), 1241 *arrow.NewColumn(linksField, arrow.NewChunked(linksField.Type, []arrow.Array{linksIDArr})), 1242 *arrow.NewColumn(nameField, arrow.NewChunked(nameField.Type, []arrow.Array{nameArr})), 1243 }, 2) 1244 1245 ps.roundTripTable(expected, false) 1246 } 1247 1248 func (ps *ParquetIOTestSuite) TestFixedSizeList() { 1249 bldr := array.NewFixedSizeListBuilder(memory.DefaultAllocator, 3, arrow.PrimitiveTypes.Int16) 1250 defer bldr.Release() 1251 1252 vb := bldr.ValueBuilder().(*array.Int16Builder) 1253 1254 bldr.AppendValues([]bool{true, true, true}) 1255 vb.AppendValues([]int16{1, 2, 3, 4, 5, 6, 7, 8, 9}, nil) 1256 1257 data := bldr.NewArray() 1258 field := arrow.Field{Name: "root", Type: data.DataType(), Nullable: true} 1259 expected := array.NewTable(arrow.NewSchema([]arrow.Field{field}, nil), 1260 []arrow.Column{*arrow.NewColumn(field, arrow.NewChunked(field.Type, []arrow.Array{data}))}, -1) 1261 1262 ps.roundTripTable(expected, true) 1263 } 1264 1265 func TestParquetArrowIO(t *testing.T) { 1266 suite.Run(t, new(ParquetIOTestSuite)) 1267 } 1268 1269 func TestBufferedRecWrite(t *testing.T) { 1270 sc := arrow.NewSchema([]arrow.Field{ 1271 {Name: "f32", Type: arrow.PrimitiveTypes.Float32, Nullable: true}, 1272 {Name: "i32", Type: arrow.PrimitiveTypes.Int32, Nullable: true}, 1273 {Name: "struct_i64_f64", Type: arrow.StructOf( 1274 arrow.Field{Name: "i64", Type: arrow.PrimitiveTypes.Int64, Nullable: true}, 1275 arrow.Field{Name: "f64", Type: arrow.PrimitiveTypes.Float64, Nullable: true})}, 1276 }, nil) 1277 1278 structData := array.NewData(sc.Field(2).Type, SIZELEN, 1279 []*memory.Buffer{nil, nil}, 1280 []arrow.ArrayData{testutils.RandomNullable(arrow.PrimitiveTypes.Int64, SIZELEN, 0).Data(), testutils.RandomNullable(arrow.PrimitiveTypes.Float64, SIZELEN, 0).Data()}, 0, 0) 1281 defer structData.Release() 1282 cols := []arrow.Array{ 1283 testutils.RandomNullable(sc.Field(0).Type, SIZELEN, SIZELEN/5), 1284 testutils.RandomNullable(sc.Field(1).Type, SIZELEN, SIZELEN/5), 1285 array.NewStructData(structData), 1286 } 1287 1288 rec := array.NewRecord(sc, cols, SIZELEN) 1289 defer rec.Release() 1290 1291 var ( 1292 buf bytes.Buffer 1293 ) 1294 1295 wr, err := pqarrow.NewFileWriter(sc, &buf, 1296 parquet.NewWriterProperties(parquet.WithCompression(compress.Codecs.Snappy), parquet.WithDictionaryDefault(false), parquet.WithDataPageSize(100*1024)), 1297 pqarrow.DefaultWriterProps()) 1298 require.NoError(t, err) 1299 1300 p1 := rec.NewSlice(0, SIZELEN/2) 1301 defer p1.Release() 1302 require.NoError(t, wr.WriteBuffered(p1)) 1303 1304 p2 := rec.NewSlice(SIZELEN/2, SIZELEN) 1305 defer p2.Release() 1306 require.NoError(t, wr.WriteBuffered(p2)) 1307 1308 wr.Close() 1309 1310 rdr, err := file.NewParquetReader(bytes.NewReader(buf.Bytes())) 1311 assert.NoError(t, err) 1312 1313 assert.EqualValues(t, 1, rdr.NumRowGroups()) 1314 assert.EqualValues(t, SIZELEN, rdr.NumRows()) 1315 rdr.Close() 1316 1317 tbl, err := pqarrow.ReadTable(context.Background(), bytes.NewReader(buf.Bytes()), nil, pqarrow.ArrowReadProperties{}, nil) 1318 assert.NoError(t, err) 1319 defer tbl.Release() 1320 1321 assert.EqualValues(t, SIZELEN, tbl.NumRows()) 1322 } 1323 1324 func (ps *ParquetIOTestSuite) TestArrowMapTypeRoundTrip() { 1325 bldr := array.NewMapBuilder(memory.DefaultAllocator, arrow.BinaryTypes.String, arrow.PrimitiveTypes.Int32, false) 1326 defer bldr.Release() 1327 1328 kb := bldr.KeyBuilder().(*array.StringBuilder) 1329 ib := bldr.ItemBuilder().(*array.Int32Builder) 1330 1331 bldr.Append(true) 1332 kb.AppendValues([]string{"Fee", "Fi", "Fo", "Fum"}, nil) 1333 ib.AppendValues([]int32{1, 2, 3, 4}, nil) 1334 1335 bldr.Append(true) 1336 kb.AppendValues([]string{"Fee", "Fi", "Fo"}, nil) 1337 ib.AppendValues([]int32{5, 4, 3}, nil) 1338 1339 bldr.AppendNull() 1340 1341 bldr.Append(true) 1342 kb.AppendValues([]string{"Fo", "Fi", "Fee"}, nil) 1343 ib.AppendValues([]int32{-1, 2, 3}, []bool{false, true, true}) 1344 1345 arr := bldr.NewArray() 1346 defer arr.Release() 1347 1348 fld := arrow.Field{Name: "mapped", Type: arr.DataType(), Nullable: true} 1349 tbl := array.NewTable(arrow.NewSchema([]arrow.Field{fld}, nil), 1350 []arrow.Column{*arrow.NewColumn(fld, arrow.NewChunked(arr.DataType(), []arrow.Array{arr}))}, -1) 1351 defer tbl.Release() 1352 1353 ps.roundTripTable(tbl, true) 1354 }