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